yunfengzhou-hub commented on code in PR #83:
URL: https://github.com/apache/flink-ml/pull/83#discussion_r881176285


##########
flink-ml-core/src/main/java/org/apache/flink/ml/linalg/BLAS.java:
##########
@@ -71,6 +71,43 @@ public static double dot(DenseVector x, DenseVector y) {
         return JAVA_BLAS.ddot(x.size(), x.values, 1, y.values, 1);
     }
 
+    public static double dot(DenseVector x, SparseVector y) {
+        Preconditions.checkArgument(x.size() == y.size(), "Vector size 
mismatched.");
+        double dotValue = 0.0;
+        for (int i = 0; i < y.indices.length; ++i) {
+            dotValue += y.values[i] * x.values[y.indices[i]];
+        }
+        return dotValue;
+    }
+
+    public static double dot(SparseVector x, DenseVector y) {

Review Comment:
   nit: 
   ```java
       public static double dot(SparseVector x, DenseVector y) {
           return dot(y, x);
       }
   ```



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/classification/logisticregression/OnlineLogisticRegressionParams.java:
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.logisticregression;
+
+import org.apache.flink.ml.common.param.HasBatchStrategy;
+import org.apache.flink.ml.common.param.HasElasticNet;
+import org.apache.flink.ml.common.param.HasGlobalBatchSize;
+import org.apache.flink.ml.common.param.HasLabelCol;
+import org.apache.flink.ml.common.param.HasReg;
+import org.apache.flink.ml.param.DoubleParam;
+import org.apache.flink.ml.param.IntParam;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+
+/**
+ * Params of {@link OnlineLogisticRegression}.
+ *
+ * @param <T> The class type of this instance.
+ */
+public interface OnlineLogisticRegressionParams<T>
+        extends HasLabelCol<T>,
+                HasBatchStrategy<T>,
+                HasGlobalBatchSize<T>,
+                HasReg<T>,
+                HasElasticNet<T>,
+                OnlineLogisticRegressionModelParams<T> {
+
+    Param<Double> ALPHA =
+            new DoubleParam("alpha", "The parameter alpha of ftrl.", 0.1, 
ParamValidators.gt(0.0));
+
+    default Double getAlpha() {
+        return get(ALPHA);
+    }
+
+    default T setAlpha(Double value) {
+        return set(ALPHA, value);
+    }
+
+    Param<Double> BETA =
+            new DoubleParam("alpha", "The parameter beta of ftrl.", 0.1, 
ParamValidators.gt(0.0));
+
+    default Double getBeta() {
+        return get(BETA);
+    }
+
+    default T setBeta(Double value) {
+        return set(BETA, value);
+    }
+
+    Param<Integer> MODEL_SAVE_INTERVAL =
+            new IntParam(
+                    "modelSaveInterval",
+                    "The iteration steps between two output models.",

Review Comment:
   Does this parameter mean OLR should generate a new version of model data 
after every 100 batches of train data? The term "interval" gives me an 
impression that it is related to times. Shall we modify the parameter name and 
description to better reflect its semantics?
   
   Besides, "model data" should be more proper than "models".



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/util/TestUtils.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.util;
+
+import org.apache.flink.ml.linalg.DenseVector;
+
+/** Utility methods for testing. */
+public class TestUtils {
+    /** Note: this comparator imposes orderings that are inconsistent with 
equals. */
+    public static int compare(DenseVector first, DenseVector second) {
+        for (int i = 0; i < first.size(); i++) {

Review Comment:
   nit: we should check for size equality before doing the comparison.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/classification/logisticregression/OnlineLogisticRegressionModel.java:
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.logisticregression;
+
+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.typeinfo.Types;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.ml.api.Model;
+import org.apache.flink.ml.common.datastream.TableUtils;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.linalg.Vector;
+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.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+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.types.Row;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.commons.lang3.ArrayUtils;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static 
org.apache.flink.ml.classification.logisticregression.LogisticRegressionModel.predictOneDataPoint;
+
+/**
+ * A Model which classifies data using the model data computed by {@link 
OnlineLogisticRegression}.
+ */
+public class OnlineLogisticRegressionModel
+        implements Model<OnlineLogisticRegressionModel>,
+                
OnlineLogisticRegressionModelParams<OnlineLogisticRegressionModel> {
+    public static final String MODEL_DATA_VERSION_GAUGE_KEY = 
"modelDataVersion";
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+    private Table modelDataTable;
+
+    public OnlineLogisticRegressionModel() {
+        ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+    }
+
+    @Override
+    public Table[] transform(Table... inputs) {
+        Preconditions.checkArgument(inputs.length == 1);
+
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) 
inputs[0]).getTableEnvironment();
+
+        RowTypeInfo inputTypeInfo = 
TableUtils.getRowTypeInfo(inputs[0].getResolvedSchema());
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        ArrayUtils.addAll(
+                                inputTypeInfo.getFieldTypes(),
+                                Types.DOUBLE,
+                                TypeInformation.of(DenseVector.class),
+                                Types.LONG),
+                        ArrayUtils.addAll(
+                                inputTypeInfo.getFieldNames(),
+                                getPredictionCol(),
+                                getRawPredictionCol(),
+                                getModelVersionCol()));
+
+        DataStream<Row> predictionResult =
+                tEnv.toDataStream(inputs[0])
+                        .connect(
+                                
LogisticRegressionModelData.getModelDataStream(modelDataTable)
+                                        .broadcast())
+                        .transform(
+                                "PredictLabelOperator",
+                                outputTypeInfo,
+                                new PredictLabelOperator(inputTypeInfo, 
getFeaturesCol()));
+
+        return new Table[] {tEnv.fromDataStream(predictionResult)};
+    }
+
+    /** A utility operator used for prediction. */
+    private static class PredictLabelOperator extends 
AbstractStreamOperator<Row>
+            implements TwoInputStreamOperator<Row, 
LogisticRegressionModelData, Row> {
+        private final RowTypeInfo inputTypeInfo;
+
+        private final String featuresCol;
+        private ListState<Row> bufferedPointsState;
+        private DenseVector coefficient;
+        private long modelDataVersion = 0L;
+
+        public PredictLabelOperator(RowTypeInfo inputTypeInfo, String 
featuresCol) {
+            this.inputTypeInfo = inputTypeInfo;
+            this.featuresCol = featuresCol;
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws 
Exception {
+            super.initializeState(context);
+            bufferedPointsState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new 
ListStateDescriptor<>("bufferedPoints", inputTypeInfo));
+        }
+
+        @Override
+        public void open() throws Exception {
+            super.open();
+
+            getRuntimeContext()
+                    .getMetricGroup()
+                    .gauge(
+                            MODEL_DATA_VERSION_GAUGE_KEY,
+                            (Gauge<String>) () -> 
Long.toString(modelDataVersion));
+        }
+
+        @Override
+        public void processElement1(StreamRecord<Row> streamRecord) throws 
Exception {
+            Row dataPoint = streamRecord.getValue();
+            if (coefficient == null) {
+                bufferedPointsState.add(dataPoint);
+                return;
+            }
+            Vector features = (Vector) dataPoint.getField(featuresCol);
+            Row predictionResult = predictOneDataPoint(features, coefficient);
+            output.collect(
+                    new StreamRecord<>(
+                            Row.join(
+                                    dataPoint,
+                                    Row.of(
+                                            predictionResult.getField(0),
+                                            predictionResult.getField(1),
+                                            modelDataVersion))));
+        }
+
+        @Override
+        public void processElement2(StreamRecord<LogisticRegressionModelData> 
streamRecord)
+                throws Exception {
+            LogisticRegressionModelData modelData = streamRecord.getValue();
+            coefficient = modelData.coefficient;
+            modelDataVersion = modelData.modelVersion;
+            for (Row dataPoint : bufferedPointsState.get()) {
+                processElement1(new StreamRecord<>(dataPoint));
+            }
+            bufferedPointsState.clear();
+        }
+    }
+
+    @Override
+    public void save(String path) throws IOException {
+        ReadWriteUtils.saveMetadata(this, path);
+        ReadWriteUtils.saveModelData(
+                LogisticRegressionModelData.getModelDataStream(modelDataTable),
+                path,
+                new LogisticRegressionModelData.ModelDataEncoder());
+    }
+
+    public static OnlineLogisticRegressionModel load(StreamTableEnvironment 
tEnv, String path)
+            throws IOException {
+        OnlineLogisticRegressionModel model = 
ReadWriteUtils.loadStageParam(path);
+        Table modelDataTable =
+                ReadWriteUtils.loadModelData(
+                        tEnv, path, new 
LogisticRegressionModelData.ModelDataDecoder());
+        return model.setModelData(modelDataTable);
+    }
+
+    @Override
+    public Map<Param<?>, Object> getParamMap() {
+        return paramMap;
+    }
+
+    @Override
+    public OnlineLogisticRegressionModel setModelData(Table... inputs) {
+        modelDataTable = inputs[0];

Review Comment:
   nit: Let's check `inputs.length` in this method.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/classification/logisticregression/OnlineLogisticRegression.java:
##########
@@ -0,0 +1,434 @@
+/*
+ * 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.logisticregression;
+
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+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.java.typeutils.ObjectArrayTypeInfo;
+import org.apache.flink.iteration.DataStreamList;
+import org.apache.flink.iteration.IterationBody;
+import org.apache.flink.iteration.IterationBodyResult;
+import org.apache.flink.iteration.Iterations;
+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.linalg.DenseVector;
+import org.apache.flink.ml.linalg.SparseVector;
+import org.apache.flink.ml.linalg.Vector;
+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.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+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.types.Row;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.commons.collections.IteratorUtils;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * An Estimator which implements the FTRL-Proximal online learning algorithm 
proposed by H. Brendan
+ * McMahan et al.
+ *
+ * <p>See <a href="https://doi.org/10.1145/2487575.2488200";>H. Brendan McMahan 
et al., Ad click
+ * prediction: a view from the trenches.</a>
+ */
+public class OnlineLogisticRegression
+        implements Estimator<OnlineLogisticRegression, 
OnlineLogisticRegressionModel>,
+                OnlineLogisticRegressionParams<OnlineLogisticRegression> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+    private Table initModelDataTable;
+
+    public OnlineLogisticRegression() {
+        ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public OnlineLogisticRegressionModel fit(Table... inputs) {
+        Preconditions.checkArgument(inputs.length == 1);
+
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) 
inputs[0]).getTableEnvironment();
+        DataStream<LogisticRegressionModelData> modelDataStream =
+                
LogisticRegressionModelData.getModelDataStream(initModelDataTable);
+
+        DataStream<Row> points =
+                tEnv.toDataStream(inputs[0])
+                        .map(new FeaturesExtractor(getFeaturesCol(), 
getLabelCol()));
+
+        DataStream<DenseVector> initModelData =
+                modelDataStream.map(
+                        (MapFunction<LogisticRegressionModelData, DenseVector>)
+                                value -> value.coefficient);

Review Comment:
   Should we also extract `modelVersion` from the input model data, and 
increase modelVersion on the basis of the extracted value?



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/classification/logisticregression/OnlineLogisticRegression.java:
##########
@@ -0,0 +1,434 @@
+/*
+ * 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.logisticregression;
+
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+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.java.typeutils.ObjectArrayTypeInfo;
+import org.apache.flink.iteration.DataStreamList;
+import org.apache.flink.iteration.IterationBody;
+import org.apache.flink.iteration.IterationBodyResult;
+import org.apache.flink.iteration.Iterations;
+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.linalg.DenseVector;
+import org.apache.flink.ml.linalg.SparseVector;
+import org.apache.flink.ml.linalg.Vector;
+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.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+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.types.Row;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.commons.collections.IteratorUtils;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * An Estimator which implements the FTRL-Proximal online learning algorithm 
proposed by H. Brendan
+ * McMahan et al.
+ *
+ * <p>See <a href="https://doi.org/10.1145/2487575.2488200";>H. Brendan McMahan 
et al., Ad click
+ * prediction: a view from the trenches.</a>
+ */
+public class OnlineLogisticRegression
+        implements Estimator<OnlineLogisticRegression, 
OnlineLogisticRegressionModel>,
+                OnlineLogisticRegressionParams<OnlineLogisticRegression> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+    private Table initModelDataTable;
+
+    public OnlineLogisticRegression() {
+        ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public OnlineLogisticRegressionModel fit(Table... inputs) {
+        Preconditions.checkArgument(inputs.length == 1);
+
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) 
inputs[0]).getTableEnvironment();
+        DataStream<LogisticRegressionModelData> modelDataStream =
+                
LogisticRegressionModelData.getModelDataStream(initModelDataTable);
+
+        DataStream<Row> points =
+                tEnv.toDataStream(inputs[0])
+                        .map(new FeaturesExtractor(getFeaturesCol(), 
getLabelCol()));
+
+        DataStream<DenseVector> initModelData =
+                modelDataStream.map(
+                        (MapFunction<LogisticRegressionModelData, DenseVector>)
+                                value -> value.coefficient);
+
+        initModelData.getTransformation().setParallelism(1);
+
+        IterationBody body =
+                new FtrlIterationBody(
+                        getGlobalBatchSize(),
+                        getAlpha(),
+                        getBeta(),
+                        getReg(),
+                        getElasticNet(),
+                        getModelSaveInterval());
+
+        DataStream<LogisticRegressionModelData> onlineModelData =
+                Iterations.iterateUnboundedStreams(
+                                DataStreamList.of(initModelData), 
DataStreamList.of(points), body)
+                        .get(0);
+
+        Table onlineModelDataTable = tEnv.fromDataStream(onlineModelData);
+        OnlineLogisticRegressionModel model =
+                new 
OnlineLogisticRegressionModel().setModelData(onlineModelDataTable);
+        ReadWriteUtils.updateExistingParams(model, paramMap);
+        return model;
+    }
+
+    private static class FeaturesExtractor implements MapFunction<Row, Row> {
+        private final String featuresCol;
+        private final String labelCol;
+
+        private FeaturesExtractor(String featuresCol, String labelCol) {
+            this.featuresCol = featuresCol;
+            this.labelCol = labelCol;
+        }
+
+        @Override
+        public Row map(Row row) throws Exception {
+            return Row.of(row.getField(featuresCol), row.getField(labelCol));
+        }
+    }
+
+    /**
+     * Implementation of ftrl optimizer. In this implementation, gradients are 
calculated in
+     * distributed workers and reduce to one gradient. The reduced gradient is 
used to update model
+     * by ftrl method.
+     *
+     * <p>See 
https://www.tensorflow.org/api_docs/python/tf/keras/optimizers/Ftrl
+     */
+    private static class FtrlIterationBody implements IterationBody {
+        private final int batchSize;
+        private final double alpha;
+        private final double beta;
+        private final double l1;
+        private final double l2;
+        private long modelVersion = 1L;
+        private final int modelSaveInterval;
+
+        public FtrlIterationBody(
+                int batchSize,
+                double alpha,
+                double beta,
+                double reg,
+                double elasticNet,
+                int modelSaveInterval) {
+            this.batchSize = batchSize;
+            this.alpha = alpha;
+            this.beta = beta;
+            this.l1 = elasticNet * reg;
+            this.l2 = (1 - elasticNet) * reg;
+            this.modelSaveInterval = modelSaveInterval;
+        }
+
+        @Override
+        public IterationBodyResult process(
+                DataStreamList variableStreams, DataStreamList dataStreams) {
+            DataStream<DenseVector> modelData = variableStreams.get(0);
+
+            DataStream<Row> points = dataStreams.get(0);
+            int parallelism = points.getParallelism();
+            Preconditions.checkState(
+                    parallelism <= batchSize,
+                    "There are more subtasks in the training process than the 
number "
+                            + "of elements in each batch. Some subtasks might 
be idling forever.");
+
+            DataStream<DenseVector[]> newGradient =
+                    DataStreamUtils.generateBatchData(points, parallelism, 
batchSize)
+                            .connect(modelData.broadcast())
+                            .transform(
+                                    "LocalGradientCalculator",
+                                    TypeInformation.of(DenseVector[].class),
+                                    new CalculateLocalGradient())
+                            .setParallelism(parallelism)
+                            .countWindowAll(parallelism)
+                            .reduce(
+                                    (ReduceFunction<DenseVector[]>)
+                                            (gradientInfo, newGradientInfo) -> 
{
+                                                for (int i = 0;
+                                                        i < 
newGradientInfo[1].size();
+                                                        ++i) {
+                                                    
newGradientInfo[0].values[i] =
+                                                            
gradientInfo[0].values[i]
+                                                                    + 
newGradientInfo[0].values[i];
+                                                    
newGradientInfo[1].values[i] =
+                                                            
gradientInfo[1].values[i]
+                                                                    + 
newGradientInfo[1].values[i];
+                                                    if (newGradientInfo[2] == 
null) {
+                                                        newGradientInfo[2] = 
gradientInfo[2];
+                                                    }
+                                                }
+                                                return newGradientInfo;
+                                            });
+            DataStream<DenseVector> feedbackModelData =
+                    newGradient
+                            .transform(
+                                    "ModelDataUpdater",
+                                    TypeInformation.of(DenseVector.class),
+                                    new UpdateModel(alpha, beta, l1, l2))
+                            .setParallelism(1);
+
+            DataStream<LogisticRegressionModelData> outputModelData =
+                    feedbackModelData
+                            .filter(
+                                    new FilterFunction<DenseVector>() {
+                                        private int step = 0;
+
+                                        @Override
+                                        public boolean filter(DenseVector 
denseVector) {
+                                            step++;
+                                            return step % modelSaveInterval == 
0;
+                                        }
+                                    })
+                            .setParallelism(1)
+                            .map(
+                                    (MapFunction<DenseVector, 
LogisticRegressionModelData>)
+                                            value ->
+                                                    new 
LogisticRegressionModelData(
+                                                            value, 
modelVersion++))
+                            .setParallelism(1);
+            return new IterationBodyResult(
+                    DataStreamList.of(feedbackModelData), 
DataStreamList.of(outputModelData));
+        }
+    }
+
+    /** Updates model. */
+    private static class UpdateModel extends 
AbstractStreamOperator<DenseVector>
+            implements OneInputStreamOperator<DenseVector[], DenseVector> {
+        private ListState<double[]> nParamState;
+        private ListState<double[]> zParamState;
+        private final double alpha;
+        private final double beta;
+        private final double l1;
+        private final double l2;
+        private double[] nParam;
+        private double[] zParam;
+
+        public UpdateModel(double alpha, double beta, double l1, double l2) {
+            this.alpha = alpha;
+            this.beta = beta;
+            this.l1 = l1;
+            this.l2 = l2;
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws 
Exception {
+            super.initializeState(context);
+            nParamState =
+                    context.getOperatorStateStore()
+                            .getListState(new 
ListStateDescriptor<>("nParamState", double[].class));
+            zParamState =
+                    context.getOperatorStateStore()
+                            .getListState(new 
ListStateDescriptor<>("zParamState", double[].class));
+        }
+
+        @Override
+        public void processElement(StreamRecord<DenseVector[]> streamRecord) 
throws Exception {
+            DenseVector[] gradientInfo = streamRecord.getValue();
+            double[] coefficient = gradientInfo[2].values;
+            double[] g = gradientInfo[0].values;
+            for (int i = 0; i < g.length; ++i) {
+                if (gradientInfo[1].values[i] != 0.0) {
+                    g[i] = g[i] / gradientInfo[1].values[i];
+                }
+            }
+            if (zParam == null) {
+                zParam = new double[g.length];
+                nParam = new double[g.length];
+                nParamState.add(nParam);
+                zParamState.add(zParam);
+            }
+
+            for (int i = 0; i < zParam.length; ++i) {
+                double sigma = (Math.sqrt(nParam[i] + g[i] * g[i]) - 
Math.sqrt(nParam[i])) / alpha;
+                zParam[i] += g[i] - sigma * coefficient[i];
+                nParam[i] += g[i] * g[i];
+
+                if (Math.abs(zParam[i]) <= l1) {
+                    coefficient[i] = 0.0;
+                } else {
+                    coefficient[i] =
+                            ((zParam[i] < 0 ? -1 : 1) * l1 - zParam[i])
+                                    / ((beta + Math.sqrt(nParam[i])) / alpha + 
l2);
+                }
+            }
+            output.collect(new StreamRecord<>(new DenseVector(coefficient)));
+        }
+    }
+
+    private static class CalculateLocalGradient extends 
AbstractStreamOperator<DenseVector[]>

Review Comment:
   Is it possible for us to reuse codes in 
`org.apache.flink.ml.common.lossfunc` and 
`org.apache.flink.ml.common.optimizer` for the implementation of this class? Or 
shall we make FTRL a common optimizer and place it in 
`org.apache.flink.ml.common`?



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/classification/logisticregression/OnlineLogisticRegression.java:
##########
@@ -0,0 +1,434 @@
+/*
+ * 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.logisticregression;
+
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+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.java.typeutils.ObjectArrayTypeInfo;
+import org.apache.flink.iteration.DataStreamList;
+import org.apache.flink.iteration.IterationBody;
+import org.apache.flink.iteration.IterationBodyResult;
+import org.apache.flink.iteration.Iterations;
+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.linalg.DenseVector;
+import org.apache.flink.ml.linalg.SparseVector;
+import org.apache.flink.ml.linalg.Vector;
+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.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+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.types.Row;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.commons.collections.IteratorUtils;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * An Estimator which implements the FTRL-Proximal online learning algorithm 
proposed by H. Brendan
+ * McMahan et al.
+ *
+ * <p>See <a href="https://doi.org/10.1145/2487575.2488200";>H. Brendan McMahan 
et al., Ad click
+ * prediction: a view from the trenches.</a>
+ */
+public class OnlineLogisticRegression
+        implements Estimator<OnlineLogisticRegression, 
OnlineLogisticRegressionModel>,
+                OnlineLogisticRegressionParams<OnlineLogisticRegression> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+    private Table initModelDataTable;
+
+    public OnlineLogisticRegression() {
+        ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public OnlineLogisticRegressionModel fit(Table... inputs) {
+        Preconditions.checkArgument(inputs.length == 1);
+
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) 
inputs[0]).getTableEnvironment();
+        DataStream<LogisticRegressionModelData> modelDataStream =
+                
LogisticRegressionModelData.getModelDataStream(initModelDataTable);
+
+        DataStream<Row> points =
+                tEnv.toDataStream(inputs[0])
+                        .map(new FeaturesExtractor(getFeaturesCol(), 
getLabelCol()));
+
+        DataStream<DenseVector> initModelData =
+                modelDataStream.map(
+                        (MapFunction<LogisticRegressionModelData, DenseVector>)
+                                value -> value.coefficient);
+
+        initModelData.getTransformation().setParallelism(1);
+
+        IterationBody body =
+                new FtrlIterationBody(
+                        getGlobalBatchSize(),
+                        getAlpha(),
+                        getBeta(),
+                        getReg(),
+                        getElasticNet(),
+                        getModelSaveInterval());
+
+        DataStream<LogisticRegressionModelData> onlineModelData =
+                Iterations.iterateUnboundedStreams(
+                                DataStreamList.of(initModelData), 
DataStreamList.of(points), body)
+                        .get(0);
+
+        Table onlineModelDataTable = tEnv.fromDataStream(onlineModelData);
+        OnlineLogisticRegressionModel model =
+                new 
OnlineLogisticRegressionModel().setModelData(onlineModelDataTable);
+        ReadWriteUtils.updateExistingParams(model, paramMap);
+        return model;
+    }
+
+    private static class FeaturesExtractor implements MapFunction<Row, Row> {
+        private final String featuresCol;
+        private final String labelCol;
+
+        private FeaturesExtractor(String featuresCol, String labelCol) {
+            this.featuresCol = featuresCol;
+            this.labelCol = labelCol;
+        }
+
+        @Override
+        public Row map(Row row) throws Exception {
+            return Row.of(row.getField(featuresCol), row.getField(labelCol));
+        }
+    }
+
+    /**
+     * Implementation of ftrl optimizer. In this implementation, gradients are 
calculated in
+     * distributed workers and reduce to one gradient. The reduced gradient is 
used to update model
+     * by ftrl method.
+     *
+     * <p>See 
https://www.tensorflow.org/api_docs/python/tf/keras/optimizers/Ftrl
+     */
+    private static class FtrlIterationBody implements IterationBody {
+        private final int batchSize;
+        private final double alpha;
+        private final double beta;
+        private final double l1;
+        private final double l2;
+        private long modelVersion = 1L;
+        private final int modelSaveInterval;
+
+        public FtrlIterationBody(
+                int batchSize,
+                double alpha,
+                double beta,
+                double reg,
+                double elasticNet,
+                int modelSaveInterval) {
+            this.batchSize = batchSize;
+            this.alpha = alpha;
+            this.beta = beta;
+            this.l1 = elasticNet * reg;
+            this.l2 = (1 - elasticNet) * reg;
+            this.modelSaveInterval = modelSaveInterval;
+        }
+
+        @Override
+        public IterationBodyResult process(
+                DataStreamList variableStreams, DataStreamList dataStreams) {
+            DataStream<DenseVector> modelData = variableStreams.get(0);
+
+            DataStream<Row> points = dataStreams.get(0);
+            int parallelism = points.getParallelism();
+            Preconditions.checkState(
+                    parallelism <= batchSize,
+                    "There are more subtasks in the training process than the 
number "
+                            + "of elements in each batch. Some subtasks might 
be idling forever.");
+
+            DataStream<DenseVector[]> newGradient =
+                    DataStreamUtils.generateBatchData(points, parallelism, 
batchSize)
+                            .connect(modelData.broadcast())
+                            .transform(
+                                    "LocalGradientCalculator",
+                                    TypeInformation.of(DenseVector[].class),
+                                    new CalculateLocalGradient())
+                            .setParallelism(parallelism)
+                            .countWindowAll(parallelism)
+                            .reduce(
+                                    (ReduceFunction<DenseVector[]>)
+                                            (gradientInfo, newGradientInfo) -> 
{
+                                                for (int i = 0;
+                                                        i < 
newGradientInfo[1].size();
+                                                        ++i) {
+                                                    
newGradientInfo[0].values[i] =
+                                                            
gradientInfo[0].values[i]
+                                                                    + 
newGradientInfo[0].values[i];
+                                                    
newGradientInfo[1].values[i] =
+                                                            
gradientInfo[1].values[i]
+                                                                    + 
newGradientInfo[1].values[i];
+                                                    if (newGradientInfo[2] == 
null) {
+                                                        newGradientInfo[2] = 
gradientInfo[2];
+                                                    }
+                                                }
+                                                return newGradientInfo;
+                                            });
+            DataStream<DenseVector> feedbackModelData =
+                    newGradient
+                            .transform(
+                                    "ModelDataUpdater",
+                                    TypeInformation.of(DenseVector.class),
+                                    new UpdateModel(alpha, beta, l1, l2))
+                            .setParallelism(1);
+
+            DataStream<LogisticRegressionModelData> outputModelData =
+                    feedbackModelData
+                            .filter(
+                                    new FilterFunction<DenseVector>() {
+                                        private int step = 0;
+
+                                        @Override
+                                        public boolean filter(DenseVector 
denseVector) {
+                                            step++;
+                                            return step % modelSaveInterval == 
0;
+                                        }
+                                    })
+                            .setParallelism(1)
+                            .map(
+                                    (MapFunction<DenseVector, 
LogisticRegressionModelData>)
+                                            value ->
+                                                    new 
LogisticRegressionModelData(
+                                                            value, 
modelVersion++))
+                            .setParallelism(1);
+            return new IterationBodyResult(
+                    DataStreamList.of(feedbackModelData), 
DataStreamList.of(outputModelData));
+        }
+    }
+
+    /** Updates model. */
+    private static class UpdateModel extends 
AbstractStreamOperator<DenseVector>
+            implements OneInputStreamOperator<DenseVector[], DenseVector> {
+        private ListState<double[]> nParamState;
+        private ListState<double[]> zParamState;
+        private final double alpha;
+        private final double beta;
+        private final double l1;
+        private final double l2;
+        private double[] nParam;
+        private double[] zParam;
+
+        public UpdateModel(double alpha, double beta, double l1, double l2) {
+            this.alpha = alpha;
+            this.beta = beta;
+            this.l1 = l1;
+            this.l2 = l2;
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws 
Exception {
+            super.initializeState(context);
+            nParamState =
+                    context.getOperatorStateStore()
+                            .getListState(new 
ListStateDescriptor<>("nParamState", double[].class));
+            zParamState =
+                    context.getOperatorStateStore()
+                            .getListState(new 
ListStateDescriptor<>("zParamState", double[].class));
+        }
+
+        @Override
+        public void processElement(StreamRecord<DenseVector[]> streamRecord) 
throws Exception {
+            DenseVector[] gradientInfo = streamRecord.getValue();
+            double[] coefficient = gradientInfo[2].values;
+            double[] g = gradientInfo[0].values;
+            for (int i = 0; i < g.length; ++i) {
+                if (gradientInfo[1].values[i] != 0.0) {
+                    g[i] = g[i] / gradientInfo[1].values[i];
+                }
+            }
+            if (zParam == null) {
+                zParam = new double[g.length];
+                nParam = new double[g.length];
+                nParamState.add(nParam);
+                zParamState.add(zParam);
+            }
+
+            for (int i = 0; i < zParam.length; ++i) {
+                double sigma = (Math.sqrt(nParam[i] + g[i] * g[i]) - 
Math.sqrt(nParam[i])) / alpha;
+                zParam[i] += g[i] - sigma * coefficient[i];
+                nParam[i] += g[i] * g[i];
+
+                if (Math.abs(zParam[i]) <= l1) {
+                    coefficient[i] = 0.0;
+                } else {
+                    coefficient[i] =
+                            ((zParam[i] < 0 ? -1 : 1) * l1 - zParam[i])
+                                    / ((beta + Math.sqrt(nParam[i])) / alpha + 
l2);
+                }
+            }
+            output.collect(new StreamRecord<>(new DenseVector(coefficient)));
+        }
+    }
+
+    private static class CalculateLocalGradient extends 
AbstractStreamOperator<DenseVector[]>
+            implements TwoInputStreamOperator<Row[], DenseVector, 
DenseVector[]> {
+        private ListState<DenseVector> modelDataState;
+        private ListState<Row[]> localBatchDataState;
+        private double[] gradient;
+        private double[] weight;
+        private int[] denseVectorIndices;
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws 
Exception {
+            super.initializeState(context);
+            modelDataState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>("modelData", 
DenseVector.class));
+            TypeInformation<Row[]> type =
+                    
ObjectArrayTypeInfo.getInfoFor(TypeInformation.of(Row.class));
+            localBatchDataState =
+                    context.getOperatorStateStore()
+                            .getListState(new 
ListStateDescriptor<>("localBatch", type));
+        }
+
+        @Override
+        public void processElement1(StreamRecord<Row[]> pointsRecord) throws 
Exception {
+            localBatchDataState.add(pointsRecord.getValue());
+            calculateGradient();
+        }
+
+        private void calculateGradient() throws Exception {
+            if (!modelDataState.get().iterator().hasNext()
+                    || !localBatchDataState.get().iterator().hasNext()) {
+                return;
+            }
+            DenseVector modelData =
+                    OperatorStateUtils.getUniqueElement(modelDataState, 
"modelData").get();
+            modelDataState.clear();
+
+            List<Row[]> pointsList = 
IteratorUtils.toList(localBatchDataState.get().iterator());
+            Row[] points = pointsList.remove(0);
+            localBatchDataState.update(pointsList);
+
+            for (Row point : points) {
+                Vector vec = point.getFieldAs(0);
+                double label = point.getFieldAs(1);
+                if (gradient == null) {
+                    gradient = new double[vec.size()];
+                    weight = new double[gradient.length];
+                    if (vec instanceof DenseVector) {
+                        denseVectorIndices = new int[vec.size()];
+                        for (int i = 0; i < denseVectorIndices.length; ++i) {
+                            denseVectorIndices[i] = i;
+                        }
+                    }
+                }
+
+                int[] indices;
+                double[] values;
+                if (vec instanceof DenseVector) {
+                    DenseVector denseVector = (DenseVector) vec;
+                    indices = denseVectorIndices;
+                    values = denseVector.values;
+                } else {
+                    SparseVector sparseVector = (SparseVector) vec;
+                    indices = sparseVector.indices;
+                    values = sparseVector.values;
+                }
+                double p = 0.0;
+                for (int i = 0; i < indices.length; ++i) {
+                    int idx = indices[i];
+                    p += modelData.values[idx] * values[i];
+                }
+                p = 1 / (1 + Math.exp(-p));
+                for (int i = 0; i < indices.length; ++i) {
+                    int idx = indices[i];
+                    gradient[idx] += (p - label) * values[i];
+                    weight[idx] += 1.0;
+                }
+            }
+
+            if (points.length > 0) {
+                output.collect(
+                        new StreamRecord<>(
+                                new DenseVector[] {
+                                    new DenseVector(gradient),
+                                    new DenseVector(weight),
+                                    
(getRuntimeContext().getIndexOfThisSubtask() == 0)
+                                            ? modelData
+                                            : null
+                                }));
+            }
+            Arrays.fill(gradient, 0.0);
+            Arrays.fill(weight, 0.0);
+        }
+
+        @Override
+        public void processElement2(StreamRecord<DenseVector> modelDataRecord) 
throws Exception {
+            modelDataState.add(modelDataRecord.getValue());
+            calculateGradient();
+        }
+    }
+
+    @Override
+    public void save(String path) throws IOException {
+        ReadWriteUtils.saveMetadata(this, path);
+        ReadWriteUtils.saveModelData(

Review Comment:
   In `OnlineKMeans`'s PR it has been discussed that a stage should not attempt 
to save and load unbounded stream. If an online algorithm is to be reloaded, 
the model data stream should be externally set instead of being loaded.



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/classification/OnlineLogisticRegressionTest.java:
##########
@@ -0,0 +1,661 @@
+/*
+ * 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;
+
+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.configuration.RestOptions;
+import org.apache.flink.metrics.Gauge;
+import 
org.apache.flink.ml.classification.logisticregression.LogisticRegression;
+import 
org.apache.flink.ml.classification.logisticregression.LogisticRegressionModel;
+import 
org.apache.flink.ml.classification.logisticregression.LogisticRegressionModelData;
+import 
org.apache.flink.ml.classification.logisticregression.OnlineLogisticRegression;
+import 
org.apache.flink.ml.classification.logisticregression.OnlineLogisticRegressionModel;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.linalg.SparseVector;
+import org.apache.flink.ml.linalg.Vectors;
+import org.apache.flink.ml.util.InMemorySinkFunction;
+import org.apache.flink.ml.util.InMemorySourceFunction;
+import org.apache.flink.ml.util.TestUtils;
+import org.apache.flink.runtime.minicluster.MiniCluster;
+import org.apache.flink.runtime.minicluster.MiniClusterConfiguration;
+import org.apache.flink.runtime.testutils.InMemoryReporter;
+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.flink.util.TestLogger;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import static 
org.apache.flink.ml.classification.logisticregression.OnlineLogisticRegressionModel.MODEL_DATA_VERSION_GAUGE_KEY;
+
+/** Tests {@link OnlineLogisticRegression} and {@link 
OnlineLogisticRegressionModel}. */
+public class OnlineLogisticRegressionTest extends TestLogger {
+    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+
+    private static final double[] ONE_ARRAY = new double[] {1.0, 1.0, 1.0};
+
+    private static final Row[] TRAIN_DENSE_ROWS_1 =
+            new Row[] {
+                Row.of(Vectors.dense(0.1, 2.), 0.),
+                Row.of(Vectors.dense(0.2, 2.), 0.),
+                Row.of(Vectors.dense(0.3, 2.), 0.),
+                Row.of(Vectors.dense(0.4, 2.), 0.),
+                Row.of(Vectors.dense(0.5, 2.), 0.),
+                Row.of(Vectors.dense(11., 12.), 1.),
+                Row.of(Vectors.dense(12., 11.), 1.),
+                Row.of(Vectors.dense(13., 12.), 1.),
+                Row.of(Vectors.dense(14., 12.), 1.),
+                Row.of(Vectors.dense(15., 12.), 1.)
+            };
+
+    private static final Row[] TRAIN_DENSE_ROWS_2 =
+            new Row[] {
+                Row.of(Vectors.dense(0.2, 3.), 0.),
+                Row.of(Vectors.dense(0.8, 1.), 0.),
+                Row.of(Vectors.dense(0.7, 1.), 0.),
+                Row.of(Vectors.dense(0.6, 2.), 0.),
+                Row.of(Vectors.dense(0.2, 2.), 0.),
+                Row.of(Vectors.dense(14., 17.), 1.),
+                Row.of(Vectors.dense(15., 10.), 1.),
+                Row.of(Vectors.dense(16., 16.), 1.),
+                Row.of(Vectors.dense(17., 10.), 1.),
+                Row.of(Vectors.dense(18., 13.), 1.)
+            };
+
+    private static final Row[] PREDICT_DENSE_ROWS =
+            new Row[] {
+                Row.of(Vectors.dense(0.8, 2.7), 0.0), 
Row.of(Vectors.dense(15.5, 11.2), 1.0)
+            };
+
+    private static final Row[] TRAIN_SPARSE_ROWS_1 =
+            new Row[] {
+                Row.of(Vectors.sparse(10, new int[] {1, 3, 4}, ONE_ARRAY), 0.),
+                Row.of(Vectors.sparse(10, new int[] {0, 2, 3}, ONE_ARRAY), 0.),
+                Row.of(Vectors.sparse(10, new int[] {0, 3, 4}, ONE_ARRAY), 0.),
+                Row.of(Vectors.sparse(10, new int[] {2, 3, 4}, ONE_ARRAY), 0.),
+                Row.of(Vectors.sparse(10, new int[] {1, 3, 4}, ONE_ARRAY), 0.),
+                Row.of(Vectors.sparse(10, new int[] {6, 7, 8}, ONE_ARRAY), 1.),
+                Row.of(Vectors.sparse(10, new int[] {6, 8, 9}, ONE_ARRAY), 1.),
+                Row.of(Vectors.sparse(10, new int[] {5, 8, 9}, ONE_ARRAY), 1.),
+                Row.of(Vectors.sparse(10, new int[] {5, 6, 7}, ONE_ARRAY), 1.)

Review Comment:
   Shall we use the same train/predict data for dense and sparse vector? That 
should simplify the code structure and make it clearer that the algorithm can 
produce the same results no matter which data type input data is.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/classification/logisticregression/OnlineLogisticRegression.java:
##########
@@ -0,0 +1,434 @@
+/*
+ * 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.logisticregression;
+
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+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.java.typeutils.ObjectArrayTypeInfo;
+import org.apache.flink.iteration.DataStreamList;
+import org.apache.flink.iteration.IterationBody;
+import org.apache.flink.iteration.IterationBodyResult;
+import org.apache.flink.iteration.Iterations;
+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.linalg.DenseVector;
+import org.apache.flink.ml.linalg.SparseVector;
+import org.apache.flink.ml.linalg.Vector;
+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.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+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.types.Row;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.commons.collections.IteratorUtils;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * An Estimator which implements the FTRL-Proximal online learning algorithm 
proposed by H. Brendan
+ * McMahan et al.
+ *
+ * <p>See <a href="https://doi.org/10.1145/2487575.2488200";>H. Brendan McMahan 
et al., Ad click
+ * prediction: a view from the trenches.</a>
+ */
+public class OnlineLogisticRegression
+        implements Estimator<OnlineLogisticRegression, 
OnlineLogisticRegressionModel>,
+                OnlineLogisticRegressionParams<OnlineLogisticRegression> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+    private Table initModelDataTable;
+
+    public OnlineLogisticRegression() {
+        ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public OnlineLogisticRegressionModel fit(Table... inputs) {
+        Preconditions.checkArgument(inputs.length == 1);
+
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) 
inputs[0]).getTableEnvironment();
+        DataStream<LogisticRegressionModelData> modelDataStream =
+                
LogisticRegressionModelData.getModelDataStream(initModelDataTable);
+
+        DataStream<Row> points =
+                tEnv.toDataStream(inputs[0])
+                        .map(new FeaturesExtractor(getFeaturesCol(), 
getLabelCol()));
+
+        DataStream<DenseVector> initModelData =
+                modelDataStream.map(
+                        (MapFunction<LogisticRegressionModelData, DenseVector>)
+                                value -> value.coefficient);
+
+        initModelData.getTransformation().setParallelism(1);
+
+        IterationBody body =
+                new FtrlIterationBody(
+                        getGlobalBatchSize(),
+                        getAlpha(),
+                        getBeta(),
+                        getReg(),
+                        getElasticNet(),
+                        getModelSaveInterval());
+
+        DataStream<LogisticRegressionModelData> onlineModelData =
+                Iterations.iterateUnboundedStreams(
+                                DataStreamList.of(initModelData), 
DataStreamList.of(points), body)
+                        .get(0);
+
+        Table onlineModelDataTable = tEnv.fromDataStream(onlineModelData);
+        OnlineLogisticRegressionModel model =
+                new 
OnlineLogisticRegressionModel().setModelData(onlineModelDataTable);
+        ReadWriteUtils.updateExistingParams(model, paramMap);
+        return model;
+    }
+
+    private static class FeaturesExtractor implements MapFunction<Row, Row> {
+        private final String featuresCol;
+        private final String labelCol;
+
+        private FeaturesExtractor(String featuresCol, String labelCol) {
+            this.featuresCol = featuresCol;
+            this.labelCol = labelCol;
+        }
+
+        @Override
+        public Row map(Row row) throws Exception {
+            return Row.of(row.getField(featuresCol), row.getField(labelCol));
+        }
+    }
+
+    /**
+     * Implementation of ftrl optimizer. In this implementation, gradients are 
calculated in
+     * distributed workers and reduce to one gradient. The reduced gradient is 
used to update model
+     * by ftrl method.
+     *
+     * <p>See 
https://www.tensorflow.org/api_docs/python/tf/keras/optimizers/Ftrl
+     */
+    private static class FtrlIterationBody implements IterationBody {
+        private final int batchSize;
+        private final double alpha;
+        private final double beta;
+        private final double l1;
+        private final double l2;
+        private long modelVersion = 1L;
+        private final int modelSaveInterval;
+
+        public FtrlIterationBody(
+                int batchSize,
+                double alpha,
+                double beta,
+                double reg,
+                double elasticNet,
+                int modelSaveInterval) {
+            this.batchSize = batchSize;
+            this.alpha = alpha;
+            this.beta = beta;
+            this.l1 = elasticNet * reg;
+            this.l2 = (1 - elasticNet) * reg;
+            this.modelSaveInterval = modelSaveInterval;
+        }
+
+        @Override
+        public IterationBodyResult process(
+                DataStreamList variableStreams, DataStreamList dataStreams) {
+            DataStream<DenseVector> modelData = variableStreams.get(0);
+
+            DataStream<Row> points = dataStreams.get(0);
+            int parallelism = points.getParallelism();
+            Preconditions.checkState(
+                    parallelism <= batchSize,
+                    "There are more subtasks in the training process than the 
number "
+                            + "of elements in each batch. Some subtasks might 
be idling forever.");
+
+            DataStream<DenseVector[]> newGradient =
+                    DataStreamUtils.generateBatchData(points, parallelism, 
batchSize)
+                            .connect(modelData.broadcast())
+                            .transform(
+                                    "LocalGradientCalculator",
+                                    TypeInformation.of(DenseVector[].class),
+                                    new CalculateLocalGradient())
+                            .setParallelism(parallelism)
+                            .countWindowAll(parallelism)
+                            .reduce(
+                                    (ReduceFunction<DenseVector[]>)
+                                            (gradientInfo, newGradientInfo) -> 
{
+                                                for (int i = 0;
+                                                        i < 
newGradientInfo[1].size();
+                                                        ++i) {
+                                                    
newGradientInfo[0].values[i] =
+                                                            
gradientInfo[0].values[i]
+                                                                    + 
newGradientInfo[0].values[i];
+                                                    
newGradientInfo[1].values[i] =
+                                                            
gradientInfo[1].values[i]
+                                                                    + 
newGradientInfo[1].values[i];
+                                                    if (newGradientInfo[2] == 
null) {
+                                                        newGradientInfo[2] = 
gradientInfo[2];
+                                                    }
+                                                }
+                                                return newGradientInfo;
+                                            });
+            DataStream<DenseVector> feedbackModelData =
+                    newGradient
+                            .transform(
+                                    "ModelDataUpdater",
+                                    TypeInformation.of(DenseVector.class),
+                                    new UpdateModel(alpha, beta, l1, l2))
+                            .setParallelism(1);
+
+            DataStream<LogisticRegressionModelData> outputModelData =
+                    feedbackModelData
+                            .filter(
+                                    new FilterFunction<DenseVector>() {
+                                        private int step = 0;
+
+                                        @Override
+                                        public boolean filter(DenseVector 
denseVector) {
+                                            step++;
+                                            return step % modelSaveInterval == 
0;
+                                        }
+                                    })
+                            .setParallelism(1)
+                            .map(
+                                    (MapFunction<DenseVector, 
LogisticRegressionModelData>)
+                                            value ->
+                                                    new 
LogisticRegressionModelData(
+                                                            value, 
modelVersion++))

Review Comment:
   Some values, like `modelVersion` or `step`, should be checkpoint and 
restored from snapshot during failover, while the current implementation has 
not achieved this. Could you please check all variables and see if they need to 
be stored in state?



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/classification/logisticregression/OnlineLogisticRegression.java:
##########
@@ -0,0 +1,434 @@
+/*
+ * 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.logisticregression;
+
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+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.java.typeutils.ObjectArrayTypeInfo;
+import org.apache.flink.iteration.DataStreamList;
+import org.apache.flink.iteration.IterationBody;
+import org.apache.flink.iteration.IterationBodyResult;
+import org.apache.flink.iteration.Iterations;
+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.linalg.DenseVector;
+import org.apache.flink.ml.linalg.SparseVector;
+import org.apache.flink.ml.linalg.Vector;
+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.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+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.types.Row;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.commons.collections.IteratorUtils;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * An Estimator which implements the FTRL-Proximal online learning algorithm 
proposed by H. Brendan
+ * McMahan et al.
+ *
+ * <p>See <a href="https://doi.org/10.1145/2487575.2488200";>H. Brendan McMahan 
et al., Ad click
+ * prediction: a view from the trenches.</a>
+ */
+public class OnlineLogisticRegression
+        implements Estimator<OnlineLogisticRegression, 
OnlineLogisticRegressionModel>,
+                OnlineLogisticRegressionParams<OnlineLogisticRegression> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+    private Table initModelDataTable;
+
+    public OnlineLogisticRegression() {
+        ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public OnlineLogisticRegressionModel fit(Table... inputs) {
+        Preconditions.checkArgument(inputs.length == 1);
+
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) 
inputs[0]).getTableEnvironment();
+        DataStream<LogisticRegressionModelData> modelDataStream =
+                
LogisticRegressionModelData.getModelDataStream(initModelDataTable);
+
+        DataStream<Row> points =
+                tEnv.toDataStream(inputs[0])
+                        .map(new FeaturesExtractor(getFeaturesCol(), 
getLabelCol()));
+
+        DataStream<DenseVector> initModelData =
+                modelDataStream.map(
+                        (MapFunction<LogisticRegressionModelData, DenseVector>)
+                                value -> value.coefficient);
+
+        initModelData.getTransformation().setParallelism(1);
+
+        IterationBody body =
+                new FtrlIterationBody(
+                        getGlobalBatchSize(),
+                        getAlpha(),
+                        getBeta(),
+                        getReg(),
+                        getElasticNet(),
+                        getModelSaveInterval());
+
+        DataStream<LogisticRegressionModelData> onlineModelData =
+                Iterations.iterateUnboundedStreams(
+                                DataStreamList.of(initModelData), 
DataStreamList.of(points), body)
+                        .get(0);
+
+        Table onlineModelDataTable = tEnv.fromDataStream(onlineModelData);
+        OnlineLogisticRegressionModel model =
+                new 
OnlineLogisticRegressionModel().setModelData(onlineModelDataTable);
+        ReadWriteUtils.updateExistingParams(model, paramMap);
+        return model;
+    }
+
+    private static class FeaturesExtractor implements MapFunction<Row, Row> {
+        private final String featuresCol;
+        private final String labelCol;
+
+        private FeaturesExtractor(String featuresCol, String labelCol) {
+            this.featuresCol = featuresCol;
+            this.labelCol = labelCol;
+        }
+
+        @Override
+        public Row map(Row row) throws Exception {
+            return Row.of(row.getField(featuresCol), row.getField(labelCol));
+        }
+    }
+
+    /**
+     * Implementation of ftrl optimizer. In this implementation, gradients are 
calculated in
+     * distributed workers and reduce to one gradient. The reduced gradient is 
used to update model
+     * by ftrl method.
+     *
+     * <p>See 
https://www.tensorflow.org/api_docs/python/tf/keras/optimizers/Ftrl
+     */
+    private static class FtrlIterationBody implements IterationBody {
+        private final int batchSize;
+        private final double alpha;
+        private final double beta;
+        private final double l1;
+        private final double l2;
+        private long modelVersion = 1L;
+        private final int modelSaveInterval;
+
+        public FtrlIterationBody(
+                int batchSize,
+                double alpha,
+                double beta,
+                double reg,
+                double elasticNet,
+                int modelSaveInterval) {
+            this.batchSize = batchSize;
+            this.alpha = alpha;
+            this.beta = beta;
+            this.l1 = elasticNet * reg;
+            this.l2 = (1 - elasticNet) * reg;
+            this.modelSaveInterval = modelSaveInterval;
+        }
+
+        @Override
+        public IterationBodyResult process(
+                DataStreamList variableStreams, DataStreamList dataStreams) {
+            DataStream<DenseVector> modelData = variableStreams.get(0);
+
+            DataStream<Row> points = dataStreams.get(0);
+            int parallelism = points.getParallelism();
+            Preconditions.checkState(
+                    parallelism <= batchSize,
+                    "There are more subtasks in the training process than the 
number "
+                            + "of elements in each batch. Some subtasks might 
be idling forever.");
+
+            DataStream<DenseVector[]> newGradient =
+                    DataStreamUtils.generateBatchData(points, parallelism, 
batchSize)
+                            .connect(modelData.broadcast())
+                            .transform(
+                                    "LocalGradientCalculator",
+                                    TypeInformation.of(DenseVector[].class),
+                                    new CalculateLocalGradient())
+                            .setParallelism(parallelism)
+                            .countWindowAll(parallelism)
+                            .reduce(
+                                    (ReduceFunction<DenseVector[]>)
+                                            (gradientInfo, newGradientInfo) -> 
{
+                                                for (int i = 0;
+                                                        i < 
newGradientInfo[1].size();
+                                                        ++i) {
+                                                    
newGradientInfo[0].values[i] =
+                                                            
gradientInfo[0].values[i]
+                                                                    + 
newGradientInfo[0].values[i];
+                                                    
newGradientInfo[1].values[i] =
+                                                            
gradientInfo[1].values[i]
+                                                                    + 
newGradientInfo[1].values[i];
+                                                    if (newGradientInfo[2] == 
null) {
+                                                        newGradientInfo[2] = 
gradientInfo[2];
+                                                    }
+                                                }
+                                                return newGradientInfo;
+                                            });
+            DataStream<DenseVector> feedbackModelData =
+                    newGradient
+                            .transform(
+                                    "ModelDataUpdater",
+                                    TypeInformation.of(DenseVector.class),
+                                    new UpdateModel(alpha, beta, l1, l2))
+                            .setParallelism(1);
+
+            DataStream<LogisticRegressionModelData> outputModelData =
+                    feedbackModelData
+                            .filter(
+                                    new FilterFunction<DenseVector>() {
+                                        private int step = 0;
+
+                                        @Override
+                                        public boolean filter(DenseVector 
denseVector) {
+                                            step++;
+                                            return step % modelSaveInterval == 
0;
+                                        }
+                                    })
+                            .setParallelism(1)

Review Comment:
   Do you think we should extract this part of code as infrastructure, like 
`org.apache.flink.ml.common.iteration.ForwardInputsOfLastRound`?



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/classification/logisticregression/OnlineLogisticRegression.java:
##########
@@ -0,0 +1,434 @@
+/*
+ * 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.logisticregression;
+
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+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.java.typeutils.ObjectArrayTypeInfo;
+import org.apache.flink.iteration.DataStreamList;
+import org.apache.flink.iteration.IterationBody;
+import org.apache.flink.iteration.IterationBodyResult;
+import org.apache.flink.iteration.Iterations;
+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.linalg.DenseVector;
+import org.apache.flink.ml.linalg.SparseVector;
+import org.apache.flink.ml.linalg.Vector;
+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.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+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.types.Row;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.commons.collections.IteratorUtils;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * An Estimator which implements the FTRL-Proximal online learning algorithm 
proposed by H. Brendan
+ * McMahan et al.
+ *
+ * <p>See <a href="https://doi.org/10.1145/2487575.2488200";>H. Brendan McMahan 
et al., Ad click
+ * prediction: a view from the trenches.</a>
+ */
+public class OnlineLogisticRegression
+        implements Estimator<OnlineLogisticRegression, 
OnlineLogisticRegressionModel>,
+                OnlineLogisticRegressionParams<OnlineLogisticRegression> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+    private Table initModelDataTable;
+
+    public OnlineLogisticRegression() {
+        ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")

Review Comment:
   nit: this could be removed.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/classification/logisticregression/LogisticRegressionModel.java:
##########
@@ -161,8 +163,12 @@ public Row map(Row dataPoint) {
      * @param coefficient The model parameters.
      * @return The prediction label and the raw probabilities.
      */
-    private static Row predictOneDataPoint(DenseVector feature, DenseVector 
coefficient) {
-        double dotValue = BLAS.dot(feature, coefficient);
+    public static Row predictOneDataPoint(Vector feature, DenseVector 
coefficient) {
+
+        double dotValue =
+                feature instanceof SparseVector
+                        ? BLAS.dot((SparseVector) feature, coefficient)
+                        : BLAS.dot((DenseVector) feature, coefficient);

Review Comment:
   nit: It might be better to add a `BLAS.dot(Vector, Vector)` method and make 
this type conversion in that method.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/classification/logisticregression/OnlineLogisticRegressionModel.java:
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.logisticregression;
+
+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.typeinfo.Types;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.ml.api.Model;
+import org.apache.flink.ml.common.datastream.TableUtils;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.linalg.Vector;
+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.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+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.types.Row;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.commons.lang3.ArrayUtils;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static 
org.apache.flink.ml.classification.logisticregression.LogisticRegressionModel.predictOneDataPoint;
+
+/**
+ * A Model which classifies data using the model data computed by {@link 
OnlineLogisticRegression}.
+ */
+public class OnlineLogisticRegressionModel
+        implements Model<OnlineLogisticRegressionModel>,
+                
OnlineLogisticRegressionModelParams<OnlineLogisticRegressionModel> {
+    public static final String MODEL_DATA_VERSION_GAUGE_KEY = 
"modelDataVersion";
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+    private Table modelDataTable;
+
+    public OnlineLogisticRegressionModel() {
+        ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+    }
+
+    @Override
+    public Table[] transform(Table... inputs) {
+        Preconditions.checkArgument(inputs.length == 1);
+
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) 
inputs[0]).getTableEnvironment();
+
+        RowTypeInfo inputTypeInfo = 
TableUtils.getRowTypeInfo(inputs[0].getResolvedSchema());
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        ArrayUtils.addAll(
+                                inputTypeInfo.getFieldTypes(),
+                                Types.DOUBLE,
+                                TypeInformation.of(DenseVector.class),
+                                Types.LONG),
+                        ArrayUtils.addAll(
+                                inputTypeInfo.getFieldNames(),
+                                getPredictionCol(),
+                                getRawPredictionCol(),
+                                getModelVersionCol()));
+
+        DataStream<Row> predictionResult =
+                tEnv.toDataStream(inputs[0])
+                        .connect(
+                                
LogisticRegressionModelData.getModelDataStream(modelDataTable)
+                                        .broadcast())
+                        .transform(
+                                "PredictLabelOperator",
+                                outputTypeInfo,
+                                new PredictLabelOperator(inputTypeInfo, 
getFeaturesCol()));
+
+        return new Table[] {tEnv.fromDataStream(predictionResult)};
+    }
+
+    /** A utility operator used for prediction. */
+    private static class PredictLabelOperator extends 
AbstractStreamOperator<Row>
+            implements TwoInputStreamOperator<Row, 
LogisticRegressionModelData, Row> {
+        private final RowTypeInfo inputTypeInfo;
+
+        private final String featuresCol;
+        private ListState<Row> bufferedPointsState;
+        private DenseVector coefficient;
+        private long modelDataVersion = 0L;
+
+        public PredictLabelOperator(RowTypeInfo inputTypeInfo, String 
featuresCol) {
+            this.inputTypeInfo = inputTypeInfo;
+            this.featuresCol = featuresCol;
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws 
Exception {
+            super.initializeState(context);
+            bufferedPointsState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new 
ListStateDescriptor<>("bufferedPoints", inputTypeInfo));
+        }
+
+        @Override
+        public void open() throws Exception {
+            super.open();
+
+            getRuntimeContext()
+                    .getMetricGroup()
+                    .gauge(
+                            MODEL_DATA_VERSION_GAUGE_KEY,
+                            (Gauge<String>) () -> 
Long.toString(modelDataVersion));
+        }
+
+        @Override
+        public void processElement1(StreamRecord<Row> streamRecord) throws 
Exception {
+            Row dataPoint = streamRecord.getValue();
+            if (coefficient == null) {
+                bufferedPointsState.add(dataPoint);
+                return;
+            }
+            Vector features = (Vector) dataPoint.getField(featuresCol);
+            Row predictionResult = predictOneDataPoint(features, coefficient);
+            output.collect(
+                    new StreamRecord<>(
+                            Row.join(
+                                    dataPoint,
+                                    Row.of(
+                                            predictionResult.getField(0),
+                                            predictionResult.getField(1),
+                                            modelDataVersion))));
+        }
+
+        @Override
+        public void processElement2(StreamRecord<LogisticRegressionModelData> 
streamRecord)
+                throws Exception {
+            LogisticRegressionModelData modelData = streamRecord.getValue();
+            coefficient = modelData.coefficient;
+            modelDataVersion = modelData.modelVersion;
+            for (Row dataPoint : bufferedPointsState.get()) {
+                processElement1(new StreamRecord<>(dataPoint));

Review Comment:
   nit: Shall we create some method like `processElement(Row)` and reuse this 
method in processElement1 and processElement2? Wrapping the value in 
StreamRecord and soon unwrap it in processElement1 seems not elegant.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to