lindong28 commented on a change in pull request #70: URL: https://github.com/apache/flink-ml/pull/70#discussion_r830430626
########## File path: flink-ml-lib/src/main/java/org/apache/flink/ml/clustering/kmeans/StreamingKMeans.java ########## @@ -0,0 +1,404 @@ +/* + * 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.clustering.kmeans; + +import org.apache.flink.api.common.functions.AggregateFunction; +import org.apache.flink.api.common.functions.MapFunction; +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.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; +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.Iterations; +import org.apache.flink.ml.api.Estimator; +import org.apache.flink.ml.common.distance.DistanceMeasure; +import org.apache.flink.ml.common.param.HasBatchStrategy; +import org.apache.flink.ml.linalg.BLAS; +import org.apache.flink.ml.linalg.DenseVector; +import org.apache.flink.ml.linalg.Vectors; +import org.apache.flink.ml.linalg.typeinfo.DenseVectorTypeInfo; +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.environment.StreamExecutionEnvironment; +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.bridge.java.internal.StreamTableEnvironmentImpl; +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 org.apache.commons.lang3.ArrayUtils; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; + +/** + * StreamingKMeans extends the function of {@link KMeans}, supporting to train a K-Means model + * continuously according to an unbounded stream of train data. + */ +public class StreamingKMeans + implements Estimator<StreamingKMeans, StreamingKMeansModel>, + StreamingKMeansParams<StreamingKMeans> { + private final Map<Param<?>, Object> paramMap = new HashMap<>(); + private Table initModelDataTable; + + public StreamingKMeans() { + ParamUtils.initializeMapWithDefaultValues(paramMap, this); + } + + public StreamingKMeans(Table... initModelDataTables) { + Preconditions.checkArgument(initModelDataTables.length == 1); + this.initModelDataTable = initModelDataTables[0]; + ParamUtils.initializeMapWithDefaultValues(paramMap, this); + setInitMode("direct"); + } + + @Override + public StreamingKMeansModel fit(Table... inputs) { + Preconditions.checkArgument(inputs.length == 1); + Preconditions.checkArgument(HasBatchStrategy.COUNT_STRATEGY.equals(getBatchStrategy())); + + StreamTableEnvironment tEnv = + (StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment(); + StreamExecutionEnvironment env = ((StreamTableEnvironmentImpl) tEnv).execEnv(); + + DataStream<DenseVector> points = + tEnv.toDataStream(inputs[0]).map(new FeaturesExtractor(getFeaturesCol())); + points.getTransformation().setParallelism(1); + + DataStream<KMeansModelData> initModelDataStream; + if (getInitMode().equals("random")) { + initModelDataStream = createRandomCentroids(env, getDims(), getK(), getSeed()); + } else { + initModelDataStream = KMeansModelData.getModelDataStream(initModelDataTable); + } + DataStream<Tuple2<KMeansModelData, DenseVector>> initModelDataWithWeightsStream = + initModelDataStream.map(new InitWeightAssigner(getInitWeights())); + initModelDataWithWeightsStream.getTransformation().setParallelism(1); + + IterationBody body = + new StreamingKMeansIterationBody( + DistanceMeasure.getInstance(getDistanceMeasure()), + getDecayFactor(), + getBatchSize(), + getK()); + + DataStream<KMeansModelData> finalModelDataStream = + Iterations.iterateUnboundedStreams( + DataStreamList.of(initModelDataWithWeightsStream), + DataStreamList.of(points), + body) + .get(0); + finalModelDataStream = finalModelDataStream.union(initModelDataStream); + + Table finalModelDataTable = tEnv.fromDataStream(finalModelDataStream); + StreamingKMeansModel model = new StreamingKMeansModel().setModelData(finalModelDataTable); + ReadWriteUtils.updateExistingParams(model, paramMap); + return model; + } + + private static class InitWeightAssigner + implements MapFunction<KMeansModelData, Tuple2<KMeansModelData, DenseVector>> { + private final double[] initWeights; + + private InitWeightAssigner(Double[] initWeights) { + this.initWeights = ArrayUtils.toPrimitive(initWeights); + } + + @Override + public Tuple2<KMeansModelData, DenseVector> map(KMeansModelData modelData) + throws Exception { + return Tuple2.of(modelData, Vectors.dense(initWeights)); + } + } + + @Override + public void save(String path) throws IOException { + if (initModelDataTable != null) { + ReadWriteUtils.saveModelData( + KMeansModelData.getModelDataStream(initModelDataTable), + path, + new KMeansModelData.ModelDataEncoder()); + } + + ReadWriteUtils.saveMetadata(this, path); + } + + public static StreamingKMeans load(StreamExecutionEnvironment env, String path) + throws IOException { + StreamingKMeans kMeans = ReadWriteUtils.loadStageParam(path); + + Path initModelDataPath = Paths.get(path, "data"); + if (Files.exists(initModelDataPath)) { + StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); + + DataStream<KMeansModelData> initModelDataStream = + ReadWriteUtils.loadModelData(env, path, new KMeansModelData.ModelDataDecoder()); + + kMeans.initModelDataTable = tEnv.fromDataStream(initModelDataStream); + kMeans.setInitMode("direct"); + } + + return kMeans; + } + + @Override + public Map<Param<?>, Object> getParamMap() { + return paramMap; + } + + private static class StreamingKMeansIterationBody implements IterationBody { + private final DistanceMeasure distanceMeasure; + private final double decayFactor; + private final int batchSize; + private final int k; + + public StreamingKMeansIterationBody( + DistanceMeasure distanceMeasure, double decayFactor, int batchSize, int k) { + this.distanceMeasure = distanceMeasure; + this.decayFactor = decayFactor; + this.batchSize = batchSize; + this.k = k; + } + + @Override + public IterationBodyResult process( + DataStreamList variableStreams, DataStreamList dataStreams) { + DataStream<Tuple2<KMeansModelData, DenseVector>> modelDataWithWeights = + variableStreams.get(0); + DataStream<DenseVector> points = dataStreams.get(0); + + DataStream<Tuple2<KMeansModelData, DenseVector>> newModelDataWithWeights = + points.countWindowAll(batchSize) + .aggregate(new MiniBatchCreator()) + .connect(modelDataWithWeights.broadcast()) + .transform( + "UpdateModelData", + new TupleTypeInfo<>( + TypeInformation.of(KMeansModelData.class), + DenseVectorTypeInfo.INSTANCE), + new UpdateModelDataOperator(distanceMeasure, decayFactor, k)) + .setParallelism(1); + + DataStream<KMeansModelData> newModelData = + newModelDataWithWeights.map( + (MapFunction<Tuple2<KMeansModelData, DenseVector>, KMeansModelData>) + x -> x.f0); + + return new IterationBodyResult( + DataStreamList.of(newModelDataWithWeights), DataStreamList.of(newModelData)); + } + } + + // TODO: change this single-threaded implementation to support training in a distributed way, + // after model data + // version mechanism is implemented. + private static class UpdateModelDataOperator + extends AbstractStreamOperator<Tuple2<KMeansModelData, DenseVector>> + implements TwoInputStreamOperator< + DenseVector[], + Tuple2<KMeansModelData, DenseVector>, + Tuple2<KMeansModelData, DenseVector>> { + private final DistanceMeasure distanceMeasure; + private final double decayFactor; + private final int k; + private ListState<DenseVector[]> miniBatchState; + private ListState<KMeansModelData> modelDataState; + private ListState<DenseVector> weightsState; + + public UpdateModelDataOperator(DistanceMeasure distanceMeasure, double decayFactor, int k) { + this.distanceMeasure = distanceMeasure; + this.decayFactor = decayFactor; + this.k = k; + } + + @Override + public void initializeState(StateInitializationContext context) throws Exception { + super.initializeState(context); + + TypeInformation<DenseVector[]> type = + ObjectArrayTypeInfo.getInfoFor(DenseVectorTypeInfo.INSTANCE); + miniBatchState = + context.getOperatorStateStore() + .getListState(new ListStateDescriptor<>("miniBatch", type)); + + modelDataState = + context.getOperatorStateStore() + .getListState( + new ListStateDescriptor<>("modelData", KMeansModelData.class)); + + weightsState = + context.getOperatorStateStore() + .getListState( + new ListStateDescriptor<>( + "weights", DenseVectorTypeInfo.INSTANCE)); + } + + @Override + public void processElement1(StreamRecord<DenseVector[]> streamRecord) throws Exception { + miniBatchState.add(streamRecord.getValue()); + processElement(); + } + + @Override + public void processElement2(StreamRecord<Tuple2<KMeansModelData, DenseVector>> streamRecord) + throws Exception { + modelDataState.add(streamRecord.getValue().f0); + weightsState.add(streamRecord.getValue().f1); + processElement(); + } + + private void processElement() throws Exception { + if (!modelDataState.get().iterator().hasNext() + || !miniBatchState.get().iterator().hasNext()) { + return; + } + + // Retrieves data from states. + List<KMeansModelData> modelDataList = + IteratorUtils.toList(modelDataState.get().iterator()); + if (modelDataList.size() != 1) { + throw new RuntimeException( + "The operator received " + + modelDataList.size() + + " list of model data in this round"); + } + DenseVector[] centroids = modelDataList.get(0).centroids; + modelDataState.clear(); + + List<DenseVector> weightsList = IteratorUtils.toList(weightsState.get().iterator()); + if (weightsList.size() != 1) { + throw new RuntimeException( + "The operator received " + + weightsList.size() + + " list of weights in this round"); + } + DenseVector weights = weightsList.get(0); + weightsState.clear(); + + List<DenseVector[]> pointsList = IteratorUtils.toList(miniBatchState.get().iterator()); + DenseVector[] points = pointsList.get(0); + pointsList.remove(0); + miniBatchState.clear(); + miniBatchState.addAll(pointsList); Review comment: I am not sure that the model data and input data would be inputted into this operator in lockstep, when we take into consideration the physical transmission latency of those data. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org