zhipeng93 commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r825878727



##########
File path: 
flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.feature.minmaxscaler;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.RichMapPartitionFunction;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.common.datastream.DataStreamUtils;
+import org.apache.flink.ml.linalg.DenseVector;
+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.environment.StreamExecutionEnvironment;
+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.Collector;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+/**
+ * An Estimator which implements the MinMaxScaler algorithm.
+ *
+ * <p>See 
https://en.wikipedia.org/wiki/Feature_scaling#Rescaling_(min-max_normalization).
+ */
+public class MinMaxScaler
+        implements Estimator<MinMaxScaler, MinMaxScalerModel>, 
MinMaxScalerParams<MinMaxScaler> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+
+    public MinMaxScaler() {
+        ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+    }
+
+    @Override
+    public MinMaxScalerModel fit(Table... inputs) {
+        Preconditions.checkArgument(inputs.length == 1);
+        final String featureCol = getFeaturesCol();
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) 
inputs[0]).getTableEnvironment();
+        DataStream<DenseVector> vectors =

Review comment:
       How about renaming `vectors` to `features`?

##########
File path: 
flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.feature.minmaxscaler;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.RichMapPartitionFunction;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.common.datastream.DataStreamUtils;
+import org.apache.flink.ml.linalg.DenseVector;
+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.environment.StreamExecutionEnvironment;
+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.Collector;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+/**
+ * An Estimator which implements the MinMaxScaler algorithm.
+ *
+ * <p>See 
https://en.wikipedia.org/wiki/Feature_scaling#Rescaling_(min-max_normalization).
+ */
+public class MinMaxScaler
+        implements Estimator<MinMaxScaler, MinMaxScalerModel>, 
MinMaxScalerParams<MinMaxScaler> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+
+    public MinMaxScaler() {
+        ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+    }
+
+    @Override
+    public MinMaxScalerModel fit(Table... inputs) {
+        Preconditions.checkArgument(inputs.length == 1);
+        final String featureCol = getFeaturesCol();
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) 
inputs[0]).getTableEnvironment();
+        DataStream<DenseVector> vectors =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                (MapFunction<Row, DenseVector>)
+                                        value -> (DenseVector) 
value.getField(featureCol));
+        DataStream<DenseVector> minMaxVectors =
+                vectors.transform(
+                                "reduceInEachPartition",
+                                vectors.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .transform(
+                                "reduceInFinalPartition",
+                                vectors.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .setParallelism(1);
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(
+                        minMaxVectors,
+                        new RichMapPartitionFunction<DenseVector, 
MinMaxScalerModelData>() {
+                            @Override
+                            public void mapPartition(
+                                    Iterable<DenseVector> values,
+                                    Collector<MinMaxScalerModelData> out) {
+                                Iterator<DenseVector> iter = values.iterator();
+                                DenseVector minDenseVector = iter.next();
+                                DenseVector maxDenseVector = iter.next();
+                                out.collect(
+                                        new 
MinMaxScalerModelData(minDenseVector, maxDenseVector));
+                            }
+                        });
+
+        MinMaxScalerModel model =
+                new 
MinMaxScalerModel().setModelData(tEnv.fromDataStream(modelData));
+        ReadWriteUtils.updateExistingParams(model, getParamMap());
+        return model;
+    }
+
+    /**
+     * A stream operator to compute the min max vectors in each partition of 
the input bounded data
+     * stream.
+     */
+    private static class MinMaxReduceFunctionOperator extends 
AbstractStreamOperator<DenseVector>
+            implements OneInputStreamOperator<DenseVector, DenseVector>, 
BoundedOneInput {
+        private ListState<DenseVector> minDenseVectorState;
+        private ListState<DenseVector> maxDenseVectorState;
+
+        private DenseVector minDenseVector;
+        private DenseVector maxDenseVector;
+
+        @Override
+        public void endInput() {
+            output.collect(new StreamRecord<>(minDenseVector));
+            output.collect(new StreamRecord<>(maxDenseVector));
+        }
+
+        @Override
+        public void processElement(StreamRecord<DenseVector> streamRecord) {
+            DenseVector currentValue = streamRecord.getValue();
+            if (minDenseVector == null) {
+                int vecSize = currentValue.size();
+                minDenseVector = new DenseVector(vecSize);
+                maxDenseVector = new DenseVector(vecSize);
+                System.arraycopy(currentValue.values, 0, 
minDenseVector.values, 0, vecSize);
+                System.arraycopy(currentValue.values, 0, 
maxDenseVector.values, 0, vecSize);
+
+            } else {
+                for (int i = 0; i < currentValue.size(); ++i) {
+                    minDenseVector.values[i] =
+                            Math.min(minDenseVector.values[i], 
currentValue.values[i]);
+                    maxDenseVector.values[i] =
+                            Math.max(maxDenseVector.values[i], 
currentValue.values[i]);
+                }
+            }
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")
+        public void initializeState(StateInitializationContext context) throws 
Exception {
+            super.initializeState(context);
+            minDenseVectorState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "minDenseVectorState",
+                                            getOperatorConfig()
+                                                    .getTypeSerializerIn(
+                                                            0, 
getClass().getClassLoader())));
+            maxDenseVectorState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "maxDenseVectorState",
+                                            getOperatorConfig()
+                                                    .getTypeSerializerIn(
+                                                            0, 
getClass().getClassLoader())));
+            Iterator<DenseVector> minIterator = 
minDenseVectorState.get().iterator();
+            Iterator<DenseVector> maxIterator = 
maxDenseVectorState.get().iterator();
+            if (minIterator.hasNext()) {

Review comment:
       How about check two of the states? Or simply use a utility function 
`OperatorStateUtils.getUniqueElement()`?

##########
File path: 
flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.feature.minmaxscaler;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.RichMapPartitionFunction;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.common.datastream.DataStreamUtils;
+import org.apache.flink.ml.linalg.DenseVector;
+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.environment.StreamExecutionEnvironment;
+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.Collector;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+/**
+ * An Estimator which implements the MinMaxScaler algorithm.
+ *
+ * <p>See 
https://en.wikipedia.org/wiki/Feature_scaling#Rescaling_(min-max_normalization).
+ */
+public class MinMaxScaler
+        implements Estimator<MinMaxScaler, MinMaxScalerModel>, 
MinMaxScalerParams<MinMaxScaler> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+
+    public MinMaxScaler() {
+        ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+    }
+
+    @Override
+    public MinMaxScalerModel fit(Table... inputs) {
+        Preconditions.checkArgument(inputs.length == 1);
+        final String featureCol = getFeaturesCol();
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) 
inputs[0]).getTableEnvironment();
+        DataStream<DenseVector> vectors =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                (MapFunction<Row, DenseVector>)
+                                        value -> (DenseVector) 
value.getField(featureCol));
+        DataStream<DenseVector> minMaxVectors =
+                vectors.transform(
+                                "reduceInEachPartition",
+                                vectors.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .transform(
+                                "reduceInFinalPartition",
+                                vectors.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .setParallelism(1);
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(
+                        minMaxVectors,
+                        new RichMapPartitionFunction<DenseVector, 
MinMaxScalerModelData>() {
+                            @Override
+                            public void mapPartition(
+                                    Iterable<DenseVector> values,
+                                    Collector<MinMaxScalerModelData> out) {
+                                Iterator<DenseVector> iter = values.iterator();
+                                DenseVector minDenseVector = iter.next();
+                                DenseVector maxDenseVector = iter.next();
+                                out.collect(
+                                        new 
MinMaxScalerModelData(minDenseVector, maxDenseVector));
+                            }
+                        });
+
+        MinMaxScalerModel model =
+                new 
MinMaxScalerModel().setModelData(tEnv.fromDataStream(modelData));
+        ReadWriteUtils.updateExistingParams(model, getParamMap());
+        return model;
+    }
+
+    /**
+     * A stream operator to compute the min max vectors in each partition of 
the input bounded data
+     * stream.
+     */
+    private static class MinMaxReduceFunctionOperator extends 
AbstractStreamOperator<DenseVector>
+            implements OneInputStreamOperator<DenseVector, DenseVector>, 
BoundedOneInput {
+        private ListState<DenseVector> minDenseVectorState;
+        private ListState<DenseVector> maxDenseVectorState;
+
+        private DenseVector minDenseVector;

Review comment:
       How about initialize these two vectors in the construction method? 
Otherwise if there is no record in a partition, it throws a NPE.

##########
File path: 
flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.feature.minmaxscaler;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.RichMapPartitionFunction;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.common.datastream.DataStreamUtils;
+import org.apache.flink.ml.linalg.DenseVector;
+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.environment.StreamExecutionEnvironment;
+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.Collector;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+/**
+ * An Estimator which implements the MinMaxScaler algorithm.
+ *
+ * <p>See 
https://en.wikipedia.org/wiki/Feature_scaling#Rescaling_(min-max_normalization).
+ */
+public class MinMaxScaler
+        implements Estimator<MinMaxScaler, MinMaxScalerModel>, 
MinMaxScalerParams<MinMaxScaler> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+
+    public MinMaxScaler() {
+        ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+    }
+
+    @Override
+    public MinMaxScalerModel fit(Table... inputs) {
+        Preconditions.checkArgument(inputs.length == 1);
+        final String featureCol = getFeaturesCol();
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) 
inputs[0]).getTableEnvironment();
+        DataStream<DenseVector> vectors =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                (MapFunction<Row, DenseVector>)
+                                        value -> (DenseVector) 
value.getField(featureCol));
+        DataStream<DenseVector> minMaxVectors =

Review comment:
       How about renaming `minMaxVectors` to `minMaxValue`?

##########
File path: 
flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.feature;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.ml.feature.minmaxscaler.MinMaxScaler;
+import org.apache.flink.ml.feature.minmaxscaler.MinMaxScalerModel;
+import org.apache.flink.ml.feature.minmaxscaler.MinMaxScalerModelData;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.linalg.Vectors;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.ml.util.StageTestUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import 
org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Schema;
+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.commons.collections.IteratorUtils;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+/** Tests {@link MinMaxScaler} and {@link MinMaxScalerModel}. */
+public class MinMaxScalerTest {
+    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+    private StreamExecutionEnvironment env;
+    private StreamTableEnvironment tEnv;
+    private Table trainData;
+    private Table predictData;
+    private static final List<Row> trainRows =
+            new ArrayList<>(
+                    Arrays.asList(
+                            Row.of(Vectors.dense(0.0, 3.0)),
+                            Row.of(Vectors.dense(2.1, 0.0)),
+                            Row.of(Vectors.dense(4.1, 5.1)),
+                            Row.of(Vectors.dense(6.1, 8.1)),
+                            Row.of(Vectors.dense(200, 300))));
+    private static final List<Row> predictRows =
+            new 
ArrayList<>(Collections.singletonList(Row.of(Vectors.dense(150.0, 90.0))));
+
+    @Before
+    public void before() {
+        Configuration config = new Configuration();
+        
config.set(ExecutionCheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, 
true);
+        env = StreamExecutionEnvironment.getExecutionEnvironment(config);
+        env.setParallelism(4);
+        env.enableCheckpointing(100);
+        env.setRestartStrategy(RestartStrategies.noRestart());
+        tEnv = StreamTableEnvironment.create(env);
+        Schema schema = Schema.newBuilder().column("f0", 
DataTypes.of(DenseVector.class)).build();
+        DataStream<Row> dataStream = env.fromCollection(trainRows);
+        trainData = tEnv.fromDataStream(dataStream, schema).as("features");
+        DataStream<Row> predDataStream = env.fromCollection(predictRows);
+        predictData = tEnv.fromDataStream(predDataStream, 
schema).as("features");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol) 
throws Exception {
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) 
output).getTableEnvironment();
+        DataStream<DenseVector> stream =
+                tEnv.toDataStream(output)
+                        .map(
+                                (MapFunction<Row, DenseVector>)
+                                        row -> (DenseVector) 
row.getField(outputCol));
+        List<DenseVector> result = 
IteratorUtils.toList(stream.executeAndCollect());
+        assertEquals(Vectors.dense(0.75, 0.3), result.get(0));
+    }
+
+    @Test
+    public void testParam() {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        assertEquals("features", minMaxScaler.getFeaturesCol());
+        assertEquals(1.0, minMaxScaler.getMax(), 0.0001);
+        assertEquals(0.0, minMaxScaler.getMin(), 0.0001);
+        assertEquals("prediction", minMaxScaler.getPredictionCol());
+        minMaxScaler
+                .setFeaturesCol("test_features")
+                .setMax(4.0)
+                .setMin(1.0)
+                .setPredictionCol("test_output");
+        assertEquals("test_features", minMaxScaler.getFeaturesCol());
+        assertEquals(1.0, minMaxScaler.getMin(), 0.0001);
+        assertEquals(4.0, minMaxScaler.getMax(), 0.0001);
+        assertEquals("test_output", minMaxScaler.getPredictionCol());
+    }
+
+    @Test
+    public void testFeaturePredictionParam() {
+        MinMaxScaler minMaxScaler =
+                new MinMaxScaler()
+                        .setMin(1.0)
+                        .setMax(4.0)
+                        .setFeaturesCol("test_features")
+                        .setPredictionCol("test_output");
+        MinMaxScalerModel model = 
minMaxScaler.fit(trainData.as("test_features"));
+        Table output = model.transform(predictData.as("test_features"))[0];
+        assertEquals(
+                Arrays.asList("test_features", "test_output"),
+                output.getResolvedSchema().getColumnNames());
+    }
+
+    @Test
+    public void testFewerDistinctPointsThanCluster() {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScalerModel model = minMaxScaler.fit(predictData);
+        model.transform(predictData);
+    }
+
+    @Test
+    public void testFitAndPredict() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScalerModel minMaxScalerModel = minMaxScaler.fit(trainData);
+        Table output = minMaxScalerModel.transform(predictData)[0];
+        verifyPredictionResult(output, minMaxScaler.getPredictionCol());
+    }
+
+    @Test
+    public void testSaveLoadAndPredict() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScaler loadedMinMaxScaler =
+                StageTestUtils.saveAndReload(
+                        env, minMaxScaler, 
tempFolder.newFolder().getAbsolutePath());
+        MinMaxScalerModel minMaxScalerModel = 
loadedMinMaxScaler.fit(trainData);
+        minMaxScalerModel =
+                StageTestUtils.saveAndReload(
+                        env, minMaxScalerModel, 
tempFolder.newFolder().getAbsolutePath());
+        assertEquals(
+                Arrays.asList("minVector", "maxVector"),
+                
minMaxScalerModel.getModelData()[0].getResolvedSchema().getColumnNames());
+        Table output = minMaxScalerModel.transform(predictData)[0];
+        verifyPredictionResult(output, minMaxScaler.getPredictionCol());
+    }
+
+    @Test
+    public void testModelSaveLoadAndPredict() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScalerModel minMaxScalerModel = minMaxScaler.fit(trainData);
+        MinMaxScalerModel newModel =
+                StageTestUtils.saveAndReload(
+                        env, minMaxScalerModel, 
tempFolder.newFolder().getAbsolutePath());
+        Table output = newModel.transform(predictData)[0];
+        verifyPredictionResult(output, minMaxScaler.getPredictionCol());
+    }
+
+    @Test
+    public void testGetModelData() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScalerModel minMaxScalerModel = minMaxScaler.fit(trainData);
+        Table modelData = minMaxScalerModel.getModelData()[0];
+        DataStream<Row> output = tEnv.toDataStream(modelData);
+        assertEquals("minVector", 
modelData.getResolvedSchema().getColumnNames().get(0));
+        assertEquals("maxVector", 
modelData.getResolvedSchema().getColumnNames().get(1));
+        List<Row> modelRows = IteratorUtils.toList(output.executeAndCollect());
+        MinMaxScalerModelData data =
+                new MinMaxScalerModelData(
+                        (DenseVector) modelRows.get(0).getField(0),
+                        (DenseVector) modelRows.get(0).getField(1));
+        Assert.assertNotNull(data);
+        assertEquals(data.maxVector.size(), data.maxVector.size());

Review comment:
       Could you please also check the value of model data?

##########
File path: 
flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.feature.minmaxscaler;
+
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.ml.api.Model;
+import org.apache.flink.ml.common.broadcast.BroadcastUtils;
+import org.apache.flink.ml.common.datastream.TableUtils;
+import org.apache.flink.ml.linalg.DenseVector;
+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.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+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.table.runtime.typeutils.ExternalTypeInfo;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.commons.lang3.ArrayUtils;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * A Model which do a minMax scaler operation using the model data computed by 
{@link MinMaxScaler}.
+ */
+public class MinMaxScalerModel
+        implements Model<MinMaxScalerModel>, 
MinMaxScalerParams<MinMaxScalerModel> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+    private Table modelDataTable;
+
+    public MinMaxScalerModel() {
+        ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+    }
+
+    @Override
+    public MinMaxScalerModel setModelData(Table... inputs) {
+        modelDataTable = inputs[0];
+        return this;
+    }
+
+    @Override
+    public Table[] getModelData() {
+        return new Table[] {modelDataTable};
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public Table[] transform(Table... inputs) {
+        Preconditions.checkArgument(inputs.length == 1);
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) 
inputs[0]).getTableEnvironment();
+        DataStream<Row> data = tEnv.toDataStream(inputs[0]);
+        DataStream<MinMaxScalerModelData> minMaxScalerModel =
+                MinMaxScalerModelData.getModelDataStream(modelDataTable);
+        final String broadcastModelKey = "broadcastModelKey";
+        RowTypeInfo inputTypeInfo = 
TableUtils.getRowTypeInfo(inputs[0].getResolvedSchema());
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        ArrayUtils.addAll(
+                                inputTypeInfo.getFieldTypes(),
+                                ExternalTypeInfo.of(DenseVector.class)),
+                        ArrayUtils.addAll(inputTypeInfo.getFieldNames(), 
getOutputCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, 
minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictLabelFunction(
+                                            broadcastModelKey,
+                                            getMax(),
+                                            getMIN(),
+                                            getFeaturesCol()),
+                                    outputTypeInfo);
+                        });
+        return new Table[] {tEnv.fromDataStream(output)};
+    }
+
+    @Override
+    public Map<Param<?>, Object> getParamMap() {
+        return paramMap;
+    }
+
+    @Override
+    public void save(String path) throws IOException {
+        ReadWriteUtils.saveMetadata(this, path);
+        ReadWriteUtils.saveModelData(
+                MinMaxScalerModelData.getModelDataStream(modelDataTable),
+                path,
+                new MinMaxScalerModelData.ModelDataEncoder());
+    }
+
+    /**
+     * Loads model data from path.
+     *
+     * @param env Stream execution environment.
+     * @param path Model path.
+     * @return MinMaxScalerModel model.
+     */
+    public static MinMaxScalerModel load(StreamExecutionEnvironment env, 
String path)
+            throws IOException {
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+        MinMaxScalerModel model = ReadWriteUtils.loadStageParam(path);
+        DataStream<MinMaxScalerModelData> modelData =
+                ReadWriteUtils.loadModelData(
+                        env, path, new 
MinMaxScalerModelData.ModelDataDecoder());
+        return model.setModelData(tEnv.fromDataStream(modelData));
+    }
+
+    /** This operator loads model data and predicts result. */
+    private static class PredictLabelFunction extends RichMapFunction<Row, 
Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double max;
+        private final double min;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictLabelFunction(
+                String broadcastKey, double max, double min, String 
featureCol) {
+            this.max = max;

Review comment:
       I am still a little bit concerned about the naming. What is the 
difference between the semantic of `min/max` and `minVector/maxVector`?

##########
File path: 
flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.feature.minmaxscaler;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.RichMapPartitionFunction;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.common.datastream.DataStreamUtils;
+import org.apache.flink.ml.linalg.DenseVector;
+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.environment.StreamExecutionEnvironment;
+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.Collector;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+/**
+ * An Estimator which implements the MinMaxScaler algorithm.
+ *
+ * <p>See 
https://en.wikipedia.org/wiki/Feature_scaling#Rescaling_(min-max_normalization).
+ */
+public class MinMaxScaler
+        implements Estimator<MinMaxScaler, MinMaxScalerModel>, 
MinMaxScalerParams<MinMaxScaler> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+
+    public MinMaxScaler() {
+        ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+    }
+
+    @Override
+    public MinMaxScalerModel fit(Table... inputs) {
+        Preconditions.checkArgument(inputs.length == 1);
+        final String featureCol = getFeaturesCol();
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) 
inputs[0]).getTableEnvironment();
+        DataStream<DenseVector> vectors =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                (MapFunction<Row, DenseVector>)
+                                        value -> (DenseVector) 
value.getField(featureCol));
+        DataStream<DenseVector> minMaxVectors =
+                vectors.transform(
+                                "reduceInEachPartition",
+                                vectors.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .transform(
+                                "reduceInFinalPartition",
+                                vectors.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .setParallelism(1);
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(
+                        minMaxVectors,
+                        new RichMapPartitionFunction<DenseVector, 
MinMaxScalerModelData>() {
+                            @Override
+                            public void mapPartition(
+                                    Iterable<DenseVector> values,
+                                    Collector<MinMaxScalerModelData> out) {
+                                Iterator<DenseVector> iter = values.iterator();
+                                DenseVector minDenseVector = iter.next();
+                                DenseVector maxDenseVector = iter.next();
+                                out.collect(
+                                        new 
MinMaxScalerModelData(minDenseVector, maxDenseVector));
+                            }
+                        });
+
+        MinMaxScalerModel model =
+                new 
MinMaxScalerModel().setModelData(tEnv.fromDataStream(modelData));
+        ReadWriteUtils.updateExistingParams(model, getParamMap());
+        return model;
+    }
+
+    /**
+     * A stream operator to compute the min max vectors in each partition of 
the input bounded data
+     * stream.
+     */
+    private static class MinMaxReduceFunctionOperator extends 
AbstractStreamOperator<DenseVector>
+            implements OneInputStreamOperator<DenseVector, DenseVector>, 
BoundedOneInput {
+        private ListState<DenseVector> minDenseVectorState;

Review comment:
       How about using `minState/maxState` or `lowerBoundState` and 
`upperBoundState`?

##########
File path: 
flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.feature;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.ml.feature.minmaxscaler.MinMaxScaler;
+import org.apache.flink.ml.feature.minmaxscaler.MinMaxScalerModel;
+import org.apache.flink.ml.feature.minmaxscaler.MinMaxScalerModelData;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.linalg.Vectors;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.ml.util.StageTestUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import 
org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Schema;
+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.commons.collections.IteratorUtils;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+/** Tests {@link MinMaxScaler} and {@link MinMaxScalerModel}. */
+public class MinMaxScalerTest {
+    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+    private StreamExecutionEnvironment env;
+    private StreamTableEnvironment tEnv;
+    private Table trainData;
+    private Table predictData;
+    private static final List<Row> trainRows =

Review comment:
       nits: How about using `trainData`?

##########
File path: 
flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.feature;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.ml.feature.minmaxscaler.MinMaxScaler;
+import org.apache.flink.ml.feature.minmaxscaler.MinMaxScalerModel;
+import org.apache.flink.ml.feature.minmaxscaler.MinMaxScalerModelData;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.linalg.Vectors;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.ml.util.StageTestUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import 
org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Schema;
+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.commons.collections.IteratorUtils;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+/** Tests {@link MinMaxScaler} and {@link MinMaxScalerModel}. */
+public class MinMaxScalerTest {
+    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+    private StreamExecutionEnvironment env;
+    private StreamTableEnvironment tEnv;
+    private Table trainData;
+    private Table predictData;
+    private static final List<Row> trainRows =
+            new ArrayList<>(
+                    Arrays.asList(
+                            Row.of(Vectors.dense(0.0, 3.0)),
+                            Row.of(Vectors.dense(2.1, 0.0)),
+                            Row.of(Vectors.dense(4.1, 5.1)),
+                            Row.of(Vectors.dense(6.1, 8.1)),
+                            Row.of(Vectors.dense(200, 300))));
+    private static final List<Row> predictRows =
+            new 
ArrayList<>(Collections.singletonList(Row.of(Vectors.dense(150.0, 90.0))));
+
+    @Before
+    public void before() {
+        Configuration config = new Configuration();
+        
config.set(ExecutionCheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, 
true);
+        env = StreamExecutionEnvironment.getExecutionEnvironment(config);
+        env.setParallelism(4);
+        env.enableCheckpointing(100);
+        env.setRestartStrategy(RestartStrategies.noRestart());
+        tEnv = StreamTableEnvironment.create(env);
+        Schema schema = Schema.newBuilder().column("f0", 
DataTypes.of(DenseVector.class)).build();
+        DataStream<Row> dataStream = env.fromCollection(trainRows);
+        trainData = tEnv.fromDataStream(dataStream, schema).as("features");
+        DataStream<Row> predDataStream = env.fromCollection(predictRows);
+        predictData = tEnv.fromDataStream(predDataStream, 
schema).as("features");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol) 
throws Exception {
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) 
output).getTableEnvironment();
+        DataStream<DenseVector> stream =
+                tEnv.toDataStream(output)
+                        .map(
+                                (MapFunction<Row, DenseVector>)
+                                        row -> (DenseVector) 
row.getField(outputCol));
+        List<DenseVector> result = 
IteratorUtils.toList(stream.executeAndCollect());

Review comment:
       Shall we also check the size of `result`?




-- 
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]


Reply via email to