zhipeng93 commented on code in PR #83: URL: https://github.com/apache/flink-ml/pull/83#discussion_r868883335
########## flink-ml-lib/src/main/java/org/apache/flink/ml/classification/logisticregression/OnlineLogisticRegression.java: ########## @@ -0,0 +1,382 @@ +/* + * 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.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.GlobalBatchCreator; +import org.apache.flink.ml.common.datastream.DataStreamUtils.GlobalBatchSplitter; +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.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 java.io.IOException; +import java.util.HashMap; +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<DenseVector[]> initModelData = + modelDataStream.map( + (MapFunction<LogisticRegressionModelData, DenseVector[]>) + value -> new DenseVector[] {value.coefficient}); + + initModelData.getTransformation().setParallelism(1); + + IterationBody body = + new FtrlIterationBody( + getGlobalBatchSize(), + getAlpha(), + getBeta(), + getL1(), + getL2(), + getFeaturesCol(), + getLabelCol()); + + DataStream<LogisticRegressionModelData> onlineModelData = + Iterations.iterateUnboundedStreams( + DataStreamList.of(initModelData), + DataStreamList.of(tEnv.toDataStream(inputs[0])), + body) + .get(0); + + Table onlineModelDataTable = tEnv.fromDataStream(onlineModelData); + OnlineLogisticRegressionModel model = + new OnlineLogisticRegressionModel().setModelData(onlineModelDataTable); + ReadWriteUtils.updateExistingParams(model, paramMap); + return model; + } + + /** Iteration body of ftrl algorithm. */ + 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 final String featureCol; + private final String labelCol; + + public FtrlIterationBody( + int batchSize, + double alpha, + double beta, + double l1, + double l2, + String featureCol, + String labelCol) { + this.batchSize = batchSize; + this.alpha = alpha; + this.beta = beta; + this.l1 = l1; + this.l2 = l2; + this.featureCol = featureCol; + this.labelCol = labelCol; + } + + @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[]> newModelData = + points.countWindowAll(batchSize) + .apply(new GlobalBatchCreator<>()) + .flatMap(new GlobalBatchSplitter<>(parallelism)) + .rebalance() Review Comment: This may lead to a case where one downstream task does not receive one record and others receive more than one record. Is this expected? ########## flink-ml-core/src/main/java/org/apache/flink/ml/common/datastream/DataStreamUtils.java: ########## @@ -182,4 +191,49 @@ public void snapshotState(StateSnapshotContext context) throws Exception { } } } + + /** + * An operator that splits a global batch into evenly-sized local batches, and distributes them + * to downstream operator. + * + * @param <T> Data type of batch data. + */ + public static class GlobalBatchSplitter<T> implements FlatMapFunction<T[], T[]> { Review Comment: Is it better to put `GlobalBatchSplitter` and `GlobalBatchCreator` in separate classes? Also, could these two functions merged into one? ########## flink-ml-lib/src/main/java/org/apache/flink/ml/classification/logisticregression/OnlineLogisticRegression.java: ########## @@ -0,0 +1,428 @@ +/* + * 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.FlatMapFunction; +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.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.functions.windowing.AllWindowFunction; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; +import org.apache.flink.streaming.api.windowing.windows.GlobalWindow; +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.Collector; +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 OnlineLogisticRegression algorithm. + * + * <p>See https://en.wikipedia.org/wiki/Online_machine_learning. + */ +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<DenseVector[]> initModelData = + modelDataStream.map( + (MapFunction<LogisticRegressionModelData, DenseVector[]>) + value -> new DenseVector[] {value.coefficient}); + + initModelData.getTransformation().setParallelism(1); + + IterationBody body = + new FtrlIterationBody( + getGlobalBatchSize(), + getAlpha(), + getBETA(), + getL1(), + getL2(), + getFeaturesCol(), + getLabelCol()); + + DataStream<LogisticRegressionModelData> onlineModelData = + Iterations.iterateUnboundedStreams( + DataStreamList.of(initModelData), + DataStreamList.of(tEnv.toDataStream(inputs[0])), + body) + .get(0); + + Table onlineModelDataTable = tEnv.fromDataStream(onlineModelData); + OnlineLogisticRegressionModel model = + new OnlineLogisticRegressionModel().setModelData(onlineModelDataTable); + ReadWriteUtils.updateExistingParams(model, paramMap); + return model; + } + + /** Iteration body of ftrl algorithm. */ Review Comment: As I understand from Algorithm 1 in paper [1], there is no init model data for FTRL and the init implementation is sequential. And for the distributed implemenation, the only one I can find is Tensorflow#FTRL [2][3]. In this implementation, we accumulated the gradient from workers and then use gradient to update n and z, then w. The above process is quite different from the current implementation. Could you explain a bit about the difference here? [1] https://static.googleusercontent.com/media/research.google.com/en//pubs/archive/41159.pdf [2] https://www.tensorflow.org/api_docs/python/tf/keras/optimizers/Ftrl [3] https://github.com/keras-team/keras/blob/d8fcb9d4d4dad45080ecfdd575483653028f8eda/keras/optimizer_v2/ftrl.py#L216 ########## flink-ml-lib/src/main/java/org/apache/flink/ml/classification/logisticregression/OnlineLogisticRegression.java: ########## @@ -0,0 +1,382 @@ +/* + * 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.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.GlobalBatchCreator; +import org.apache.flink.ml.common.datastream.DataStreamUtils.GlobalBatchSplitter; +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.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 java.io.IOException; +import java.util.HashMap; +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<DenseVector[]> initModelData = + modelDataStream.map( + (MapFunction<LogisticRegressionModelData, DenseVector[]>) + value -> new DenseVector[] {value.coefficient}); + + initModelData.getTransformation().setParallelism(1); + + IterationBody body = + new FtrlIterationBody( + getGlobalBatchSize(), + getAlpha(), + getBeta(), + getL1(), + getL2(), + getFeaturesCol(), + getLabelCol()); + + DataStream<LogisticRegressionModelData> onlineModelData = + Iterations.iterateUnboundedStreams( + DataStreamList.of(initModelData), + DataStreamList.of(tEnv.toDataStream(inputs[0])), + body) + .get(0); + + Table onlineModelDataTable = tEnv.fromDataStream(onlineModelData); + OnlineLogisticRegressionModel model = + new OnlineLogisticRegressionModel().setModelData(onlineModelDataTable); + ReadWriteUtils.updateExistingParams(model, paramMap); + return model; + } + + /** Iteration body of ftrl algorithm. */ + 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 final String featureCol; + private final String labelCol; + + public FtrlIterationBody( + int batchSize, + double alpha, + double beta, + double l1, + double l2, + String featureCol, + String labelCol) { + this.batchSize = batchSize; + this.alpha = alpha; + this.beta = beta; + this.l1 = l1; + this.l2 = l2; + this.featureCol = featureCol; + this.labelCol = labelCol; + } + + @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[]> newModelData = + points.countWindowAll(batchSize) + .apply(new GlobalBatchCreator<>()) + .flatMap(new GlobalBatchSplitter<>(parallelism)) + .rebalance() + .connect(modelData.broadcast()) + .transform( + "ModelDataLocalUpdater", + TypeInformation.of(DenseVector[].class), + new ModelDataLocalUpdater( + alpha, beta, l1, l2, featureCol, labelCol)) + .setParallelism(parallelism) + .countWindowAll(parallelism) + .reduce(new ModelDataGlobalReducer()); + DataStream<DenseVector[]> feedbackModelData = + newModelData + .map( + (MapFunction<DenseVector[], DenseVector[]>) + value -> { + double[] z = value[1].values; + double[] n = value[2].values; + for (int i = 0; i < z.length; ++i) { + if (Math.abs(z[i]) <= l1) { + value[0].values[i] = 0.0; + } else { + value[0].values[i] = + ((z[i] < 0 ? -1 : 1) * l1 - z[i]) + / ((beta + Math.sqrt(n[i])) + / alpha + + l2); + } + } + return new DenseVector[] {value[0]}; + }) + .setParallelism(1); + + DataStream<LogisticRegressionModelData> outputModelData = + feedbackModelData + .map( + (MapFunction<DenseVector[], LogisticRegressionModelData>) + value -> + new LogisticRegressionModelData( + value[0], System.nanoTime())) Review Comment: I guess here you want to use the current timestamp as the model version. Could you follow the existing practice we have aggreed on in `OnlineKmeans` and use incremental counts? ########## flink-ml-lib/src/main/java/org/apache/flink/ml/classification/logisticregression/OnlineLogisticRegression.java: ########## @@ -0,0 +1,382 @@ +/* + * 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.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.GlobalBatchCreator; +import org.apache.flink.ml.common.datastream.DataStreamUtils.GlobalBatchSplitter; +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.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 java.io.IOException; +import java.util.HashMap; +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<DenseVector[]> initModelData = + modelDataStream.map( + (MapFunction<LogisticRegressionModelData, DenseVector[]>) + value -> new DenseVector[] {value.coefficient}); + + initModelData.getTransformation().setParallelism(1); + + IterationBody body = + new FtrlIterationBody( + getGlobalBatchSize(), + getAlpha(), + getBeta(), + getL1(), + getL2(), + getFeaturesCol(), + getLabelCol()); + + DataStream<LogisticRegressionModelData> onlineModelData = + Iterations.iterateUnboundedStreams( + DataStreamList.of(initModelData), + DataStreamList.of(tEnv.toDataStream(inputs[0])), + body) + .get(0); + + Table onlineModelDataTable = tEnv.fromDataStream(onlineModelData); + OnlineLogisticRegressionModel model = + new OnlineLogisticRegressionModel().setModelData(onlineModelDataTable); + ReadWriteUtils.updateExistingParams(model, paramMap); + return model; + } + + /** Iteration body of ftrl algorithm. */ + 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 final String featureCol; + private final String labelCol; + + public FtrlIterationBody( + int batchSize, + double alpha, + double beta, + double l1, + double l2, + String featureCol, + String labelCol) { + this.batchSize = batchSize; + this.alpha = alpha; + this.beta = beta; + this.l1 = l1; + this.l2 = l2; + this.featureCol = featureCol; + this.labelCol = labelCol; + } + + @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[]> newModelData = + points.countWindowAll(batchSize) + .apply(new GlobalBatchCreator<>()) + .flatMap(new GlobalBatchSplitter<>(parallelism)) + .rebalance() + .connect(modelData.broadcast()) + .transform( + "ModelDataLocalUpdater", + TypeInformation.of(DenseVector[].class), + new ModelDataLocalUpdater( + alpha, beta, l1, l2, featureCol, labelCol)) + .setParallelism(parallelism) + .countWindowAll(parallelism) + .reduce(new ModelDataGlobalReducer()); + DataStream<DenseVector[]> feedbackModelData = Review Comment: It seems that `feedbackModelData` contains only one DenseVector. How about we change the type to `DenseVector`? ########## flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java: ########## @@ -111,7 +101,7 @@ private static void verifyPredictionResult( (MapFunction<Row, DenseVector>) row -> (DenseVector) row.getField(outputCol)); List<DenseVector> result = IteratorUtils.toList(stream.executeAndCollect()); - result.sort(MinMaxScalerTest::compare); + result.sort(TestUtils::compare); Review Comment: nit: what about using `TestBaseUtils.compareResultCollections`? ########## flink-ml-lib/src/main/java/org/apache/flink/ml/classification/logisticregression/OnlineLogisticRegression.java: ########## @@ -0,0 +1,382 @@ +/* + * 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.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.GlobalBatchCreator; +import org.apache.flink.ml.common.datastream.DataStreamUtils.GlobalBatchSplitter; +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.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 java.io.IOException; +import java.util.HashMap; +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<DenseVector[]> initModelData = Review Comment: `initModelData` seems not a `DenseVector[]`, but a `LogisticRegressionModelData`? Could you do the refactor here? ########## flink-ml-core/src/main/java/org/apache/flink/ml/linalg/BLAS.java: ########## @@ -71,6 +71,21 @@ public static double dot(DenseVector x, DenseVector y) { return JAVA_BLAS.ddot(x.size(), x.values, 1, y.values, 1); } + /** x \cdot y . x maybe DenseVector or SparseVector. */ + public static double dot(Vector x, DenseVector y) { Review Comment: How about we refactor the code following the structure of `hdot` and exposing the following function to users: `public static double dot(Vector, Vector)`? Moreover, unit tests should be added to verify this implementation. ########## flink-ml-lib/src/test/java/org/apache/flink/ml/classification/LogisticRegressionTest.java: ########## @@ -232,7 +232,7 @@ public void testSaveLoadAndPredict() throws Exception { LogisticRegressionModel model = logisticRegression.fit(binomialDataTable); model = StageTestUtils.saveAndReload(tEnv, model, tempFolder.newFolder().getAbsolutePath()); assertEquals( - Collections.singletonList("coefficient"), + Arrays.asList("coefficient", "modelVersion"), Review Comment: Could also verify the model version in `testGetModelData`? -- 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]
