zhipeng93 commented on code in PR #83: URL: https://github.com/apache/flink-ml/pull/83#discussion_r854856268
########## 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. */ + 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: Can we add the model version when the infra for model version is ready? -- 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]
