zhipeng93 commented on a change in pull request #28: URL: https://github.com/apache/flink-ml/pull/28#discussion_r753651488
########## File path: flink-ml-lib/src/main/java/org/apache/flink/ml/classification/linear/LogisticRegression.java ########## @@ -0,0 +1,594 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.ml.classification.linear; + +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.iteration.DataStreamList; +import org.apache.flink.iteration.IterationBody; +import org.apache.flink.iteration.IterationBodyResult; +import org.apache.flink.iteration.IterationConfig; +import org.apache.flink.iteration.IterationConfig.OperatorLifeCycle; +import org.apache.flink.iteration.IterationListener; +import org.apache.flink.iteration.Iterations; +import org.apache.flink.iteration.ReplayableDataStreamList; +import org.apache.flink.iteration.operator.OperatorStateUtils; +import org.apache.flink.ml.api.core.Estimator; +import org.apache.flink.ml.common.broadcast.BroadcastUtils; +import org.apache.flink.ml.common.datastream.AllReduceUtils; +import org.apache.flink.ml.common.datastream.DataStreamUtils; +import org.apache.flink.ml.common.linalg.BLAS; +import org.apache.flink.ml.param.Param; +import org.apache.flink.ml.util.ParamUtils; +import org.apache.flink.ml.util.ReadWriteUtils; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator; +import org.apache.flink.streaming.api.operators.BoundedMultiInput; +import org.apache.flink.streaming.api.operators.BoundedOneInput; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.internal.TableImpl; +import org.apache.flink.util.Collector; +import org.apache.flink.util.OutputTag; +import org.apache.flink.util.Preconditions; + +import org.apache.commons.collections.IteratorUtils; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; + +/** This class implements methods to train a logistic regression model. */ +public class LogisticRegression + implements Estimator<LogisticRegression, LogisticRegressionModel>, + LogisticRegressionParams<LogisticRegression> { + + Map<Param<?>, Object> paramMap; + + private static final OutputTag<Tuple2<double[], double[]>> MODEL_OUTPUT = + new OutputTag<Tuple2<double[], double[]>>("MODEL_OUTPUT") {}; + + public LogisticRegression(Map<Param<?>, Object> paramMap) { + this.paramMap = paramMap; + ParamUtils.initializeMapWithDefaultValues(this.paramMap, this); + } + + public LogisticRegression() { + this(new HashMap<>()); + } + + @Override + public Map<Param<?>, Object> getParamMap() { + return paramMap; + } + + @Override + public void save(String path) throws IOException { + ReadWriteUtils.saveMetadata(this, path); + } + + public static LogisticRegression load(StreamExecutionEnvironment env, String path) + throws IOException { + return ReadWriteUtils.loadStageParam(path); + } + + @Override + @SuppressWarnings("unchecked") + public LogisticRegressionModel fit(Table... inputs) { + Preconditions.checkArgument(inputs.length == 1); + StreamTableEnvironment tEnv = + (StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment(); + + DataStream<Tuple3<Double, Double, double[]>> trainData = + tEnv.toDataStream(inputs[0]) + .map( + row -> + Tuple3.of( + getWeightCol() == null + ? new Double(1.0) + : (Double) row.getField(getWeightCol()), + (Double) row.getField(getLabelCol()), + (double[]) row.getField(getVectorCol()))) + .returns( + new TupleTypeInfo<>( + BasicTypeInfo.DOUBLE_TYPE_INFO, + BasicTypeInfo.DOUBLE_TYPE_INFO, + PrimitiveArrayTypeInfo.DOUBLE_PRIMITIVE_ARRAY_TYPE_INFO)); + + DataStream<Double> labelValues = DataStreamUtils.distinct(trainData.map(x -> x.f1)); + + final String broadcastLabelsName = "broadcastLabels"; + trainData = + BroadcastUtils.withBroadcastStream( + Collections.singletonList(trainData), + Collections.singletonMap(broadcastLabelsName, labelValues), + inputList -> { + DataStream data = inputList.get(0); + return data.transform( + "preprocess", + new TupleTypeInfo<>( + BasicTypeInfo.DOUBLE_TYPE_INFO, + BasicTypeInfo.DOUBLE_TYPE_INFO, + TypeInformation.of(double[].class)), + new PreprocessDataOp( + new PreprocessOneRecord(broadcastLabelsName))); + }); + + DataStream<double[]> initModel = + trainData + .transform( + "getInitModel", + TypeInformation.of(double[].class), + new GetInitModel()) + .returns(TypeInformation.of(double[].class)); + + DataStream<Tuple2<double[], double[]>> modelAndLossCurve = train(trainData, initModel); + + DataStream<LogisticRegressionModelData> modelData = + modelAndLossCurve + .connect(labelValues) + .transform( + "composeModelData", + TypeInformation.of(LogisticRegressionModelData.class), + new ComposeModelDataOp()) + .setParallelism(1); + + LogisticRegressionModel model = + new LogisticRegressionModel(new HashMap<>()) + .setModelData(tEnv.fromDataStream(modelData)); + ReadWriteUtils.updateExistingParams(model, paramMap); + return model; + } + + /** Pre-processes the training data. */ + private static class PreprocessDataOp + extends AbstractUdfStreamOperator< + Tuple3<Double, Double, double[]>, + RichMapFunction< + Tuple3<Double, Double, double[]>, Tuple3<Double, Double, double[]>>> + implements OneInputStreamOperator< + Tuple3<Double, Double, double[]>, Tuple3<Double, Double, double[]>> { + public PreprocessDataOp( + RichMapFunction<Tuple3<Double, Double, double[]>, Tuple3<Double, Double, double[]>> + userFunction) { + super(userFunction); + } + + @Override + public void processElement(StreamRecord<Tuple3<Double, Double, double[]>> streamRecord) + throws Exception { + streamRecord.replace(userFunction.map(streamRecord.getValue())); + output.collect(streamRecord); + } + } + + /** Pre-processes the training data. */ + private static class PreprocessOneRecord + extends RichMapFunction< + Tuple3<Double, Double, double[]>, Tuple3<Double, Double, double[]>> { + + String broadcastName; + double[] labelValues; + + public PreprocessOneRecord(String broadcastName) { + this.broadcastName = broadcastName; + } + + @Override + public Tuple3<Double, Double, double[]> map(Tuple3<Double, Double, double[]> value) { + if (labelValues == null) { + List<Double> labelList = getRuntimeContext().getBroadcastVariable(broadcastName); + labelValues = labelList.stream().mapToDouble(Double::doubleValue).toArray(); + } + value.f1 = getLabel(labelValues, value.f1); + return value; + } + + private double getLabel(double[] labels, double label) { + if (Math.abs(label - labels[0]) < 1e-7) { + return 1.; + } else { + return -1.; + } + } + } + + /** + * Gets initialized model, note that the parallelism is same as the input train data, not one. + */ + private static class GetInitModel extends AbstractStreamOperator<double[]> + implements OneInputStreamOperator<Tuple3<Double, Double, double[]>, double[]>, + BoundedOneInput { + + int dim = 0; + + @Override + public void endInput() { + output.collect(new StreamRecord<>(new double[dim])); + } + + @Override + public void processElement(StreamRecord<Tuple3<Double, Double, double[]>> streamRecord) { + dim = Math.max(dim, streamRecord.getValue().f2.length); + } + } + + /** + * Trains a machine learning model on the input data and initialized model, return the trained + * model and loss curve. + * + * @param trainData the training data + * @param initModel the init model + * @return the trained model and loss during the training + */ + private DataStream<Tuple2<double[], double[]>> train( + DataStream<Tuple3<Double, Double, double[]>> trainData, + DataStream<double[]> initModel) { + LogisticGradient logisticGradient = new LogisticGradient(getL1(), getL2()); + + DataStreamList resultList = + Iterations.iterateBoundedStreamsUntilTermination( + DataStreamList.of(initModel), + ReplayableDataStreamList.notReplay(trainData), + IterationConfig.newBuilder() + .setOperatorLifeCycle(OperatorLifeCycle.ALL_ROUND) + .build(), + new TrainIterationBody( + logisticGradient, + getBatchSize(), + getLearningRate(), + getMaxIter(), + getEpsilon())); + + return resultList.get(0); + } + + /** The iteration implementation for training process. */ + private static class TrainIterationBody implements IterationBody { + + private final LogisticGradient logisticGradient; + + private final int batchSize; + + private final double learningRate; + + private final int maxIter; + + private final double epsilon; + + public TrainIterationBody( + LogisticGradient logisticGradient, + int batchSize, + double learningRate, + int maxIter, + double epsilon) { + this.logisticGradient = logisticGradient; + this.batchSize = batchSize; + this.learningRate = learningRate; + this.maxIter = maxIter; + this.epsilon = epsilon; + } + + @Override + public IterationBodyResult process( + DataStreamList variableStreams, DataStreamList dataStreams) { + DataStream<double[]> initModelOrGradientsAndWeightAndLoss = variableStreams.get(0); + DataStream<Tuple3<Double, Double, double[]>> trainData = dataStreams.get(0); + SingleOutputStreamOperator<double[]> gradientAndWeightAndLoss = + trainData + .connect(initModelOrGradientsAndWeightAndLoss) + .transform( + "updateModelAndComputeGradients", + TypeInformation.of(double[].class), + new CacheDataAndUpdateModelAndComputeGradient( + logisticGradient, + batchSize, + learningRate, + maxIter, + epsilon)); + + DataStreamList outputs = + IterationBody.forEachRound( + DataStreamList.of(gradientAndWeightAndLoss), + input -> { + DataStream<double[]> feedback = + AllReduceUtils.allReduce(input.get(0)); + return DataStreamList.of(feedback); + }); + DataStream<Integer> terminatation = + outputs.get(0) + .map( + x -> { + double[] value = (double[]) x; + return value[value.length - 1] / value[value.length - 2]; + }) + .flatMap(new TerminationCriteria(maxIter, epsilon)); + + return new IterationBodyResult( + DataStreamList.of(outputs.get(0)), + DataStreamList.of(gradientAndWeightAndLoss.getSideOutput(MODEL_OUTPUT)), + terminatation); + } + } + + /** Terminates if epochId is greater than {maxIter} or loss smaller than {epsilon}. */ + private static class TerminationCriteria + implements IterationListener<Integer>, FlatMapFunction<Double, Integer> { + + private final int maxIter; + private final double epsilon; + double loss = 0; + + public TerminationCriteria(int maxIter, double epsilon) { + this.maxIter = maxIter; + this.epsilon = epsilon; + } + + @Override + public void flatMap(Double value, Collector<Integer> out) { + this.loss = value; + } + + @Override + public void onEpochWatermarkIncremented( + int epochWatermark, Context context, Collector<Integer> collector) { + if ((epochWatermark + 1) < maxIter && this.loss > epsilon) { + collector.collect(0); + } + } + + @Override + public void onIterationTerminated(Context context, Collector<Integer> collector) {} + } + + /** First input is data, second input is initModelOrGradientAndweightAndLoss. */ + private static class CacheDataAndUpdateModelAndComputeGradient + extends AbstractStreamOperator<double[]> + implements TwoInputStreamOperator<Tuple3<Double, Double, double[]>, double[], double[]>, + IterationListener<double[]> { + + /** training specific parameters. */ + double[] model; Review comment: Thanks for the comments. I think we should NOT use modelState directly. Machine learning models are updated frequently during iterations and updating it to `states` in each iteration may incur expensive (de)serialization cost. -- 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]
