lindong28 commented on a change in pull request #28:
URL: https://github.com/apache/flink-ml/pull/28#discussion_r760717691



##########
File path: 
flink-ml-lib/src/main/java/org/apache/flink/ml/classification/linear/LogisticRegression.java
##########
@@ -0,0 +1,488 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.ml.classification.linear;
+
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.iteration.DataStreamList;
+import org.apache.flink.iteration.IterationBody;
+import org.apache.flink.iteration.IterationBodyResult;
+import org.apache.flink.iteration.IterationConfig;
+import org.apache.flink.iteration.IterationConfig.OperatorLifeCycle;
+import org.apache.flink.iteration.IterationListener;
+import org.apache.flink.iteration.Iterations;
+import org.apache.flink.iteration.ReplayableDataStreamList;
+import org.apache.flink.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.common.datastream.DataStreamUtils;
+import org.apache.flink.ml.common.feature.LabeledPointWithWeight;
+import org.apache.flink.ml.common.iteration.TerminateOnMaxIterOrTol;
+import org.apache.flink.ml.linalg.BLAS;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.internal.TableImpl;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.OutputTag;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.commons.collections.IteratorUtils;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+/**
+ * This class implements methods to train a logistic regression model. For 
details, see
+ * https://en.wikipedia.org/wiki/Logistic_regression.
+ */
+public class LogisticRegression
+        implements Estimator<LogisticRegression, LogisticRegressionModel>,
+                LogisticRegressionParams<LogisticRegression> {
+
+    private Map<Param<?>, Object> paramMap = new HashMap<>();
+
+    private static final OutputTag<LogisticRegressionModelData> MODEL_OUTPUT =
+            new OutputTag<LogisticRegressionModelData>("MODEL_OUTPUT") {};
+
+    private static final OutputTag<double[]> LOSS_OUTPUT =
+            new OutputTag<double[]>("LOSS_OUTPUT") {};
+
+    public LogisticRegression() {
+        ParamUtils.initializeMapWithDefaultValues(this.paramMap, this);
+    }
+
+    @Override
+    public Map<Param<?>, Object> getParamMap() {
+        return paramMap;
+    }
+
+    @Override
+    public void save(String path) throws IOException {
+        ReadWriteUtils.saveMetadata(this, path);
+    }
+
+    public static LogisticRegression load(StreamExecutionEnvironment env, 
String path)
+            throws IOException {
+        return ReadWriteUtils.loadStageParam(path);
+    }
+
+    @Override
+    @SuppressWarnings("rawTypes")
+    public LogisticRegressionModel fit(Table... inputs) {
+        Preconditions.checkArgument(inputs.length == 1);
+        Preconditions.checkArgument(
+                "auto".equals(getMultiClass()) || 
"bionomial".equals(getMultiClass()),
+                "Currently we only support binary classification.");

Review comment:
       nits: this error message is a bit "conversational". Could you change the 
error message to something more formal, similar to the error message in 
`DistanceMeasure.java`?
   
   Same for other error messages.

##########
File path: 
flink-ml-lib/src/main/java/org/apache/flink/ml/classification/linear/LogisticRegressionParams.java
##########
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.ml.classification.linear;
+
+import org.apache.flink.ml.common.param.HasFeaturesCol;
+import org.apache.flink.ml.common.param.HasGlobalBatchSize;
+import org.apache.flink.ml.common.param.HasLabelCol;
+import org.apache.flink.ml.common.param.HasLearningRate;
+import org.apache.flink.ml.common.param.HasMaxIter;
+import org.apache.flink.ml.common.param.HasMultiClass;
+import org.apache.flink.ml.common.param.HasReg;
+import org.apache.flink.ml.common.param.HasTol;
+import org.apache.flink.ml.common.param.HasWeightCol;
+
+/** Params for {@link LogisticRegression}. */
+public interface LogisticRegressionParams<T>
+        extends HasLabelCol<T>,
+                HasMaxIter<T>,
+                HasFeaturesCol<T>,

Review comment:
       `HasFeaturesCol` is already included in `LogisticRegressionModelParams`. 
Should we remove it here?

##########
File path: 
flink-ml-lib/src/main/java/org/apache/flink/ml/classification/linear/LogisticRegression.java
##########
@@ -0,0 +1,488 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.ml.classification.linear;
+
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.iteration.DataStreamList;
+import org.apache.flink.iteration.IterationBody;
+import org.apache.flink.iteration.IterationBodyResult;
+import org.apache.flink.iteration.IterationConfig;
+import org.apache.flink.iteration.IterationConfig.OperatorLifeCycle;
+import org.apache.flink.iteration.IterationListener;
+import org.apache.flink.iteration.Iterations;
+import org.apache.flink.iteration.ReplayableDataStreamList;
+import org.apache.flink.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.common.datastream.DataStreamUtils;
+import org.apache.flink.ml.common.feature.LabeledPointWithWeight;
+import org.apache.flink.ml.common.iteration.TerminateOnMaxIterOrTol;
+import org.apache.flink.ml.linalg.BLAS;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.internal.TableImpl;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.OutputTag;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.commons.collections.IteratorUtils;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+/**
+ * This class implements methods to train a logistic regression model. For 
details, see
+ * https://en.wikipedia.org/wiki/Logistic_regression.
+ */
+public class LogisticRegression
+        implements Estimator<LogisticRegression, LogisticRegressionModel>,
+                LogisticRegressionParams<LogisticRegression> {
+
+    private Map<Param<?>, Object> paramMap = new HashMap<>();
+
+    private static final OutputTag<LogisticRegressionModelData> MODEL_OUTPUT =

Review comment:
       I believe in general we don't use class private variable if there could 
be only one producer and one consumer of this value, and the value is not 
exposed to user of this class. This is what we do for `broadcastModelKey`.
   
   Could `MODEL_OUTPUT` be passed directly from `TrainIterationBody` to 
`CacheDataAndDoTrain`?
   
   Same for `LOSS_OUTPUT`.

##########
File path: 
flink-ml-core/src/main/java/org/apache/flink/ml/common/iteration/TerminateOnMaxIterOrTol.java
##########
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.ml.common.iteration;
+
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.iteration.IterationListener;
+import org.apache.flink.util.Collector;
+
+/**
+ * A FlatMapFunction that emits values iff the iteration's epochWatermark does 
not exceed a certain
+ * threshold and the loss does not exceed a certain tolerance.
+ *
+ * <p>When the output of this FlatMapFunction is used as the termination 
criteria of an iteration

Review comment:
       The statement here does not match the code. Could this be fixed?

##########
File path: 
flink-ml-lib/src/test/java/org/apache/flink/ml/classification/linear/LogisticRegressionTest.java
##########
@@ -0,0 +1,291 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.ml.classification.linear;
+
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import 
org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.types.Row;
+
+import org.apache.commons.collections.IteratorUtils;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/** Tests {@link LogisticRegression} and {@link LogisticRegressionModel}. */
+public class LogisticRegressionTest {
+
+    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+
+    private StreamExecutionEnvironment env;
+
+    private StreamTableEnvironment tEnv;
+
+    private static final List<Row> bionomialTrainData =
+            Arrays.asList(
+                    Row.of(new DenseVector(new double[] {1, 2, 3, 4}), 0., 1.),
+                    Row.of(new DenseVector(new double[] {2, 2, 3, 4}), 0., 2.),
+                    Row.of(new DenseVector(new double[] {3, 2, 3, 4}), 0., 3.),
+                    Row.of(new DenseVector(new double[] {4, 2, 3, 4}), 0., 4.),
+                    Row.of(new DenseVector(new double[] {5, 2, 3, 4}), 0., 5.),
+                    Row.of(new DenseVector(new double[] {11, 2, 3, 4}), 1., 
1.),
+                    Row.of(new DenseVector(new double[] {12, 2, 3, 4}), 1., 
2.),
+                    Row.of(new DenseVector(new double[] {13, 2, 3, 4}), 1., 
3.),
+                    Row.of(new DenseVector(new double[] {14, 2, 3, 4}), 1., 
4.),
+                    Row.of(new DenseVector(new double[] {15, 2, 3, 4}), 1., 
5.));
+
+    private static final List<Row> multinomialTrainData =
+            Arrays.asList(
+                    Row.of(new DenseVector(new double[] {1, 2, 3, 4}), 0., 1.),
+                    Row.of(new DenseVector(new double[] {2, 2, 3, 4}), 0., 2.),
+                    Row.of(new DenseVector(new double[] {3, 2, 3, 4}), 2., 3.),
+                    Row.of(new DenseVector(new double[] {4, 2, 3, 4}), 2., 4.),
+                    Row.of(new DenseVector(new double[] {5, 2, 3, 4}), 2., 5.),
+                    Row.of(new DenseVector(new double[] {11, 2, 3, 4}), 1., 
1.),
+                    Row.of(new DenseVector(new double[] {12, 2, 3, 4}), 1., 
2.),
+                    Row.of(new DenseVector(new double[] {13, 2, 3, 4}), 1., 
3.),
+                    Row.of(new DenseVector(new double[] {14, 2, 3, 4}), 1., 
4.),
+                    Row.of(new DenseVector(new double[] {15, 2, 3, 4}), 1., 
5.));
+
+    private static final double[] expectedCoefficient =
+            new double[] {0.528, -0.286, -0.429, -0.572};
+
+    private static final double TOLERANCE = 1e-6;
+
+    private Table bionomialDataTable;
+
+    private Table multinomialDataTable;
+
+    @Before
+    public void before() {
+        Configuration config = new Configuration();
+        
config.set(ExecutionCheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, 
true);
+        env = StreamExecutionEnvironment.getExecutionEnvironment(config);
+        env.setParallelism(4);
+        env.enableCheckpointing(100);
+        env.setRestartStrategy(RestartStrategies.noRestart());
+        tEnv = StreamTableEnvironment.create(env);
+        Collections.shuffle(bionomialTrainData);
+        bionomialDataTable =
+                tEnv.fromDataStream(
+                        env.fromCollection(
+                                bionomialTrainData,
+                                new RowTypeInfo(
+                                        new TypeInformation[] {
+                                            
TypeInformation.of(DenseVector.class),
+                                            Types.DOUBLE,
+                                            Types.DOUBLE
+                                        },
+                                        new String[] {"features", "label", 
"weight"})));
+        multinomialDataTable =
+                tEnv.fromDataStream(
+                        env.fromCollection(
+                                multinomialTrainData,
+                                new RowTypeInfo(
+                                        new TypeInformation[] {
+                                            
TypeInformation.of(DenseVector.class),
+                                            Types.DOUBLE,
+                                            Types.DOUBLE
+                                        },
+                                        new String[] {"features", "label", 
"weight"})));
+    }
+
+    @SuppressWarnings("ConstantConditions")
+    private void verifyPredictionResult(
+            Table output, String featuresCol, String predictionCol, String 
rawPredictionCol)
+            throws Exception {
+        List<Row> predResult = 
IteratorUtils.toList(tEnv.toDataStream(output).executeAndCollect());
+        for (Row predictionRow : predResult) {
+            DenseVector feature = (DenseVector) 
predictionRow.getField(featuresCol);
+            double prediction = (double) predictionRow.getField(predictionCol);
+            double[] rawPrediction = (double[]) 
predictionRow.getField(rawPredictionCol);
+            if (feature.get(0) <= 5) {
+                assertEquals(0, prediction, TOLERANCE);
+                assertTrue(rawPrediction[0] > 0.5);
+            } else {
+                assertEquals(1, prediction, TOLERANCE);
+                assertTrue(rawPrediction[0] < 0.5);
+            }
+        }
+    }
+
+    @Test
+    public void testParam() {
+        LogisticRegression logisticRegression = new LogisticRegression();
+        assertEquals(logisticRegression.getMaxIter(), 20);
+        assertNull(logisticRegression.getWeightCol());
+        assertEquals(logisticRegression.getReg(), 0, TOLERANCE);
+        assertEquals(logisticRegression.getLearningRate(), 0.1, TOLERANCE);
+        assertEquals(logisticRegression.getGlobalBatchSize(), 32);
+        assertEquals(logisticRegression.getTol(), 1e-6, TOLERANCE);
+        assertEquals(logisticRegression.getLabelCol(), "label");
+        assertEquals(logisticRegression.getMultiClass(), "auto");
+        assertEquals(logisticRegression.getPredictionCol(), "prediction");
+        assertEquals(logisticRegression.getRawPredictionCol(), 
"rawPrediction");
+
+        logisticRegression
+                .setFeaturesCol("features")
+                .setLabelCol("label")
+                .setWeightCol("weight")
+                .setMaxIter(1000)
+                .setTol(0.1)
+                .setLearningRate(0.5)
+                .setGlobalBatchSize(1000)
+                .setReg(0.1)
+                .setMultiClass("bionomial")
+                .setPredictionCol("pred")
+                .setRawPredictionCol("rawPred");
+        assertEquals(logisticRegression.getFeaturesCol(), "features");
+        assertEquals(logisticRegression.getLabelCol(), "label");
+        assertEquals(logisticRegression.getWeightCol(), "weight");
+        assertEquals(logisticRegression.getMaxIter(), 1000);
+        assertEquals(logisticRegression.getTol(), 0.1, TOLERANCE);
+        assertEquals(logisticRegression.getGlobalBatchSize(), 1000);
+        assertEquals(logisticRegression.getReg(), 0.1, TOLERANCE);
+        assertEquals(logisticRegression.getMultiClass(), "bionomial");
+        assertEquals(logisticRegression.getPredictionCol(), "pred");
+        assertEquals(logisticRegression.getRawPredictionCol(), "rawPred");
+    }
+
+    @Test
+    public void testFeaturePredictionParam() {
+        LogisticRegression logisticRegression =
+                new LogisticRegression()
+                        .setFeaturesCol("features")

Review comment:
       Could we use non-default values for these column names?

##########
File path: flink-ml-core/src/main/java/org/apache/flink/ml/linalg/BLAS.java
##########
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.ml.linalg;
+
+import org.apache.flink.util.Preconditions;
+
+/** A utility class that provides BLAS routines over matrices and vectors. */
+public class BLAS {
+
+    private static final dev.ludovic.netlib.BLAS NATIVE_BLAS =
+            dev.ludovic.netlib.BLAS.getInstance();
+
+    /**
+     * \sum_i |x_i| .
+     *
+     * @param x x
+     * @return

Review comment:
       nits: specify the return value here?

##########
File path: 
flink-ml-lib/src/test/java/org/apache/flink/ml/classification/linear/LogisticRegressionTest.java
##########
@@ -0,0 +1,291 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.ml.classification.linear;
+
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import 
org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.types.Row;
+
+import org.apache.commons.collections.IteratorUtils;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/** Tests {@link LogisticRegression} and {@link LogisticRegressionModel}. */
+public class LogisticRegressionTest {
+
+    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+
+    private StreamExecutionEnvironment env;
+
+    private StreamTableEnvironment tEnv;
+
+    private static final List<Row> bionomialTrainData =
+            Arrays.asList(
+                    Row.of(new DenseVector(new double[] {1, 2, 3, 4}), 0., 1.),

Review comment:
       Can we use `Vectors.dense` here? Same for other lines.

##########
File path: 
flink-ml-lib/src/main/java/org/apache/flink/ml/common/param/HasMultiClass.java
##########
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.ml.common.param;
+
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+import org.apache.flink.ml.param.StringParam;
+import org.apache.flink.ml.param.WithParams;
+
+/**
+ * Interface for the shared multi-class param.
+ *
+ * <p>Supported options:
+ * <li>auto: select the version based on the number of classes: If numClasses 
is one or two, set to
+ *     "binomial". Otherwise, set to "multinomial".
+ * <li>binomial: Binary logistic regression.
+ * <li>multinomial: Multinomial logistic regression.
+ */
+public interface HasMultiClass<T> extends WithParams<T> {
+    Param<String> MULTI_CLASS =
+            new StringParam(
+                    "multiClass",
+                    "Type of classification.",

Review comment:
       nits: it will be nice to use similar doc style for enum-like parameters. 
Could we use `The classification type` here, similar to `HasDistanceMeasure`? 
We could also change `HasDistanceMeasure`'s doc to `The type of distance 
measure`. IMO it is simpler to avoid using `of` here.

##########
File path: 
flink-ml-core/src/main/java/org/apache/flink/ml/common/iteration/TerminateOnMaxIter.java
##########
@@ -27,16 +27,17 @@
  * threshold.
  *
  * <p>When the output of this FlatMapFunction is used as the termination 
criteria of an iteration
- * body, the iteration will be executed for at most the given `numRounds` 
rounds.
+ * body, the iteration will be executed for at most the given `maxIter` rounds.
  *
  * @param <T> The class type of the input element.
  */
-public class TerminateOnMaxIterationNum<T>
+public class TerminateOnMaxIter<T>

Review comment:
       Could we replace `RoundBasedTerminationCriteria` with this 
`TerminateOnMaxIter`?

##########
File path: 
flink-ml-lib/src/main/java/org/apache/flink/ml/classification/linear/LogisticRegressionModel.java
##########
@@ -0,0 +1,189 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.ml.classification.linear;
+
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.ml.api.Model;
+import org.apache.flink.ml.common.broadcast.BroadcastUtils;
+import org.apache.flink.ml.linalg.BLAS;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.internal.TableImpl;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+/** This class implements {@link Model} for {@link LogisticRegression}. */
+public class LogisticRegressionModel
+        implements Model<LogisticRegressionModel>,
+                LogisticRegressionModelParams<LogisticRegressionModel> {
+
+    private Map<Param<?>, Object> paramMap = new HashMap<>();
+
+    private Table model;
+
+    public LogisticRegressionModel() {
+        ParamUtils.initializeMapWithDefaultValues(this.paramMap, this);
+    }
+
+    @Override
+    public Map<Param<?>, Object> getParamMap() {
+        return paramMap;
+    }
+
+    @Override
+    public void save(String path) throws IOException {
+        ReadWriteUtils.saveMetadata(this, path);
+        ReadWriteUtils.saveModelData(
+                LogisticRegressionModelData.getModelDataStream(model),
+                path,
+                LogisticRegressionModelData.getModelDataEncoder());
+    }
+
+    public static LogisticRegressionModel load(StreamExecutionEnvironment env, 
String path)
+            throws IOException {
+        LogisticRegressionModel model = ReadWriteUtils.loadStageParam(path);
+        Table modelData =
+                ReadWriteUtils.loadModelData(
+                        env, path, 
LogisticRegressionModelData.getModelDataDecoder());
+        return model.setModelData(modelData);
+    }
+
+    @Override
+    public LogisticRegressionModel setModelData(Table... inputs) {
+        model = inputs[0];
+        return this;
+    }
+
+    @Override
+    public Table[] getModelData() {
+        return new Table[] {model};
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public Table[] transform(Table... inputs) {
+        Preconditions.checkArgument(inputs.length == 1);
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) 
inputs[0]).getTableEnvironment();
+        DataStream<Row> data = tEnv.toDataStream(inputs[0]);
+        final String broadcastModelKey = "broadcastModel";
+        DataStream<LogisticRegressionModelData> modelData =
+                LogisticRegressionModelData.getModelDataStream(model);
+        DataStream<Row> predictResult =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, modelData),
+                        inputList -> {
+                            DataStream inputData = inputList.get(0);
+                            return inputData.transform(
+                                    "doPrediction",
+                                    new RowTypeInfo(
+                                            new TypeInformation[] {
+                                                
TypeInformation.of(DenseVector.class),
+                                                BasicTypeInfo.DOUBLE_TYPE_INFO,
+                                                PrimitiveArrayTypeInfo
+                                                        
.DOUBLE_PRIMITIVE_ARRAY_TYPE_INFO
+                                            },
+                                            new String[] {
+                                                getFeaturesCol(),
+                                                getPredictionCol(),
+                                                getRawPredictionCol()
+                                            }),
+                                    new PredictOp(
+                                            new PredictOneRecord(
+                                                    getFeaturesCol(), 
broadcastModelKey)));
+                        });
+
+        return new Table[] {tEnv.fromDataStream(predictResult)};
+    }
+
+    /** A utility operator used for prediction. */
+    private static class PredictOp extends AbstractUdfStreamOperator<Row, 
RichMapFunction<Row, Row>>

Review comment:
       There are already a couple places (e.g. `MapPartitionOperator`) where we 
specify the computation logic in `AbstractStreamOperator::processElement` 
instead of putting this logic in a separate `RichMapFunction` subclass. Could 
we do the same here for consistency and simplicity?
   

##########
File path: 
flink-ml-core/src/main/java/org/apache/flink/ml/common/datastream/DataStreamUtils.java
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.ml.common.datastream;
+
+import org.apache.flink.api.common.functions.MapPartitionFunction;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.TimestampedCollector;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+
+import org.apache.commons.collections.IteratorUtils;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/** Provides utility functions for {@link DataStream}. */
+public class DataStreamUtils {
+    /**
+     * Applies allReduceSum on the input data stream. The input data stream is 
supposed to contain
+     * one double array in each partition. The result data stream has the same 
parallelism as the
+     * input, where each partition contains one double array that sums all of 
the double arrays in
+     * the input data stream.
+     *
+     * <p>Note that we throw exception when one of the following two cases 
happen:
+     * <li>There exists one partition that contains more than one double array.
+     * <li>The length of the double array is not consistent among all 
partitions.
+     *
+     * @param input The input data stream.
+     * @return The result data stream.
+     */
+    public static DataStream<double[]> allReduceSum(DataStream<double[]> 
input) {
+        return AllReduceImpl.allReduceSum(input);
+    }
+
+    /**
+     * Collects distinct values in a bounded data stream. The parallelism of 
the output stream is 1.
+     *
+     * @param <T> The class type of the input data stream.
+     * @param input The bounded input data stream.
+     * @return The result data stream that contains all the distinct values.
+     */
+    public static <T> DataStream<T> distinct(DataStream<T> input) {

Review comment:
       It looks like `distinct()` is only used in tests. Could we only add this 
method when it is needed by production code?
   
   Same for `sortPartition()`.
   

##########
File path: 
flink-ml-core/src/main/java/org/apache/flink/ml/common/datastream/DataStreamUtils.java
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.ml.common.datastream;
+
+import org.apache.flink.api.common.functions.MapPartitionFunction;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.TimestampedCollector;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+
+import org.apache.commons.collections.IteratorUtils;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/** Provides utility functions for {@link DataStream}. */
+public class DataStreamUtils {
+    /**
+     * Applies allReduceSum on the input data stream. The input data stream is 
supposed to contain
+     * one double array in each partition. The result data stream has the same 
parallelism as the
+     * input, where each partition contains one double array that sums all of 
the double arrays in
+     * the input data stream.
+     *
+     * <p>Note that we throw exception when one of the following two cases 
happen:
+     * <li>There exists one partition that contains more than one double array.
+     * <li>The length of the double array is not consistent among all 
partitions.
+     *
+     * @param input The input data stream.
+     * @return The result data stream.
+     */
+    public static DataStream<double[]> allReduceSum(DataStream<double[]> 
input) {
+        return AllReduceImpl.allReduceSum(input);
+    }
+
+    /**
+     * Collects distinct values in a bounded data stream. The parallelism of 
the output stream is 1.
+     *
+     * @param <T> The class type of the input data stream.
+     * @param input The bounded input data stream.
+     * @return The result data stream that contains all the distinct values.
+     */
+    public static <T> DataStream<T> distinct(DataStream<T> input) {
+        return input.transform(
+                        "distinctInEachPartition",
+                        input.getType(),
+                        new DistinctPartitionOperator<>())
+                .setParallelism(input.getParallelism())
+                .transform(
+                        "distinctInFinalPartition",
+                        input.getType(),
+                        new DistinctPartitionOperator<>())
+                .setParallelism(1);
+    }
+
+    /**
+     * Applies a {@link MapPartitionFunction} on a bounded data stream.
+     *
+     * @param input The input data stream.
+     * @param func The user defined mapPartition function.
+     * @param <IN> The class type of the input element.
+     * @param <OUT> The class type of output element.
+     * @return The result data stream.
+     */
+    public static <IN, OUT> DataStream<OUT> mapPartition(
+            DataStream<IN> input, MapPartitionFunction<IN, OUT> func) {
+        TypeInformation<OUT> resultType =
+                TypeExtractor.getMapPartitionReturnTypes(func, 
input.getType(), null, true);
+        return input.transform("mapPartition", resultType, new 
MapPartitionOperator<>(func))
+                .setParallelism(input.getParallelism());
+    }
+
+    /**
+     * Sorts the elements in each partition of the input bounded data stream.
+     *
+     * @param input The input data stream.
+     * @param comparator The comparator used to sort the elements.
+     * @param <IN> The class type of input element.
+     * @return The sorted data stream.
+     */
+    public static <IN> DataStream<IN> sortPartition(
+            DataStream<IN> input, TypeComparator<IN> comparator) {
+        return input.transform(
+                        "sortPartition", input.getType(), new 
SortPartitionOperator<>(comparator))
+                .setParallelism(input.getParallelism());
+    }
+
+    /**
+     * A stream operator to compute the distinct values in each partition of 
the input bounded data
+     * stream.
+     */
+    private static class DistinctPartitionOperator<T> extends 
AbstractStreamOperator<T>
+            implements OneInputStreamOperator<T, T>, BoundedOneInput {

Review comment:
       nits: How about we remove `BoundedOneInput` and use 
`AbstractStreamOperator::close` consistently, similar to what we do when the 
operator has 2+ inputs?
   
   Same for other usages of `BoundedOneInput`.
   

##########
File path: 
flink-ml-lib/src/main/java/org/apache/flink/ml/common/param/HasMultiClass.java
##########
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.ml.common.param;
+
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+import org.apache.flink.ml.param.StringParam;
+import org.apache.flink.ml.param.WithParams;
+
+/**
+ * Interface for the shared multi-class param.
+ *
+ * <p>Supported options:
+ * <li>auto: select the version based on the number of classes: If numClasses 
is one or two, set to

Review comment:
       Spark has the `ClassificationModel::numClasses` API to expose number of 
classes. We don't have this yet.
   
   Should we add the `ClassificationModel` and `Classifier` abstract classes 
like what Spark does? Otherwise, maybe we should remove `auto` since it is not 
clear how we are going to automatically choose the classification type.

##########
File path: 
flink-ml-lib/src/main/java/org/apache/flink/ml/classification/linear/LogisticRegressionModel.java
##########
@@ -0,0 +1,189 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.ml.classification.linear;
+
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.ml.api.Model;
+import org.apache.flink.ml.common.broadcast.BroadcastUtils;
+import org.apache.flink.ml.linalg.BLAS;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.internal.TableImpl;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+/** This class implements {@link Model} for {@link LogisticRegression}. */
+public class LogisticRegressionModel
+        implements Model<LogisticRegressionModel>,
+                LogisticRegressionModelParams<LogisticRegressionModel> {
+
+    private Map<Param<?>, Object> paramMap = new HashMap<>();
+
+    private Table model;

Review comment:
       nits: could we rename this to `modelData` to distinguish it from the 
`Model` class?

##########
File path: 
flink-ml-lib/src/test/java/org/apache/flink/ml/classification/linear/LogisticRegressionTest.java
##########
@@ -0,0 +1,291 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.ml.classification.linear;
+
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import 
org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.types.Row;
+
+import org.apache.commons.collections.IteratorUtils;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/** Tests {@link LogisticRegression} and {@link LogisticRegressionModel}. */
+public class LogisticRegressionTest {
+
+    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+
+    private StreamExecutionEnvironment env;
+
+    private StreamTableEnvironment tEnv;
+
+    private static final List<Row> bionomialTrainData =
+            Arrays.asList(
+                    Row.of(new DenseVector(new double[] {1, 2, 3, 4}), 0., 1.),
+                    Row.of(new DenseVector(new double[] {2, 2, 3, 4}), 0., 2.),
+                    Row.of(new DenseVector(new double[] {3, 2, 3, 4}), 0., 3.),
+                    Row.of(new DenseVector(new double[] {4, 2, 3, 4}), 0., 4.),
+                    Row.of(new DenseVector(new double[] {5, 2, 3, 4}), 0., 5.),
+                    Row.of(new DenseVector(new double[] {11, 2, 3, 4}), 1., 
1.),
+                    Row.of(new DenseVector(new double[] {12, 2, 3, 4}), 1., 
2.),
+                    Row.of(new DenseVector(new double[] {13, 2, 3, 4}), 1., 
3.),
+                    Row.of(new DenseVector(new double[] {14, 2, 3, 4}), 1., 
4.),
+                    Row.of(new DenseVector(new double[] {15, 2, 3, 4}), 1., 
5.));
+
+    private static final List<Row> multinomialTrainData =
+            Arrays.asList(
+                    Row.of(new DenseVector(new double[] {1, 2, 3, 4}), 0., 1.),
+                    Row.of(new DenseVector(new double[] {2, 2, 3, 4}), 0., 2.),
+                    Row.of(new DenseVector(new double[] {3, 2, 3, 4}), 2., 3.),
+                    Row.of(new DenseVector(new double[] {4, 2, 3, 4}), 2., 4.),
+                    Row.of(new DenseVector(new double[] {5, 2, 3, 4}), 2., 5.),
+                    Row.of(new DenseVector(new double[] {11, 2, 3, 4}), 1., 
1.),
+                    Row.of(new DenseVector(new double[] {12, 2, 3, 4}), 1., 
2.),
+                    Row.of(new DenseVector(new double[] {13, 2, 3, 4}), 1., 
3.),
+                    Row.of(new DenseVector(new double[] {14, 2, 3, 4}), 1., 
4.),
+                    Row.of(new DenseVector(new double[] {15, 2, 3, 4}), 1., 
5.));
+
+    private static final double[] expectedCoefficient =
+            new double[] {0.528, -0.286, -0.429, -0.572};
+
+    private static final double TOLERANCE = 1e-6;
+
+    private Table bionomialDataTable;
+
+    private Table multinomialDataTable;
+
+    @Before
+    public void before() {
+        Configuration config = new Configuration();
+        
config.set(ExecutionCheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, 
true);
+        env = StreamExecutionEnvironment.getExecutionEnvironment(config);
+        env.setParallelism(4);
+        env.enableCheckpointing(100);
+        env.setRestartStrategy(RestartStrategies.noRestart());
+        tEnv = StreamTableEnvironment.create(env);
+        Collections.shuffle(bionomialTrainData);
+        bionomialDataTable =
+                tEnv.fromDataStream(
+                        env.fromCollection(
+                                bionomialTrainData,
+                                new RowTypeInfo(
+                                        new TypeInformation[] {
+                                            
TypeInformation.of(DenseVector.class),
+                                            Types.DOUBLE,
+                                            Types.DOUBLE
+                                        },
+                                        new String[] {"features", "label", 
"weight"})));
+        multinomialDataTable =
+                tEnv.fromDataStream(
+                        env.fromCollection(
+                                multinomialTrainData,
+                                new RowTypeInfo(
+                                        new TypeInformation[] {
+                                            
TypeInformation.of(DenseVector.class),
+                                            Types.DOUBLE,
+                                            Types.DOUBLE
+                                        },
+                                        new String[] {"features", "label", 
"weight"})));
+    }
+
+    @SuppressWarnings("ConstantConditions")
+    private void verifyPredictionResult(
+            Table output, String featuresCol, String predictionCol, String 
rawPredictionCol)
+            throws Exception {
+        List<Row> predResult = 
IteratorUtils.toList(tEnv.toDataStream(output).executeAndCollect());
+        for (Row predictionRow : predResult) {
+            DenseVector feature = (DenseVector) 
predictionRow.getField(featuresCol);
+            double prediction = (double) predictionRow.getField(predictionCol);
+            double[] rawPrediction = (double[]) 
predictionRow.getField(rawPredictionCol);

Review comment:
       Should the return type of `rawPredictionCol` be `DenseVector` instead of 
`double[]` for consistency with `featuresCol`?

##########
File path: 
flink-ml-core/src/main/java/org/apache/flink/ml/common/datastream/DataStreamUtils.java
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.ml.common.datastream;
+
+import org.apache.flink.api.common.functions.MapPartitionFunction;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.TimestampedCollector;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+
+import org.apache.commons.collections.IteratorUtils;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/** Provides utility functions for {@link DataStream}. */
+public class DataStreamUtils {
+    /**
+     * Applies allReduceSum on the input data stream. The input data stream is 
supposed to contain
+     * one double array in each partition. The result data stream has the same 
parallelism as the
+     * input, where each partition contains one double array that sums all of 
the double arrays in
+     * the input data stream.
+     *
+     * <p>Note that we throw exception when one of the following two cases 
happen:
+     * <li>There exists one partition that contains more than one double array.
+     * <li>The length of the double array is not consistent among all 
partitions.
+     *
+     * @param input The input data stream.
+     * @return The result data stream.
+     */
+    public static DataStream<double[]> allReduceSum(DataStream<double[]> 
input) {
+        return AllReduceImpl.allReduceSum(input);
+    }
+
+    /**
+     * Collects distinct values in a bounded data stream. The parallelism of 
the output stream is 1.
+     *
+     * @param <T> The class type of the input data stream.
+     * @param input The bounded input data stream.
+     * @return The result data stream that contains all the distinct values.
+     */
+    public static <T> DataStream<T> distinct(DataStream<T> input) {
+        return input.transform(
+                        "distinctInEachPartition",
+                        input.getType(),
+                        new DistinctPartitionOperator<>())
+                .setParallelism(input.getParallelism())
+                .transform(
+                        "distinctInFinalPartition",
+                        input.getType(),
+                        new DistinctPartitionOperator<>())
+                .setParallelism(1);
+    }
+
+    /**
+     * Applies a {@link MapPartitionFunction} on a bounded data stream.
+     *
+     * @param input The input data stream.
+     * @param func The user defined mapPartition function.
+     * @param <IN> The class type of the input element.
+     * @param <OUT> The class type of output element.
+     * @return The result data stream.
+     */
+    public static <IN, OUT> DataStream<OUT> mapPartition(
+            DataStream<IN> input, MapPartitionFunction<IN, OUT> func) {
+        TypeInformation<OUT> resultType =
+                TypeExtractor.getMapPartitionReturnTypes(func, 
input.getType(), null, true);
+        return input.transform("mapPartition", resultType, new 
MapPartitionOperator<>(func))
+                .setParallelism(input.getParallelism());
+    }
+
+    /**
+     * Sorts the elements in each partition of the input bounded data stream.
+     *
+     * @param input The input data stream.
+     * @param comparator The comparator used to sort the elements.
+     * @param <IN> The class type of input element.
+     * @return The sorted data stream.
+     */
+    public static <IN> DataStream<IN> sortPartition(
+            DataStream<IN> input, TypeComparator<IN> comparator) {
+        return input.transform(
+                        "sortPartition", input.getType(), new 
SortPartitionOperator<>(comparator))
+                .setParallelism(input.getParallelism());

Review comment:
       nits: is `setParallelism(input.getParallelism())` necessary here?
   
   Same for `sortPartition()`.
   

##########
File path: 
flink-ml-core/src/test/java/org/apache/flink/ml/common/datastream/DataStreamUtilsTest.java
##########
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.ml.common.datastream;
+
+import org.apache.flink.api.common.functions.MapPartitionFunction;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.common.typeutils.base.LongComparator;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import 
org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.NumberSequenceIterator;
+
+import org.apache.commons.collections.IteratorUtils;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertArrayEquals;
+
+/** Tests the {@link DataStreamUtils}. */
+public class DataStreamUtilsTest {
+    private StreamExecutionEnvironment env;
+
+    @Before
+    public void before() {
+        Configuration config = new Configuration();
+        
config.set(ExecutionCheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, 
true);
+        env = StreamExecutionEnvironment.getExecutionEnvironment(config);
+        env.setParallelism(4);
+        env.enableCheckpointing(100);
+        env.setRestartStrategy(RestartStrategies.noRestart());
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void testDistinct() throws Exception {
+        DataStream<Double> dataStream =
+                env.fromParallelCollection(new NumberSequenceIterator(1L, 
10L), Types.LONG)

Review comment:
       It looks like the input values are already unique?

##########
File path: 
flink-ml-core/src/test/java/org/apache/flink/ml/common/datastream/DataStreamUtilsTest.java
##########
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.ml.common.datastream;
+
+import org.apache.flink.api.common.functions.MapPartitionFunction;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.common.typeutils.base.LongComparator;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import 
org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.NumberSequenceIterator;
+
+import org.apache.commons.collections.IteratorUtils;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertArrayEquals;
+
+/** Tests the {@link DataStreamUtils}. */
+public class DataStreamUtilsTest {
+    private StreamExecutionEnvironment env;
+
+    @Before
+    public void before() {
+        Configuration config = new Configuration();
+        
config.set(ExecutionCheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, 
true);
+        env = StreamExecutionEnvironment.getExecutionEnvironment(config);
+        env.setParallelism(4);
+        env.enableCheckpointing(100);
+        env.setRestartStrategy(RestartStrategies.noRestart());
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void testDistinct() throws Exception {
+        DataStream<Double> dataStream =
+                env.fromParallelCollection(new NumberSequenceIterator(1L, 
10L), Types.LONG)
+                        .map(x -> (x / 2) * 1.);
+        double[] result =
+                ((List<Double>)
+                                IteratorUtils.toList(
+                                        
DataStreamUtils.distinct(dataStream).executeAndCollect()))
+                        .stream().mapToDouble(Double::doubleValue).toArray();
+        Arrays.sort(result);
+        assertArrayEquals(new double[] {0., 1., 2., 3., 4., 5.}, result, 1e-7);
+    }
+
+    @Test
+    public void testMapPartition() throws Exception {
+        DataStream<Long> dataStream =
+                env.fromParallelCollection(new NumberSequenceIterator(0L, 
19L), Types.LONG);
+        DataStream<Integer> countsPerPartition =
+                DataStreamUtils.mapPartition(dataStream, new 
TestMapPartitionFunc());
+        List<Integer> counts = 
IteratorUtils.toList(countsPerPartition.executeAndCollect());
+        assertArrayEquals(

Review comment:
       nits: the first parameter of `assertArrayEquals` should be `expecteds`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to