zhipeng93 commented on code in PR #148: URL: https://github.com/apache/flink-ml/pull/148#discussion_r958253423
########## flink-ml-lib/src/main/java/org/apache/flink/ml/clustering/agglomerativeclustering/AgglomerativeClustering.java: ########## @@ -0,0 +1,415 @@ +/* + * 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.agglomerativeclustering; + +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.tuple.Tuple4; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.ml.api.AlgoOperator; +import org.apache.flink.ml.common.datastream.TableUtils; +import org.apache.flink.ml.common.distance.DistanceMeasure; +import org.apache.flink.ml.common.distance.EuclideanDistanceMeasure; +import org.apache.flink.ml.linalg.BLAS; +import org.apache.flink.ml.linalg.DenseVector; +import org.apache.flink.ml.linalg.Vector; +import org.apache.flink.ml.linalg.VectorWithNorm; +import org.apache.flink.ml.linalg.Vectors; +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.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.BoundedOneInput; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +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.OutputTag; +import org.apache.flink.util.Preconditions; + +import org.apache.commons.collections.IteratorUtils; +import org.apache.commons.lang3.ArrayUtils; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * An AlgoOperator that performs a hierarchical clustering using a bottom up approach. Each + * observation starts in its own cluster and the clusters are merged together one by one. Users can + * choose different strategies to merge two clusters by setting {@link + * AgglomerativeClusteringParams#LINKAGE} and different distance measure by setting {@link + * AgglomerativeClusteringParams#DISTANCE_MEASURE}. + * + * <p>See https://en.wikipedia.org/wiki/Hierarchical_clustering. + */ +public class AgglomerativeClustering + implements AlgoOperator<AgglomerativeClustering>, + AgglomerativeClusteringParams<AgglomerativeClustering> { + private final Map<Param<?>, Object> paramMap = new HashMap<>(); + + public AgglomerativeClustering() { + ParamUtils.initializeMapWithDefaultValues(paramMap, this); + } + + @Override + public Table[] transform(Table... inputs) { + Preconditions.checkArgument(inputs.length == 1); + Integer k = getK(); + Double distanceThreshold = getDistanceThreshold(); + Preconditions.checkArgument( + (k == null && distanceThreshold != null) + || (k != null && distanceThreshold == null), + "One of param k and distanceThreshold should be null."); + + if (getLinkage().equals(LINKAGE_WARD)) { + String distanceMeasure = getDistanceMeasure(); + Preconditions.checkArgument( + distanceMeasure.equals(EuclideanDistanceMeasure.NAME), + distanceMeasure + + " was provided as distance measure while linkage was ward. Ward only works with euclidean."); + } + + StreamTableEnvironment tEnv = + (StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment(); + DataStream<Row> dataStream = tEnv.toDataStream(inputs[0]); + + RowTypeInfo inputTypeInfo = TableUtils.getRowTypeInfo(inputs[0].getResolvedSchema()); + RowTypeInfo outputTypeInfo = + new RowTypeInfo( + ArrayUtils.addAll(inputTypeInfo.getFieldTypes(), Types.INT), + ArrayUtils.addAll(inputTypeInfo.getFieldNames(), getPredictionCol())); + + OutputTag<Tuple4<Integer, Integer, Double, Integer>> mergeInfoOutputTag = + new OutputTag<Tuple4<Integer, Integer, Double, Integer>>("MERGE_INFO") {}; + + SingleOutputStreamOperator<Row> output = + dataStream.transform( + "doLocalAgglomerativeClustering", + outputTypeInfo, + new LocalAgglomerativeClusteringOperator( + getFeaturesCol(), + getLinkage(), + getDistanceMeasure(), + getK(), + getDistanceThreshold(), + getComputeFullTree(), + mergeInfoOutputTag)); + output.getTransformation().setParallelism(1); + + Table outputTable = tEnv.fromDataStream(output); + + DataStream<Tuple4<Integer, Integer, Double, Integer>> mergeInfo = + output.getSideOutput(mergeInfoOutputTag); + Table mergeInfoTable = + tEnv.fromDataStream(mergeInfo) + .as("clusterId1", "clusterId2", "distance", "sizeOfMergedCluster"); + + return new Table[] {outputTable, mergeInfoTable}; Review Comment: Thanks for the comment. I have added the following description in Java doc. ``` The output contains two tables. The first one assigns one cluster Id for each data point. The second one contains the information of merging two clusters at each step. The data format of the merging information is (clusterId1, clusterId2, distance, sizeOfMergedCluster). ``` I am not sure about adding a `Table getMergeInfo()` here, since it is somehow adding a public API. For the semantic of `AgglomerativeClustering`, I made it an AlgoOperator because (1) it is not a Estimator since it has not model data and cannot be used to predict some new data. (2) It is also not a Transformer. It is clear that not all AlgoOperator/Estimator could be used in Pipeline. For those multi-input/output stages, we have `Graph`. -- 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]
