You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/01/11 11:40:26 UTC

[GitHub] [flink-ml] weibozhao opened a new pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

weibozhao opened a new pull request #54:
URL: https://github.com/apache/flink-ml/pull/54


   Add Transformer and Estimator of MinMaxScaler in FlinkML.


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r826536318



##########
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:
       OK




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829015961



##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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(trainData);
+        trainDataTable = tEnv.fromDataStream(dataStream, schema).as("features");
+        DataStream<Row> predDataStream = env.fromCollection(predictRows);
+        predictDataTable = tEnv.fromDataStream(predDataStream, schema).as("features");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol, DenseVector expected)
+            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(1, result.size());
+        assertEquals(expected, 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(trainDataTable.as("test_features"));
+        Table output = model.transform(predictDataTable.as("test_features"))[0];
+        assertEquals(
+                Arrays.asList("test_features", "test_output"),
+                output.getResolvedSchema().getColumnNames());
+    }
+
+    @Test
+    public void testFewerDistinctPointsThanCluster() throws Exception {

Review comment:
       Kmeans and knn user this name already, I just take it from other ut.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r831695433



##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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, 400))));
+    private static final List<Row> predictData =
+            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);
+        trainDataTable = tEnv.fromDataStream(env.fromCollection(trainData)).as("features");
+        predictDataTable = tEnv.fromDataStream(env.fromCollection(predictData)).as("features");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol, DenseVector expected)
+            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(1, result.size());
+        assertEquals(expected, result.get(0));
+    }
+
+    @Test
+    public void testParam() {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        assertEquals("features", minMaxScaler.getFeaturesCol());
+        assertEquals("prediction", minMaxScaler.getPredictionCol());
+        assertEquals(1.0, minMaxScaler.getMax(), 0.0001);
+        assertEquals(0.0, minMaxScaler.getMin(), 0.0001);
+        minMaxScaler
+                .setFeaturesCol("test_features")
+                .setPredictionCol("test_output")
+                .setMax(4.0)
+                .setMin(1.0);
+        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()
+                        .setFeaturesCol("test_features")
+                        .setPredictionCol("test_output")
+                        .setMin(1.0)
+                        .setMax(4.0);
+
+        MinMaxScalerModel model = minMaxScaler.fit(trainDataTable.as("test_features"));
+        Table output = model.transform(predictDataTable.as("test_features"))[0];
+        assertEquals(
+                Arrays.asList("test_features", "test_output"),
+                output.getResolvedSchema().getColumnNames());
+    }
+
+    @Test
+    public void testFewerSamplesThanParallel() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScalerModel model = minMaxScaler.fit(predictDataTable);
+        Table result = model.transform(predictDataTable)[0];
+        verifyPredictionResult(result, minMaxScaler.getPredictionCol(), Vectors.dense(0.5, 0.5));
+    }
+
+    @Test
+    public void testMaxValueEqualsMinValueButPredictValueNotEquals() throws Exception {
+        List<Row> userDefineData =
+                new ArrayList<>(Collections.singletonList(Row.of(Vectors.dense(30.0, 50.0))));
+        Table userDefineDataTable =
+                tEnv.fromDataStream(env.fromCollection(userDefineData)).as("features");
+        MinMaxScaler minMaxScaler = new MinMaxScaler().setMax(10.0).setMin(0.0);
+        MinMaxScalerModel model = minMaxScaler.fit(predictDataTable);
+        Table result = model.transform(userDefineDataTable)[0];
+        verifyPredictionResult(result, minMaxScaler.getPredictionCol(), Vectors.dense(5.0, 5.0));
+    }
+
+    @Test
+    public void testFitAndPredict() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScalerModel minMaxScalerModel = minMaxScaler.fit(trainDataTable);
+        Table output = minMaxScalerModel.transform(predictDataTable)[0];
+        verifyPredictionResult(output, minMaxScaler.getPredictionCol(), Vectors.dense(0.75, 0.225));
+    }
+
+    @Test
+    public void testSaveLoadAndPredict() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScaler loadedMinMaxScaler =
+                StageTestUtils.saveAndReload(
+                        env, minMaxScaler, tempFolder.newFolder().getAbsolutePath());
+        MinMaxScalerModel minMaxScalerModel = loadedMinMaxScaler.fit(trainDataTable);
+        minMaxScalerModel =
+                StageTestUtils.saveAndReload(
+                        env, minMaxScalerModel, tempFolder.newFolder().getAbsolutePath());
+        assertEquals(
+                Arrays.asList("minVector", "maxVector"),
+                minMaxScalerModel.getModelData()[0].getResolvedSchema().getColumnNames());
+        Table output = minMaxScalerModel.transform(predictDataTable)[0];
+        verifyPredictionResult(output, minMaxScaler.getPredictionCol(), Vectors.dense(0.75, 0.225));
+    }
+
+    @Test
+    public void testGetModelData() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScalerModel minMaxScalerModel = minMaxScaler.fit(trainDataTable);
+        Table modelData = minMaxScalerModel.getModelData()[0];
+        DataStream<Row> output = tEnv.toDataStream(modelData);
+        assertEquals("minVector", modelData.getResolvedSchema().getColumnNames().get(0));

Review comment:
       OK




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r831701594



##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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, 400))));
+    private static final List<Row> predictData =
+            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);
+        trainDataTable = tEnv.fromDataStream(env.fromCollection(trainData)).as("features");
+        predictDataTable = tEnv.fromDataStream(env.fromCollection(predictData)).as("features");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol, DenseVector expected)
+            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(1, result.size());
+        assertEquals(expected, result.get(0));
+    }
+
+    @Test
+    public void testParam() {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        assertEquals("features", minMaxScaler.getFeaturesCol());
+        assertEquals("prediction", minMaxScaler.getPredictionCol());
+        assertEquals(1.0, minMaxScaler.getMax(), 0.0001);
+        assertEquals(0.0, minMaxScaler.getMin(), 0.0001);
+        minMaxScaler
+                .setFeaturesCol("test_features")
+                .setPredictionCol("test_output")
+                .setMax(4.0)
+                .setMin(1.0);
+        assertEquals("test_features", minMaxScaler.getFeaturesCol());
+        assertEquals(1.0, minMaxScaler.getMin(), 0.0001);

Review comment:
       OK




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r831698237



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModelData.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.serialization.Encoder;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.src.reader.SimpleStreamFormat;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.linalg.typeinfo.DenseVectorSerializer;
+import org.apache.flink.streaming.api.datastream.DataStream;
+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 java.io.EOFException;
+import java.io.IOException;
+import java.io.OutputStream;
+
+/**
+ * Model data of {@link MinMaxScalerModel}.
+ *
+ * <p>This class also provides methods to convert model data from Table to a data stream, and
+ * classes to save/load model data.
+ */
+public class MinMaxScalerModelData {
+    public DenseVector minVector;
+
+    public DenseVector maxVector;
+
+    public MinMaxScalerModelData() {}
+
+    public MinMaxScalerModelData(DenseVector minVector, DenseVector maxVector) {
+        this.minVector = minVector;
+        this.maxVector = maxVector;
+    }
+
+    /**
+     * Converts the table model to a data stream.
+     *
+     * @param modelDataTable The table model data.
+     * @return The data stream model data.
+     */
+    public static DataStream<MinMaxScalerModelData> getModelDataStream(Table modelDataTable) {
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) modelDataTable).getTableEnvironment();
+        return tEnv.toDataStream(modelDataTable)
+                .map(
+                        x ->
+                                new MinMaxScalerModelData(
+                                        (DenseVector) x.getField(0), (DenseVector) x.getField(1)));
+    }
+
+    /** Encoder for {@link MinMaxScalerModelData}. */
+    public static class ModelDataEncoder implements Encoder<MinMaxScalerModelData> {
+        @Override
+        public void encode(MinMaxScalerModelData minMaxScalerModelData, OutputStream outputStream)

Review comment:
       OK, I will refine knnModelData, too.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 commented on pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
lindong28 commented on pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#issuecomment-1017123834


   Thanks for the PR.
   
   Before we continue to more and more algorithm, what's our plan to making sure these algorithm's implementation could meet our performance target? I am concerned that if we add more algorithms before we have a benchmark plan, we might end up refactoring added algorithms in a unnecessary manner, which is probably not a good long term solution.


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r830722471



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {

Review comment:
       I got it. But in BLAS.axpy, a is a double value which is a constant, not a value in vector.
   If use BLAS, we need two BLAS func: vec.*vec and axpy. This means we need two vector loop.  I don't think it's more efficiency. 
   
   Your code can avoid the judge of "if ((minVector.values[i] - maxVector.values[i]) == 0.0)" in transform, I will refine my code as you done. 
   
   Also, I will add a ut for condition of you said. 




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
lindong28 commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r831112873



##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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, 400))));
+    private static final List<Row> predictData =
+            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);
+        trainDataTable = tEnv.fromDataStream(env.fromCollection(trainData)).as("features");
+        predictDataTable = tEnv.fromDataStream(env.fromCollection(predictData)).as("features");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol, DenseVector expected)
+            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(1, result.size());
+        assertEquals(expected, result.get(0));
+    }
+
+    @Test
+    public void testParam() {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        assertEquals("features", minMaxScaler.getFeaturesCol());
+        assertEquals("prediction", minMaxScaler.getPredictionCol());
+        assertEquals(1.0, minMaxScaler.getMax(), 0.0001);
+        assertEquals(0.0, minMaxScaler.getMin(), 0.0001);
+        minMaxScaler
+                .setFeaturesCol("test_features")
+                .setPredictionCol("test_output")
+                .setMax(4.0)
+                .setMin(1.0);
+        assertEquals("test_features", minMaxScaler.getFeaturesCol());
+        assertEquals(1.0, minMaxScaler.getMin(), 0.0001);

Review comment:
       Maybe we should unify the value of epsilon used to check double values in all tests, instead of having each test uses different values.
   
   How about we use `1e-5` here and all other tests? `1e-5` seems a bit stricter and also shorter than `0.0001`. It is used in `KMeansTest`.
   
   We can change other tests in the future.

##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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, 400))));
+    private static final List<Row> predictData =
+            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);
+        trainDataTable = tEnv.fromDataStream(env.fromCollection(trainData)).as("features");
+        predictDataTable = tEnv.fromDataStream(env.fromCollection(predictData)).as("features");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol, DenseVector expected)
+            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(1, result.size());
+        assertEquals(expected, result.get(0));
+    }
+
+    @Test
+    public void testParam() {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        assertEquals("features", minMaxScaler.getFeaturesCol());
+        assertEquals("prediction", minMaxScaler.getPredictionCol());
+        assertEquals(1.0, minMaxScaler.getMax(), 0.0001);
+        assertEquals(0.0, minMaxScaler.getMin(), 0.0001);
+        minMaxScaler
+                .setFeaturesCol("test_features")
+                .setPredictionCol("test_output")
+                .setMax(4.0)
+                .setMin(1.0);
+        assertEquals("test_features", minMaxScaler.getFeaturesCol());
+        assertEquals(1.0, minMaxScaler.getMin(), 0.0001);

Review comment:
       Maybe we should unify the value of epsilon used to check double values in all tests, instead of having each test uses different values.
   
   How about we use `1e-5` here and all other tests? `1e-5` seems a bit stricter and also shorter than `0.0001`. It is used in `KMeansTest`.
   
   We can change other tests in a followup PR.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r831701382



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,189 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private final String broadcastKey;
+        private MinMaxScalerModelData minMaxScalerModelData;

Review comment:
       OK, I will refine it.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r831817366



##########
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;
+            this.min = min;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {

Review comment:
       I have discussed with @zhipeng93. We think, it's better to remove this exception. I will refine the code later. 




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829674316



##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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 =

Review comment:
       OK




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829723798



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {

Review comment:
       > if the feature vector exists, but its dimension is different from that of maxVector, or its max/min value exceeds maxVector/minVector's range, then maybe we should also throw exception.
   
   If `featureValue[i] != maxVector[i]`, the input data would have violated the rule above and would throw exceptions. So there would be no result generated in both cases, right?
   
   From this, I think we should also add tests about illegal inputs, like `OneHotEncoderTest.testNonIndexedPredictData`.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829014907



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {
+                    if ((minVector.values[i] - maxVector.values[i]) != 0.0) {
+                        outputVector.values[i] =
+                                (feature.values[i] - minVector.values[i])
+                                                / (maxVector.values[i] - minVector.values[i])

Review comment:
       I think spanVector maybe not very good. 
   1. In Spark and alink, maxVector and minVector are saved for modelData. 
   2. If we spanVector, code may be not clear as minVector and maxVector used. 

##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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(trainData);
+        trainDataTable = tEnv.fromDataStream(dataStream, schema).as("features");
+        DataStream<Row> predDataStream = env.fromCollection(predictRows);
+        predictDataTable = tEnv.fromDataStream(predDataStream, schema).as("features");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol, DenseVector expected)
+            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(1, result.size());
+        assertEquals(expected, 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(trainDataTable.as("test_features"));
+        Table output = model.transform(predictDataTable.as("test_features"))[0];
+        assertEquals(
+                Arrays.asList("test_features", "test_output"),
+                output.getResolvedSchema().getColumnNames());
+    }
+
+    @Test
+    public void testFewerDistinctPointsThanCluster() throws Exception {

Review comment:
       Kmeans and knn user this name already, I just take it from other ut.

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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> features =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                (MapFunction<Row, DenseVector>)
+                                        value -> (DenseVector) value.getField(featureCol));
+        DataStream<DenseVector> minMaxValues =
+                features.transform(
+                                "reduceInEachPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .transform(
+                                "reduceInFinalPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .setParallelism(1);
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(
+                        minMaxValues,
+                        new RichMapPartitionFunction<DenseVector, MinMaxScalerModelData>() {
+                            @Override
+                            public void mapPartition(
+                                    Iterable<DenseVector> values,
+                                    Collector<MinMaxScalerModelData> out) {
+                                Iterator<DenseVector> iter = values.iterator();
+                                DenseVector minVector = iter.next();
+                                DenseVector maxVector = iter.next();
+                                out.collect(new MinMaxScalerModelData(minVector, maxVector));
+                            }
+                        });
+
+        MinMaxScalerModel model =
+                new MinMaxScalerModel().setModelData(tEnv.fromDataStream(modelData));
+        ReadWriteUtils.updateExistingParams(model, getParamMap());
+        return model;
+    }
+
+    /**
+     * A stream operator to compute the min and max values in each partition of the input bounded
+     * data stream.
+     */
+    private static class MinMaxReduceFunctionOperator extends AbstractStreamOperator<DenseVector>
+            implements OneInputStreamOperator<DenseVector, DenseVector>, BoundedOneInput {
+        private ListState<DenseVector> minState;
+        private ListState<DenseVector> maxState;
+
+        private DenseVector minVector;
+        private DenseVector maxVector;
+
+        @Override
+        public void endInput() {
+            if (minVector != null) {
+                output.collect(new StreamRecord<>(minVector));
+            }
+            if (maxVector != null) {
+                output.collect(new StreamRecord<>(maxVector));
+            }
+        }
+
+        @Override
+        public void processElement(StreamRecord<DenseVector> streamRecord) {
+            DenseVector currentValue = streamRecord.getValue();
+            if (minVector == null) {
+                int vecSize = currentValue.size();
+                minVector = new DenseVector(vecSize);
+                maxVector = new DenseVector(vecSize);
+                System.arraycopy(currentValue.values, 0, minVector.values, 0, vecSize);
+                System.arraycopy(currentValue.values, 0, maxVector.values, 0, vecSize);
+
+            } else {
+                for (int i = 0; i < currentValue.size(); ++i) {
+                    minVector.values[i] = Math.min(minVector.values[i], currentValue.values[i]);

Review comment:
       As I Know, blas has no function like minVector or maxVector.
   If user blas1 function(minVector is blas1 function), loop may not be avoided, too.

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {

Review comment:
       There is a judge about max-min==0 exists in the loop, then we can not use blas here. 
   If we want to use blas, we need another loop to prepare data. I think it is more expensive.

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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> features =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                (MapFunction<Row, DenseVector>)
+                                        value -> (DenseVector) value.getField(featureCol));
+        DataStream<DenseVector> minMaxValues =
+                features.transform(
+                                "reduceInEachPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .transform(
+                                "reduceInFinalPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .setParallelism(1);
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(
+                        minMaxValues,
+                        new RichMapPartitionFunction<DenseVector, MinMaxScalerModelData>() {
+                            @Override
+                            public void mapPartition(
+                                    Iterable<DenseVector> values,
+                                    Collector<MinMaxScalerModelData> out) {
+                                Iterator<DenseVector> iter = values.iterator();
+                                DenseVector minVector = iter.next();
+                                DenseVector maxVector = iter.next();
+                                out.collect(new MinMaxScalerModelData(minVector, maxVector));
+                            }
+                        });
+
+        MinMaxScalerModel model =
+                new MinMaxScalerModel().setModelData(tEnv.fromDataStream(modelData));
+        ReadWriteUtils.updateExistingParams(model, getParamMap());
+        return model;
+    }
+
+    /**
+     * A stream operator to compute the min and max values in each partition of the input bounded
+     * data stream.
+     */
+    private static class MinMaxReduceFunctionOperator extends AbstractStreamOperator<DenseVector>
+            implements OneInputStreamOperator<DenseVector, DenseVector>, BoundedOneInput {
+        private ListState<DenseVector> minState;
+        private ListState<DenseVector> maxState;
+
+        private DenseVector minVector;
+        private DenseVector maxVector;
+
+        @Override
+        public void endInput() {
+            if (minVector != null) {
+                output.collect(new StreamRecord<>(minVector));
+            }
+            if (maxVector != null) {
+                output.collect(new StreamRecord<>(maxVector));
+            }
+        }
+
+        @Override
+        public void processElement(StreamRecord<DenseVector> streamRecord) {
+            DenseVector currentValue = streamRecord.getValue();
+            if (minVector == null) {
+                int vecSize = currentValue.size();
+                minVector = new DenseVector(vecSize);
+                maxVector = new DenseVector(vecSize);
+                System.arraycopy(currentValue.values, 0, minVector.values, 0, vecSize);
+                System.arraycopy(currentValue.values, 0, maxVector.values, 0, vecSize);
+
+            } else {
+                for (int i = 0; i < currentValue.size(); ++i) {
+                    minVector.values[i] = Math.min(minVector.values[i], currentValue.values[i]);

Review comment:
       As I Know, blas has no function like minVector or maxVector.
   If user blas1 function(minVector is like blas1 function), loop may not be avoided, too.

##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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(trainData);
+        trainDataTable = tEnv.fromDataStream(dataStream, schema).as("features");
+        DataStream<Row> predDataStream = env.fromCollection(predictRows);
+        predictDataTable = tEnv.fromDataStream(predDataStream, schema).as("features");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol, DenseVector expected)
+            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(1, result.size());
+        assertEquals(expected, 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(trainDataTable.as("test_features"));
+        Table output = model.transform(predictDataTable.as("test_features"))[0];
+        assertEquals(
+                Arrays.asList("test_features", "test_output"),
+                output.getResolvedSchema().getColumnNames());
+    }
+
+    @Test
+    public void testFewerDistinctPointsThanCluster() throws Exception {

Review comment:
       Kmeans and knn use this name already, I just take it from other ut.

##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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(trainData);
+        trainDataTable = tEnv.fromDataStream(dataStream, schema).as("features");
+        DataStream<Row> predDataStream = env.fromCollection(predictRows);
+        predictDataTable = tEnv.fromDataStream(predDataStream, schema).as("features");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol, DenseVector expected)
+            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(1, result.size());
+        assertEquals(expected, 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(trainDataTable.as("test_features"));
+        Table output = model.transform(predictDataTable.as("test_features"))[0];
+        assertEquals(
+                Arrays.asList("test_features", "test_output"),
+                output.getResolvedSchema().getColumnNames());
+    }
+
+    @Test
+    public void testFewerDistinctPointsThanCluster() throws Exception {

Review comment:
       OK,I got it. I will refine it later.

##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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 =

Review comment:
       OK

##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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(trainData);
+        trainDataTable = tEnv.fromDataStream(dataStream, schema).as("features");
+        DataStream<Row> predDataStream = env.fromCollection(predictRows);
+        predictDataTable = tEnv.fromDataStream(predDataStream, schema).as("features");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol, DenseVector expected)
+            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(1, result.size());
+        assertEquals(expected, 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());

Review comment:
       OK

##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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(trainData);
+        trainDataTable = tEnv.fromDataStream(dataStream, schema).as("features");
+        DataStream<Row> predDataStream = env.fromCollection(predictRows);
+        predictDataTable = tEnv.fromDataStream(predDataStream, schema).as("features");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol, DenseVector expected)
+            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(1, result.size());
+        assertEquals(expected, 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(trainDataTable.as("test_features"));
+        Table output = model.transform(predictDataTable.as("test_features"))[0];
+        assertEquals(
+                Arrays.asList("test_features", "test_output"),
+                output.getResolvedSchema().getColumnNames());
+    }
+
+    @Test
+    public void testFewerDistinctPointsThanCluster() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScalerModel model = minMaxScaler.fit(predictDataTable);
+        Table result = model.transform(predictDataTable)[0];
+        verifyPredictionResult(result, minMaxScaler.getPredictionCol(), Vectors.dense(0.5, 0.5));
+    }
+
+    @Test
+    public void testFitAndPredict() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScalerModel minMaxScalerModel = minMaxScaler.fit(trainDataTable);
+        Table output = minMaxScalerModel.transform(predictDataTable)[0];
+        verifyPredictionResult(output, minMaxScaler.getPredictionCol(), Vectors.dense(0.75, 0.3));
+    }
+
+    @Test
+    public void testSaveLoadAndPredict() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScaler loadedMinMaxScaler =
+                StageTestUtils.saveAndReload(
+                        env, minMaxScaler, tempFolder.newFolder().getAbsolutePath());
+        MinMaxScalerModel minMaxScalerModel = loadedMinMaxScaler.fit(trainDataTable);
+        minMaxScalerModel =
+                StageTestUtils.saveAndReload(
+                        env, minMaxScalerModel, tempFolder.newFolder().getAbsolutePath());
+        assertEquals(
+                Arrays.asList("minVector", "maxVector"),
+                minMaxScalerModel.getModelData()[0].getResolvedSchema().getColumnNames());
+        Table output = minMaxScalerModel.transform(predictDataTable)[0];
+        verifyPredictionResult(output, minMaxScaler.getPredictionCol(), Vectors.dense(0.75, 0.3));
+    }
+
+    @Test
+    public void testModelSaveLoadAndPredict() throws Exception {

Review comment:
       OK

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.iteration.operator.OperatorStateUtils;
+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).

Review comment:
       OK

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.iteration.operator.OperatorStateUtils;
+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> features =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                (MapFunction<Row, DenseVector>)
+                                        value -> (DenseVector) value.getField(featureCol));
+        DataStream<DenseVector> minMaxValues =
+                features.transform(
+                                "reduceInEachPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .transform(
+                                "reduceInFinalPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .setParallelism(1);
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(
+                        minMaxValues,
+                        new RichMapPartitionFunction<DenseVector, MinMaxScalerModelData>() {
+                            @Override
+                            public void mapPartition(
+                                    Iterable<DenseVector> values,
+                                    Collector<MinMaxScalerModelData> out) {
+                                Iterator<DenseVector> iter = values.iterator();
+                                DenseVector minVector = iter.next();
+                                DenseVector maxVector = iter.next();
+                                out.collect(new MinMaxScalerModelData(minVector, maxVector));
+                            }
+                        });
+
+        MinMaxScalerModel model =
+                new MinMaxScalerModel().setModelData(tEnv.fromDataStream(modelData));
+        ReadWriteUtils.updateExistingParams(model, getParamMap());
+        return model;
+    }
+
+    /**
+     * A stream operator to compute the min and max values in each partition of the input bounded
+     * data stream.
+     */
+    private static class MinMaxReduceFunctionOperator extends AbstractStreamOperator<DenseVector>
+            implements OneInputStreamOperator<DenseVector, DenseVector>, BoundedOneInput {
+        private ListState<DenseVector> minState;
+        private ListState<DenseVector> maxState;
+
+        private DenseVector minVector;
+        private DenseVector maxVector;
+
+        @Override
+        public void endInput() {
+            if (minVector != null) {

Review comment:
       OK

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {

Review comment:
       If featureValue[i] != maxVector[i], above code may not get the same result as original code.
   In read world, data for train and predict are different, featureValue[i] != maxVector[i] is existed. 

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {

Review comment:
       When maxVector[i] == minVector[i],
   If featureValue[i] != maxVector[i], above code may not get the same result as original code.
   In read world, data for train and predict are different, featureValue[i] != maxVector[i] is existed. 

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {
+                    if ((minVector.values[i] - maxVector.values[i]) != 0.0) {
+                        outputVector.values[i] =
+                                (feature.values[i] - minVector.values[i])
+                                                / (maxVector.values[i] - minVector.values[i])

Review comment:
       val scaleArray = Array.tabulate(numFeatures) { i =>
         val range = originalMax(i) - originalMin(i)

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {
+                    if ((minVector.values[i] - maxVector.values[i]) != 0.0) {
+                        outputVector.values[i] =
+                                (feature.values[i] - minVector.values[i])
+                                                / (maxVector.values[i] - minVector.values[i])

Review comment:
       In spark,  maxVector and minVector are saved for modelData. The scaleArray is computed before transform.
   I will do it as spark done.

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {

Review comment:
       When maxVector[i] == minVector[i],
   If featureValue[i] != maxVector[i], above code may not get the same result as original code.
   In real world, data for train and predict are different, featureValue[i] != maxVector[i] is existed. 

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {

Review comment:
       When maxVector[i] == minVector[i],
   If featureValue[i] != maxVector[i], above code may not get the same result as original code.
   In real world, data for train and predict may be different, featureValue[i] != maxVector[i] is existed. 

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {

Review comment:
       OK, I have add exception.

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {

Review comment:
       I don‘t think so. 
   If featureValue[i] != maxVector[i],  predict data may have mo problem. For train data defines the rule and we can't constrain predict data follow this rule. 
   
   For example, in train data one feature is people's age, all feature values in the train data is 20 years old. If predict data has a sample which feature is 21 years old which will be scaled to 0.5*(max - min). 
     




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r823423537



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.RichMapPartitionFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.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.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.Map;
+
+/** An Estimator which implements the MinMaxScaler algorithm. */
+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);
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment();
+        DataStream<Tuple2<DenseVector, DenseVector>> minMaxVectors =
+                computeMinMaxVectors(tEnv.toDataStream(inputs[0]), getFeaturesCol());
+        DataStream<MinMaxScalerModelData> modelData = genModelData(minMaxVectors);
+        MinMaxScalerModel model =
+                new MinMaxScalerModel().setModelData(tEnv.fromDataStream(modelData));
+        ReadWriteUtils.updateExistingParams(model, getParamMap());
+        return model;
+    }
+
+    @Override
+    public Map<Param<?>, Object> getParamMap() {
+        return paramMap;
+    }
+
+    @Override
+    public void save(String path) throws IOException {
+        ReadWriteUtils.saveMetadata(this, path);
+    }
+
+    public static MinMaxScaler load(StreamExecutionEnvironment env, String path)
+            throws IOException {
+        return ReadWriteUtils.loadStageParam(path);
+    }
+
+    /**
+     * Generates minMax scaler model data.
+     *
+     * @param minMaxVectors Input distributed minMaxVectors.
+     * @return MinMax scaler model data.
+     */
+    private static DataStream<MinMaxScalerModelData> genModelData(
+            DataStream<Tuple2<DenseVector, DenseVector>> minMaxVectors) {
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(

Review comment:
       This mapPartition only has fewer record. For every worker only has one record. I think it's no efficient problem.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829681672



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {
+                    if ((minVector.values[i] - maxVector.values[i]) != 0.0) {
+                        outputVector.values[i] =
+                                (feature.values[i] - minVector.values[i])
+                                                / (maxVector.values[i] - minVector.values[i])

Review comment:
       In spark,  maxVector and minVector are saved for modelData. The scaleArray is computed before transform.
   I will do it as spark done.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829647371



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {

Review comment:
       Just for this code, I think there is a simple way to do the max == min judgement just once, like follows.
   ```java
   
           public Row map(Row row) {
               if (minMaxScalerModelData == null) {
                   minMaxScalerModelData =
                           (MinMaxScalerModelData)
                                   getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
                   maxVector = minMaxScalerModelData.maxVector;
                   minVector = minMaxScalerModelData.minVector;
                   for (int i = 0; i < maxVector.size(); ++i) {
                       if ((minVector.values[i] - maxVector.values[i]) == 0.0) {
                           maxVector.values[i] = maxVector.values[i] + 1.0;
                           minVector.values[i] = minVector.values[i] - 1.0;
                       }
                   }
               }
               DenseVector feature = (DenseVector) row.getField(featureCol);
               DenseVector outputVector = new DenseVector(maxVector.size());
               if (feature != null) {
                   for (int i = 0; i < maxVector.size(); ++i) {
                       outputVector.values[i] =
                               (feature.values[i] - minVector.values[i])
                                       / (maxVector.values[i] - minVector.values[i])
                                       * (upperBound - lowerBound)
                                       + lowerBound;
                   }
                   return Row.join(row, Row.of(outputVector));
               } else {
                   throw new RuntimeException("Feature value is null, please check your input data.");
               }
           }
   ```
   With this code, now it is possible to apply BLAS operations.
   
   I noticed that Spark also uses for loop in this and the situation above, so I agree that we may postpone any further optimizations like this after this PR.
   




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829632681



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {

Review comment:
       Got it. If this is the case, then I think MinMaxScaler might not need this parameter either.
   
   What do you think of the other part of the question above?
   > if the feature vector exists, but its dimension is different from that of maxVector, or its max/min value exceeds maxVector/minVector's range, then maybe we should also throw exception.

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {
+                    if ((minVector.values[i] - maxVector.values[i]) != 0.0) {
+                        outputVector.values[i] =
+                                (feature.values[i] - minVector.values[i])
+                                                / (maxVector.values[i] - minVector.values[i])

Review comment:
       I'm not sure this is the case for Spark. In Spark's MinMaxScaler.scala I found the following code:
   ```scala
   values(i) = (values(i) - minArray(i)) * scaleArray(i) + minValue
   ```
   which shows that Spark uses `minArray` and `scaleArray`, instead of `minArray` and `maxArray`.

##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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(trainData);
+        trainDataTable = tEnv.fromDataStream(dataStream, schema).as("features");
+        DataStream<Row> predDataStream = env.fromCollection(predictRows);
+        predictDataTable = tEnv.fromDataStream(predDataStream, schema).as("features");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol, DenseVector expected)
+            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(1, result.size());
+        assertEquals(expected, 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(trainDataTable.as("test_features"));
+        Table output = model.transform(predictDataTable.as("test_features"))[0];
+        assertEquals(
+                Arrays.asList("test_features", "test_output"),
+                output.getResolvedSchema().getColumnNames());
+    }
+
+    @Test
+    public void testFewerDistinctPointsThanCluster() throws Exception {

Review comment:
       KMeans is a clustering algorithm, so I understand that there is the concept of cluster. But in MinMaxScaler there is no such "cluster" thing. I think it would be more proper if we rename it to `testMinMaxEquals()`.

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {

Review comment:
       Just for this code, I think there is a simple way to do the max == min judgement just once, like follows.
   ```java
   
           public Row map(Row row) {
               if (minMaxScalerModelData == null) {
                   minMaxScalerModelData =
                           (MinMaxScalerModelData)
                                   getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
                   maxVector = minMaxScalerModelData.maxVector;
                   minVector = minMaxScalerModelData.minVector;
                   for (int i = 0; i < maxVector.size(); ++i) {
                       if ((minVector.values[i] - maxVector.values[i]) == 0.0) {
                           maxVector.values[i] = maxVector.values[i] + 1.0;
                           minVector.values[i] = minVector.values[i] - 1.0;
                       }
                   }
               }
               DenseVector feature = (DenseVector) row.getField(featureCol);
               DenseVector outputVector = new DenseVector(maxVector.size());
               if (feature != null) {
                   for (int i = 0; i < maxVector.size(); ++i) {
                       outputVector.values[i] =
                               (feature.values[i] - minVector.values[i])
                                       / (maxVector.values[i] - minVector.values[i])
                                       * (upperBound - lowerBound)
                                       + lowerBound;
                   }
                   return Row.join(row, Row.of(outputVector));
               } else {
                   throw new RuntimeException("Feature value is null, please check your input data.");
               }
           }
   ```
   With this code, now it is possible to apply BLAS operations.
   
   I noticed that Spark also uses for loop in this and the situation above, so I agree that we may postpone any further optimizations like this after this PR.
   

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {

Review comment:
       > if the feature vector exists, but its dimension is different from that of maxVector, or its max/min value exceeds maxVector/minVector's range, then maybe we should also throw exception.
   
   If `featureValue[i] != maxVector[i]`, the input data would have violated the rule above and would throw exceptions. So there would be no result generated in both cases, right?
   
   From this, I think we should also add tests about illegal inputs, like `OneHotEncoderTest.testNonIndexedPredictData`.

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {

Review comment:
       Got it. Then how about this one?
   ```java
           @Override
           public Row map(Row row) {
               if (minMaxScalerModelData == null) {
                   minMaxScalerModelData =
                           (MinMaxScalerModelData)
                                   getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
                   maxVector = minMaxScalerModelData.maxVector;
                   minVector = minMaxScalerModelData.minVector;
                   scaleVector = new DenseVector(minVector.size());
                   offsetVector = new DenseVector(minVector.size());
                   for (int i = 0; i < maxVector.size(); ++i) {
                       if ((minVector.values[i] - maxVector.values[i]) == 0.0) {
                           scaleVector.values[i] = Double.POSITIVE_INFINITY;
                           offsetVector.values[i] = 0.5;
                       } else {
                           scaleVector.values[i] = maxVector.values[i] - minVector.values[i];
                           offsetVector.values[i] = 0.0;
                       }
                   }
               }
               DenseVector feature = (DenseVector) row.getField(featureCol);
               DenseVector outputVector = new DenseVector(maxVector.size());
               if (feature != null) {
                   for (int i = 0; i < maxVector.size(); ++i) {
                       outputVector.values[i] =
                               ((feature.values[i] - minVector.values[i])
                                                       / scaleVector.values[i]
                                                       + offsetVector.values[i])
                                               * (upperBound - lowerBound)
                                       + lowerBound;
                   }
                   return Row.join(row, Row.of(outputVector));
               } else {
                   throw new RuntimeException("Feature value is null, please check your input data.");
               }
           }
   ```
   
   This code should meet the requirements without using if conditions.

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {

Review comment:
       Besides, could you please add a test case about maxVector[i] == minVector[i] && featureValue[i] != maxVector[i]? I think it would be helpful to test these corner cases.

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {

Review comment:
       Even better:
   ```java
   
           @Override
           public Row map(Row row) {
               if (minMaxScalerModelData == null) {
                   minMaxScalerModelData =
                           (MinMaxScalerModelData)
                                   getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
                   maxVector = minMaxScalerModelData.maxVector;
                   minVector = minMaxScalerModelData.minVector;
                   scaleVector = new DenseVector(minVector.size());
                   offsetVector = new DenseVector(minVector.size());
                   for (int i = 0; i < maxVector.size(); ++i) {
                       if ((minVector.values[i] - maxVector.values[i]) == 0.0) {
                           scaleVector.values[i] = 0.0;
                           offsetVector.values[i] = (upperBound + lowerBound) / 2;
                       } else {
                           scaleVector.values[i] = (upperBound - lowerBound) / (maxVector.values[i] - minVector.values[i]);
                           offsetVector.values[i] = lowerBound - minVector.values[i] * scaleVector.values[i];
                       }
                   }
               }
               DenseVector feature = (DenseVector) row.getField(featureCol);
               DenseVector outputVector = new DenseVector(maxVector.size());
               if (feature != null) {
                   for (int i = 0; i < maxVector.size(); ++i) {
                       outputVector.values[i] = feature.values[i] * scaleVector.values[i] + offsetVector.values[i];
                   }
                   return Row.join(row, Row.of(outputVector));
               } else {
                   throw new RuntimeException("Feature value is null, please check your input data.");
               }
           }
   ```
   
   With this implementation, there would be only one multiplication and one addition in the for loop, which can definitely be replaced by `BLAS.axpy`. It would cause some precision loss (less then 1e-16 on this PR's test data), but I think the performance improvement worth it.

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {
+                    if ((minVector.values[i] - maxVector.values[i]) != 0.0) {
+                        outputVector.values[i] =
+                                (feature.values[i] - minVector.values[i])
+                                                / (maxVector.values[i] - minVector.values[i])

Review comment:
       Sounds good.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r828997214



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {

Review comment:
       Thanks for your comments. 
   
   I think, HasHandleInvalid is not needed here.
   1. Spark and alink have no parameter about invalid. 
   2. If feature range not agree each other, code will throw Exception with array range err. 
   3. In HasHandleInvalid parameter:  KEEP and SKIP are improper here.  




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] zhipeng93 commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
zhipeng93 commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r823332320



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.RichMapPartitionFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.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.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.Map;
+
+/** An Estimator which implements the MinMaxScaler algorithm. */
+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);
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment();
+        DataStream<Tuple2<DenseVector, DenseVector>> minMaxVectors =
+                computeMinMaxVectors(tEnv.toDataStream(inputs[0]), getFeaturesCol());
+        DataStream<MinMaxScalerModelData> modelData = genModelData(minMaxVectors);
+        MinMaxScalerModel model =
+                new MinMaxScalerModel().setModelData(tEnv.fromDataStream(modelData));
+        ReadWriteUtils.updateExistingParams(model, getParamMap());
+        return model;
+    }
+
+    @Override
+    public Map<Param<?>, Object> getParamMap() {
+        return paramMap;
+    }
+
+    @Override
+    public void save(String path) throws IOException {
+        ReadWriteUtils.saveMetadata(this, path);
+    }
+
+    public static MinMaxScaler load(StreamExecutionEnvironment env, String path)
+            throws IOException {
+        return ReadWriteUtils.loadStageParam(path);
+    }
+
+    /**
+     * Generates minMax scaler model data.
+     *
+     * @param minMaxVectors Input distributed minMaxVectors.
+     * @return MinMax scaler model data.
+     */
+    private static DataStream<MinMaxScalerModelData> genModelData(
+            DataStream<Tuple2<DenseVector, DenseVector>> minMaxVectors) {
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(
+                        minMaxVectors,
+                        new RichMapPartitionFunction<
+                                Tuple2<DenseVector, DenseVector>, MinMaxScalerModelData>() {
+                            @Override
+                            public void mapPartition(
+                                    Iterable<Tuple2<DenseVector, DenseVector>> dataPoints,
+                                    Collector<MinMaxScalerModelData> out) {
+                                DenseVector minVector = null;
+                                DenseVector maxVector = null;
+                                int vecSize = 0;
+                                for (Tuple2<DenseVector, DenseVector> dataPoint : dataPoints) {
+                                    if (maxVector == null) {
+                                        vecSize = dataPoint.f0.size();
+                                        maxVector = dataPoint.f1;
+                                        minVector = dataPoint.f0;
+                                    }
+                                    for (int i = 0; i < vecSize; ++i) {

Review comment:
       nits: vecSize could be replaced with `maxVector.size()`

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.RichMapPartitionFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.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.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.Map;
+
+/** An Estimator which implements the MinMaxScaler algorithm. */

Review comment:
       Could you add a doc/link to explain what is a `minmaxscaler` as we did for other algorithms?

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerParams.java
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.ml.common.param.HasFeaturesCol;
+import org.apache.flink.ml.common.param.HasOutputCol;
+import org.apache.flink.ml.param.DoubleParam;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+
+/**
+ * Params for {@link MinMaxScaler}.
+ *
+ * @param <T> The class type of this instance.
+ */
+public interface MinMaxScalerParams<T> extends HasFeaturesCol<T>, HasOutputCol<T> {
+    Param<Double> MAX =
+            new DoubleParam(
+                    "max", "Upper bound after transformation.", 1.0, ParamValidators.notNull());
+
+    default Double getMax() {
+        return get(MAX);
+    }
+
+    default T setMax(Double value) {
+        return set(MAX, value);
+    }
+
+    Param<Double> MIN =
+            new DoubleParam(
+                    "min", "Lower bound after transformation.", 0.0, ParamValidators.notNull());
+
+    default Double getMIN() {

Review comment:
       nit: `getMIN` to `getMin`.

##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,206 @@
+/*
+ * 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());
+        for (DenseVector t2 : result) {
+            assertEquals(Vectors.dense(0.75, 0.3), t2);
+        }
+    }
+
+    @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("output", minMaxScaler.getOutputCol());
+        minMaxScaler
+                .setFeaturesCol("test_features")
+                .setMax(4.0)
+                .setMIN(0.0)
+                .setOutputCol("test_output");
+        assertEquals("test_features", minMaxScaler.getFeaturesCol());
+        assertEquals(0.0, minMaxScaler.getMIN(), 0.0001);
+        assertEquals(4.0, minMaxScaler.getMax(), 0.0001);
+        assertEquals("test_output", minMaxScaler.getOutputCol());
+    }
+
+    @Test
+    public void testFeaturePredictionParam() {
+        MinMaxScaler minMaxScaler =
+                new MinMaxScaler()
+                        .setMIN(0.0)
+                        .setMax(4.0)
+                        .setFeaturesCol("test_features")
+                        .setOutputCol("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() {

Review comment:
       Is this test necessary?

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.RichMapPartitionFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.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.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.Map;
+
+/** An Estimator which implements the MinMaxScaler algorithm. */
+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);
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment();
+        DataStream<Tuple2<DenseVector, DenseVector>> minMaxVectors =
+                computeMinMaxVectors(tEnv.toDataStream(inputs[0]), getFeaturesCol());
+        DataStream<MinMaxScalerModelData> modelData = genModelData(minMaxVectors);
+        MinMaxScalerModel model =
+                new MinMaxScalerModel().setModelData(tEnv.fromDataStream(modelData));
+        ReadWriteUtils.updateExistingParams(model, getParamMap());
+        return model;
+    }
+
+    @Override
+    public Map<Param<?>, Object> getParamMap() {
+        return paramMap;
+    }
+
+    @Override
+    public void save(String path) throws IOException {
+        ReadWriteUtils.saveMetadata(this, path);
+    }
+
+    public static MinMaxScaler load(StreamExecutionEnvironment env, String path)
+            throws IOException {
+        return ReadWriteUtils.loadStageParam(path);
+    }
+
+    /**
+     * Generates minMax scaler model data.
+     *
+     * @param minMaxVectors Input distributed minMaxVectors.
+     * @return MinMax scaler model data.
+     */
+    private static DataStream<MinMaxScalerModelData> genModelData(
+            DataStream<Tuple2<DenseVector, DenseVector>> minMaxVectors) {
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(
+                        minMaxVectors,
+                        new RichMapPartitionFunction<
+                                Tuple2<DenseVector, DenseVector>, MinMaxScalerModelData>() {
+                            @Override
+                            public void mapPartition(
+                                    Iterable<Tuple2<DenseVector, DenseVector>> dataPoints,
+                                    Collector<MinMaxScalerModelData> out) {
+                                DenseVector minVector = null;
+                                DenseVector maxVector = null;
+                                int vecSize = 0;
+                                for (Tuple2<DenseVector, DenseVector> dataPoint : dataPoints) {
+                                    if (maxVector == null) {
+                                        vecSize = dataPoint.f0.size();
+                                        maxVector = dataPoint.f1;
+                                        minVector = dataPoint.f0;
+                                    }
+                                    for (int i = 0; i < vecSize; ++i) {
+                                        minVector.values[i] =
+                                                Math.min(
+                                                        dataPoint.f0.values[i],
+                                                        minVector.values[i]);
+                                        maxVector.values[i] =
+                                                Math.max(
+                                                        dataPoint.f1.values[i],
+                                                        maxVector.values[i]);
+                                    }
+                                }
+                                out.collect(new MinMaxScalerModelData(minVector, maxVector));
+                            }
+                        });
+        modelData.getTransformation().setParallelism(1);
+        return modelData;
+    }
+
+    /**
+     * Computes max and min values of features.
+     *
+     * @param inputData Input data.
+     * @param featureCol Feature column name.
+     * @return Max and min values of features.
+     */
+    private DataStream<Tuple2<DenseVector, DenseVector>> computeMinMaxVectors(

Review comment:
       The logic of `computeMinMaxVectors` and `genModelData` seems almost the same. Is merging them into one a better implementation?

##########
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:
       nit: Are `loweBound` and `upperBound` better names for `max` and `min`?

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/common/param/HasOutputCol.java
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.common.param;
+
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+import org.apache.flink.ml.param.StringParam;
+import org.apache.flink.ml.param.WithParams;
+
+/** Interface for the shared outputCol param. */
+public interface HasOutputCol<T> extends WithParams<T> {

Review comment:
       Why do we need a new param here? How about using the existing one, i.e., `HasPredictionCol`?

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.RichMapPartitionFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.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.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.Map;
+
+/** An Estimator which implements the MinMaxScaler algorithm. */
+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);
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment();
+        DataStream<Tuple2<DenseVector, DenseVector>> minMaxVectors =
+                computeMinMaxVectors(tEnv.toDataStream(inputs[0]), getFeaturesCol());
+        DataStream<MinMaxScalerModelData> modelData = genModelData(minMaxVectors);
+        MinMaxScalerModel model =
+                new MinMaxScalerModel().setModelData(tEnv.fromDataStream(modelData));
+        ReadWriteUtils.updateExistingParams(model, getParamMap());
+        return model;
+    }
+
+    @Override
+    public Map<Param<?>, Object> getParamMap() {
+        return paramMap;
+    }
+
+    @Override
+    public void save(String path) throws IOException {
+        ReadWriteUtils.saveMetadata(this, path);
+    }
+
+    public static MinMaxScaler load(StreamExecutionEnvironment env, String path)
+            throws IOException {
+        return ReadWriteUtils.loadStageParam(path);
+    }
+
+    /**
+     * Generates minMax scaler model data.
+     *
+     * @param minMaxVectors Input distributed minMaxVectors.
+     * @return MinMax scaler model data.
+     */
+    private static DataStream<MinMaxScalerModelData> genModelData(
+            DataStream<Tuple2<DenseVector, DenseVector>> minMaxVectors) {
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(

Review comment:
       nits: using `mapPartition` here may be not that efficient since we cache all the input records in the state.
   
   How about using a self-defined streamOperator as [1]?
   
   [1] https://github.com/zhipeng93/flink-ml/blob/ccd3f9919dbdd3c5123f1cc88636f832f8215ffb/flink-ml-core/src/main/java/org/apache/flink/ml/common/datastream/DataStreamUtils.java#L68

##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,206 @@
+/*
+ * 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());
+        for (DenseVector t2 : result) {

Review comment:
       Is this loop necessary here?

##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,206 @@
+/*
+ * 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());
+        for (DenseVector t2 : result) {
+            assertEquals(Vectors.dense(0.75, 0.3), t2);
+        }
+    }
+
+    @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("output", minMaxScaler.getOutputCol());
+        minMaxScaler
+                .setFeaturesCol("test_features")
+                .setMax(4.0)
+                .setMIN(0.0)

Review comment:
       How about use different default value for `min`?

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerParams.java
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.ml.common.param.HasFeaturesCol;
+import org.apache.flink.ml.common.param.HasOutputCol;
+import org.apache.flink.ml.param.DoubleParam;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+
+/**
+ * Params for {@link MinMaxScaler}.
+ *
+ * @param <T> The class type of this instance.
+ */
+public interface MinMaxScalerParams<T> extends HasFeaturesCol<T>, HasOutputCol<T> {
+    Param<Double> MAX =
+            new DoubleParam(
+                    "max", "Upper bound after transformation.", 1.0, ParamValidators.notNull());
+
+    default Double getMax() {
+        return get(MAX);
+    }
+
+    default T setMax(Double value) {
+        return set(MAX, value);
+    }
+
+    Param<Double> MIN =
+            new DoubleParam(
+                    "min", "Lower bound after transformation.", 0.0, ParamValidators.notNull());
+
+    default Double getMIN() {
+        return get(MIN);
+    }
+
+    default T setMIN(Double value) {

Review comment:
       ditto.

##########
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;
+            this.min = min;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {

Review comment:
       Should we throw an exception here if the input data does not contain the `featureCol`?




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r831700307



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,189 @@
+/*
+ * 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)),

Review comment:
       It's better.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r830722471



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {

Review comment:
       I got it. But in BLAS.axpy, a is a double value which is a constant, not a value in vector.
   If use BLAS, we need two BLAS func: vec.*vec and axpy. This means we need two vector loop.  I don't think it's more efficiency. 
   
   Your code can avoid the judge of "if ((minVector.values[i] - maxVector.values[i]) == 0.0)" in transform, I will refine my code as you done. 
   




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829018999



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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> features =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                (MapFunction<Row, DenseVector>)
+                                        value -> (DenseVector) value.getField(featureCol));
+        DataStream<DenseVector> minMaxValues =
+                features.transform(
+                                "reduceInEachPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .transform(
+                                "reduceInFinalPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .setParallelism(1);
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(
+                        minMaxValues,
+                        new RichMapPartitionFunction<DenseVector, MinMaxScalerModelData>() {
+                            @Override
+                            public void mapPartition(
+                                    Iterable<DenseVector> values,
+                                    Collector<MinMaxScalerModelData> out) {
+                                Iterator<DenseVector> iter = values.iterator();
+                                DenseVector minVector = iter.next();
+                                DenseVector maxVector = iter.next();
+                                out.collect(new MinMaxScalerModelData(minVector, maxVector));
+                            }
+                        });
+
+        MinMaxScalerModel model =
+                new MinMaxScalerModel().setModelData(tEnv.fromDataStream(modelData));
+        ReadWriteUtils.updateExistingParams(model, getParamMap());
+        return model;
+    }
+
+    /**
+     * A stream operator to compute the min and max values in each partition of the input bounded
+     * data stream.
+     */
+    private static class MinMaxReduceFunctionOperator extends AbstractStreamOperator<DenseVector>
+            implements OneInputStreamOperator<DenseVector, DenseVector>, BoundedOneInput {
+        private ListState<DenseVector> minState;
+        private ListState<DenseVector> maxState;
+
+        private DenseVector minVector;
+        private DenseVector maxVector;
+
+        @Override
+        public void endInput() {
+            if (minVector != null) {
+                output.collect(new StreamRecord<>(minVector));
+            }
+            if (maxVector != null) {
+                output.collect(new StreamRecord<>(maxVector));
+            }
+        }
+
+        @Override
+        public void processElement(StreamRecord<DenseVector> streamRecord) {
+            DenseVector currentValue = streamRecord.getValue();
+            if (minVector == null) {
+                int vecSize = currentValue.size();
+                minVector = new DenseVector(vecSize);
+                maxVector = new DenseVector(vecSize);
+                System.arraycopy(currentValue.values, 0, minVector.values, 0, vecSize);
+                System.arraycopy(currentValue.values, 0, maxVector.values, 0, vecSize);
+
+            } else {
+                for (int i = 0; i < currentValue.size(); ++i) {
+                    minVector.values[i] = Math.min(minVector.values[i], currentValue.values[i]);

Review comment:
       As I Know, blas has no function like minVector or maxVector.
   If user blas1 function(minVector is like blas1 function), loop may not be avoided, too.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829674990



##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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(trainData);
+        trainDataTable = tEnv.fromDataStream(dataStream, schema).as("features");
+        DataStream<Row> predDataStream = env.fromCollection(predictRows);
+        predictDataTable = tEnv.fromDataStream(predDataStream, schema).as("features");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol, DenseVector expected)
+            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(1, result.size());
+        assertEquals(expected, 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());

Review comment:
       OK




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] zhipeng93 commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r831697526



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerParams.java
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.ml.common.param.HasFeaturesCol;
+import org.apache.flink.ml.common.param.HasPredictionCol;
+import org.apache.flink.ml.param.DoubleParam;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+
+/**
+ * Params for {@link MinMaxScaler}.
+ *
+ * @param <T> The class type of this instance.
+ */
+public interface MinMaxScalerParams<T> extends HasFeaturesCol<T>, HasPredictionCol<T> {
+    Param<Double> MAX =
+            new DoubleParam(
+                    "max", "Upper bound after transformation.", 1.0, ParamValidators.notNull());
+
+    default Double getMax() {
+        return get(MAX);
+    }
+
+    default T setMax(Double value) {
+        return set(MAX, value);
+    }
+
+    Param<Double> MIN =

Review comment:
       OK




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
lindong28 commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r831033180



##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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 =

Review comment:
       Hmm... is this issue fixed?




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r831812941



##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.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.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.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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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, 400))));
+    private static final List<Row> predictData =
+            new ArrayList<>(
+                    Arrays.asList(
+                            Row.of(Vectors.dense(150.0, 90.0)),
+                            Row.of(Vectors.dense(50.0, 40.0)),
+                            Row.of(Vectors.dense(100.0, 50.0))));
+
+    private static final List<DenseVector> expectedData =
+            new ArrayList<>(
+                    Arrays.asList(
+                            Vectors.dense(0.25, 0.1),
+                            Vectors.dense(0.5, 0.125),
+                            Vectors.dense(0.75, 0.225)));
+
+    @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);
+        trainDataTable = tEnv.fromDataStream(env.fromCollection(trainData)).as("features");
+        predictDataTable = tEnv.fromDataStream(env.fromCollection(predictData)).as("features");
+    }
+
+    private static void verifyPredictionResult(
+            Table output, String outputCol, List<DenseVector> expected) 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());
+
+        result.sort(
+                (first, second) -> {
+                    for (int x = 0; x < first.size(); x++) {
+                        int cmp = Double.compare(first.get(x), second.get(x));
+                        if (cmp != 0) {
+                            return cmp;
+                        }
+                    }
+                    return 0;
+                });
+        assertEquals(expected.size(), result.size());

Review comment:
       I agree with you, I will remove it.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r831819041



##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.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.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.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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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, 400))));
+    private static final List<Row> predictData =
+            new ArrayList<>(
+                    Arrays.asList(
+                            Row.of(Vectors.dense(150.0, 90.0)),
+                            Row.of(Vectors.dense(50.0, 40.0)),
+                            Row.of(Vectors.dense(100.0, 50.0))));
+
+    private static final List<DenseVector> expectedData =
+            new ArrayList<>(
+                    Arrays.asList(
+                            Vectors.dense(0.25, 0.1),
+                            Vectors.dense(0.5, 0.125),
+                            Vectors.dense(0.75, 0.225)));
+
+    @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);
+        trainDataTable = tEnv.fromDataStream(env.fromCollection(trainData)).as("features");
+        predictDataTable = tEnv.fromDataStream(env.fromCollection(predictData)).as("features");
+    }
+
+    private static void verifyPredictionResult(
+            Table output, String outputCol, List<DenseVector> expected) 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());
+
+        result.sort(
+                (first, second) -> {

Review comment:
       OK, I will add the javadoc.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] zhipeng93 commented on pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
zhipeng93 commented on pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#issuecomment-1070600442


   Thanks for the update. This PR looks pretty good to me. 
   
   @lindong28 @yunfengzhou-hub Do you want to take a look at this PR?


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
lindong28 commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r831062877



##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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, 400))));
+    private static final List<Row> predictData =
+            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);
+        trainDataTable = tEnv.fromDataStream(env.fromCollection(trainData)).as("features");
+        predictDataTable = tEnv.fromDataStream(env.fromCollection(predictData)).as("features");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol, DenseVector expected)
+            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(1, result.size());
+        assertEquals(expected, result.get(0));
+    }
+
+    @Test
+    public void testParam() {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        assertEquals("features", minMaxScaler.getFeaturesCol());
+        assertEquals("prediction", minMaxScaler.getPredictionCol());
+        assertEquals(1.0, minMaxScaler.getMax(), 0.0001);
+        assertEquals(0.0, minMaxScaler.getMin(), 0.0001);
+        minMaxScaler
+                .setFeaturesCol("test_features")
+                .setPredictionCol("test_output")
+                .setMax(4.0)
+                .setMin(1.0);
+        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()
+                        .setFeaturesCol("test_features")
+                        .setPredictionCol("test_output")
+                        .setMin(1.0)
+                        .setMax(4.0);
+
+        MinMaxScalerModel model = minMaxScaler.fit(trainDataTable.as("test_features"));
+        Table output = model.transform(predictDataTable.as("test_features"))[0];
+        assertEquals(
+                Arrays.asList("test_features", "test_output"),
+                output.getResolvedSchema().getColumnNames());
+    }
+
+    @Test
+    public void testFewerSamplesThanParallel() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScalerModel model = minMaxScaler.fit(predictDataTable);
+        Table result = model.transform(predictDataTable)[0];
+        verifyPredictionResult(result, minMaxScaler.getPredictionCol(), Vectors.dense(0.5, 0.5));
+    }
+
+    @Test
+    public void testMaxValueEqualsMinValueButPredictValueNotEquals() throws Exception {
+        List<Row> userDefineData =
+                new ArrayList<>(Collections.singletonList(Row.of(Vectors.dense(30.0, 50.0))));
+        Table userDefineDataTable =
+                tEnv.fromDataStream(env.fromCollection(userDefineData)).as("features");
+        MinMaxScaler minMaxScaler = new MinMaxScaler().setMax(10.0).setMin(0.0);
+        MinMaxScalerModel model = minMaxScaler.fit(predictDataTable);
+        Table result = model.transform(userDefineDataTable)[0];
+        verifyPredictionResult(result, minMaxScaler.getPredictionCol(), Vectors.dense(5.0, 5.0));
+    }
+
+    @Test
+    public void testFitAndPredict() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScalerModel minMaxScalerModel = minMaxScaler.fit(trainDataTable);
+        Table output = minMaxScalerModel.transform(predictDataTable)[0];
+        verifyPredictionResult(output, minMaxScaler.getPredictionCol(), Vectors.dense(0.75, 0.225));
+    }
+
+    @Test
+    public void testSaveLoadAndPredict() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScaler loadedMinMaxScaler =
+                StageTestUtils.saveAndReload(
+                        env, minMaxScaler, tempFolder.newFolder().getAbsolutePath());
+        MinMaxScalerModel minMaxScalerModel = loadedMinMaxScaler.fit(trainDataTable);
+        minMaxScalerModel =
+                StageTestUtils.saveAndReload(
+                        env, minMaxScalerModel, tempFolder.newFolder().getAbsolutePath());
+        assertEquals(
+                Arrays.asList("minVector", "maxVector"),
+                minMaxScalerModel.getModelData()[0].getResolvedSchema().getColumnNames());
+        Table output = minMaxScalerModel.transform(predictDataTable)[0];
+        verifyPredictionResult(output, minMaxScaler.getPredictionCol(), Vectors.dense(0.75, 0.225));
+    }
+
+    @Test
+    public void testGetModelData() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScalerModel minMaxScalerModel = minMaxScaler.fit(trainDataTable);
+        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 =

Review comment:
       Hmm... it seems redundant to instantiate a `MinMaxScalerModelData` with the expected min/max vectors and then invoke assertEquals on these two class member variables.
   
   Would it be simpler to do the following directly?
   
   ```
   assertEquals(new DenseVector(new double[] {0.0, 0.0}), modelRows.get(0).getField(0));
   assertEquals(new DenseVector(new double[] {200.0, 400.0}), modelRows.get(0).getField(1));
   ```




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler

Posted by GitBox <gi...@apache.org>.
lindong28 commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r831805455



##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.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.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.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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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, 400))));
+    private static final List<Row> predictData =
+            new ArrayList<>(
+                    Arrays.asList(
+                            Row.of(Vectors.dense(150.0, 90.0)),
+                            Row.of(Vectors.dense(50.0, 40.0)),
+                            Row.of(Vectors.dense(100.0, 50.0))));
+
+    private static final List<DenseVector> expectedData =
+            new ArrayList<>(
+                    Arrays.asList(
+                            Vectors.dense(0.25, 0.1),
+                            Vectors.dense(0.5, 0.125),
+                            Vectors.dense(0.75, 0.225)));
+
+    @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);
+        trainDataTable = tEnv.fromDataStream(env.fromCollection(trainData)).as("features");
+        predictDataTable = tEnv.fromDataStream(env.fromCollection(predictData)).as("features");
+    }
+
+    private static void verifyPredictionResult(
+            Table output, String outputCol, List<DenseVector> expected) 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());
+
+        result.sort(
+                (first, second) -> {

Review comment:
       The Javadoc of `Comparator` says the following:
   
   ```
        * It is generally the case, but <i>not</i> strictly required that
        * <tt>(compare(x, y)==0) == (x.equals(y))</tt>.  Generally speaking,
        * any comparator that violates this condition should clearly indicate
        * this fact.  The recommended language is "Note: this comparator
        * imposes orderings that are inconsistent with equals."
   ```
   
   It might be better to take the vector size into consideration so that the comparator returns 0 iff `first == second`. The main benefit is that the code is self-contained and could be copied/used in other places.
   
   Alternatively we can just add the comment suggested by the comparator Javadoc.

##########
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;
+            this.min = min;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {

Review comment:
       @zhipeng93 @weibozhao Currently in all other model classes, we don't explicitly check whether the feature value is null. Should we follow the same practice here for consistency?
   
   We could alternatively choose to consistently check whether feature value is null, and throw exception if it is null. But even if we don't check this, we will already have NullPointerException with proper stacktrace, which should be sufficient for us to locate the problem (i.e. feature value is null). I personally think it is simpler not to check this. What do you think?
   
   
   

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.common.typeinfo.TypeInformation;
+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.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(),
+                                TypeInformation.of(DenseVector.class)),
+                        ArrayUtils.addAll(inputTypeInfo.getFieldNames(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private final String broadcastKey;
+        private final double upperBound;
+        private final double lowerBound;
+        private DenseVector scaleVector;
+        private DenseVector offsetVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (scaleVector == null) {
+                MinMaxScalerModelData minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                DenseVector minVector = minMaxScalerModelData.minVector;
+                DenseVector maxVector = minMaxScalerModelData.maxVector;
+                scaleVector = new DenseVector(minVector.size());
+                offsetVector = new DenseVector(minVector.size());
+                for (int i = 0; i < maxVector.size(); ++i) {
+                    if ((minVector.values[i] - maxVector.values[i]) == 0.0) {

Review comment:
       According to the discussion in [1][2], it is could lead to unexpected bug if we use `==` to check whether two double values are equal.
   
   How about we follow the recommendation in [3] and use the following code to check equality? The idea here is to explicitly specify an epsilon similar to what we did in tests.
   
   ```
   Math.abs(minVector.values[i] - maxVector.values[i]) < 1e-5
   ```
   
   [1] https://stackoverflow.com/questions/25160375/comparing-double-values-for-equality-in-java
   [2] https://publicobject.com/2009/11/floating-point-equality.html
   [3] https://www.baeldung.com/java-comparing-doubles

##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.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.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.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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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, 400))));
+    private static final List<Row> predictData =
+            new ArrayList<>(
+                    Arrays.asList(
+                            Row.of(Vectors.dense(150.0, 90.0)),
+                            Row.of(Vectors.dense(50.0, 40.0)),
+                            Row.of(Vectors.dense(100.0, 50.0))));
+
+    private static final List<DenseVector> expectedData =
+            new ArrayList<>(
+                    Arrays.asList(
+                            Vectors.dense(0.25, 0.1),
+                            Vectors.dense(0.5, 0.125),
+                            Vectors.dense(0.75, 0.225)));
+
+    @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);
+        trainDataTable = tEnv.fromDataStream(env.fromCollection(trainData)).as("features");
+        predictDataTable = tEnv.fromDataStream(env.fromCollection(predictData)).as("features");
+    }
+
+    private static void verifyPredictionResult(
+            Table output, String outputCol, List<DenseVector> expected) 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());
+
+        result.sort(
+                (first, second) -> {
+                    for (int x = 0; x < first.size(); x++) {
+                        int cmp = Double.compare(first.get(x), second.get(x));
+                        if (cmp != 0) {
+                            return cmp;
+                        }
+                    }
+                    return 0;
+                });
+        assertEquals(expected.size(), result.size());

Review comment:
       Do we still need this assertion given that we already have `assertEquals(expected, 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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r823414091



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.RichMapPartitionFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.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.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.Map;
+
+/** An Estimator which implements the MinMaxScaler algorithm. */

Review comment:
       OK




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r826542180



##########
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:
       OK




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r823423537



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.RichMapPartitionFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.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.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.Map;
+
+/** An Estimator which implements the MinMaxScaler algorithm. */
+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);
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment();
+        DataStream<Tuple2<DenseVector, DenseVector>> minMaxVectors =
+                computeMinMaxVectors(tEnv.toDataStream(inputs[0]), getFeaturesCol());
+        DataStream<MinMaxScalerModelData> modelData = genModelData(minMaxVectors);
+        MinMaxScalerModel model =
+                new MinMaxScalerModel().setModelData(tEnv.fromDataStream(modelData));
+        ReadWriteUtils.updateExistingParams(model, getParamMap());
+        return model;
+    }
+
+    @Override
+    public Map<Param<?>, Object> getParamMap() {
+        return paramMap;
+    }
+
+    @Override
+    public void save(String path) throws IOException {
+        ReadWriteUtils.saveMetadata(this, path);
+    }
+
+    public static MinMaxScaler load(StreamExecutionEnvironment env, String path)
+            throws IOException {
+        return ReadWriteUtils.loadStageParam(path);
+    }
+
+    /**
+     * Generates minMax scaler model data.
+     *
+     * @param minMaxVectors Input distributed minMaxVectors.
+     * @return MinMax scaler model data.
+     */
+    private static DataStream<MinMaxScalerModelData> genModelData(
+            DataStream<Tuple2<DenseVector, DenseVector>> minMaxVectors) {
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(

Review comment:
       OK




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r825551716



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerParams.java
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.ml.common.param.HasFeaturesCol;
+import org.apache.flink.ml.common.param.HasOutputCol;
+import org.apache.flink.ml.param.DoubleParam;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+
+/**
+ * Params for {@link MinMaxScaler}.
+ *
+ * @param <T> The class type of this instance.
+ */
+public interface MinMaxScalerParams<T> extends HasFeaturesCol<T>, HasOutputCol<T> {
+    Param<Double> MAX =
+            new DoubleParam(
+                    "max", "Upper bound after transformation.", 1.0, ParamValidators.notNull());
+
+    default Double getMax() {
+        return get(MAX);
+    }
+
+    default T setMax(Double value) {
+        return set(MAX, value);
+    }
+
+    Param<Double> MIN =
+            new DoubleParam(
+                    "min", "Lower bound after transformation.", 0.0, ParamValidators.notNull());
+
+    default Double getMIN() {
+        return get(MIN);
+    }
+
+    default T setMIN(Double value) {

Review comment:
       OK




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829025886



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {

Review comment:
       There is a judge about max-min==0 exists in the loop, then we can not use blas here. 
   If we want to use blas, we need another loop to prepare data. I think it is more expensive.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r828943448



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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> features =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                (MapFunction<Row, DenseVector>)
+                                        value -> (DenseVector) value.getField(featureCol));
+        DataStream<DenseVector> minMaxValues =
+                features.transform(
+                                "reduceInEachPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .transform(
+                                "reduceInFinalPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .setParallelism(1);
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(
+                        minMaxValues,
+                        new RichMapPartitionFunction<DenseVector, MinMaxScalerModelData>() {
+                            @Override
+                            public void mapPartition(
+                                    Iterable<DenseVector> values,
+                                    Collector<MinMaxScalerModelData> out) {
+                                Iterator<DenseVector> iter = values.iterator();
+                                DenseVector minVector = iter.next();
+                                DenseVector maxVector = iter.next();
+                                out.collect(new MinMaxScalerModelData(minVector, maxVector));
+                            }
+                        });
+
+        MinMaxScalerModel model =
+                new MinMaxScalerModel().setModelData(tEnv.fromDataStream(modelData));
+        ReadWriteUtils.updateExistingParams(model, getParamMap());
+        return model;
+    }
+
+    /**
+     * A stream operator to compute the min and max values in each partition of the input bounded
+     * data stream.
+     */
+    private static class MinMaxReduceFunctionOperator extends AbstractStreamOperator<DenseVector>
+            implements OneInputStreamOperator<DenseVector, DenseVector>, BoundedOneInput {
+        private ListState<DenseVector> minState;
+        private ListState<DenseVector> maxState;
+
+        private DenseVector minVector;
+        private DenseVector maxVector;
+
+        @Override
+        public void endInput() {
+            if (minVector != null) {
+                output.collect(new StreamRecord<>(minVector));
+            }
+            if (maxVector != null) {
+                output.collect(new StreamRecord<>(maxVector));
+            }
+        }
+
+        @Override
+        public void processElement(StreamRecord<DenseVector> streamRecord) {
+            DenseVector currentValue = streamRecord.getValue();
+            if (minVector == null) {
+                int vecSize = currentValue.size();
+                minVector = new DenseVector(vecSize);
+                maxVector = new DenseVector(vecSize);
+                System.arraycopy(currentValue.values, 0, minVector.values, 0, vecSize);
+                System.arraycopy(currentValue.values, 0, maxVector.values, 0, vecSize);
+
+            } else {
+                for (int i = 0; i < currentValue.size(); ++i) {
+                    minVector.values[i] = Math.min(minVector.values[i], currentValue.values[i]);

Review comment:
       Could you please help check whether there are methods in `BLAS` that could help to find the max/min value between two double arrays, instead of using a for loop? I think using for loop on vectors could be an expensive operation.
   
   If there is no direct method for this, maybe we can achieve this function by composing other BLAS operations. For example, the following methods would also help, if exists.
   - compares two double arrays and return an array containing 1/-1 values, showing which one has bigger value.
   - computes the absolute value of values in a double array.

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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> features =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                (MapFunction<Row, DenseVector>)
+                                        value -> (DenseVector) value.getField(featureCol));
+        DataStream<DenseVector> minMaxValues =
+                features.transform(
+                                "reduceInEachPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .transform(
+                                "reduceInFinalPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .setParallelism(1);
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(
+                        minMaxValues,
+                        new RichMapPartitionFunction<DenseVector, MinMaxScalerModelData>() {
+                            @Override
+                            public void mapPartition(
+                                    Iterable<DenseVector> values,
+                                    Collector<MinMaxScalerModelData> out) {
+                                Iterator<DenseVector> iter = values.iterator();
+                                DenseVector minVector = iter.next();
+                                DenseVector maxVector = iter.next();
+                                out.collect(new MinMaxScalerModelData(minVector, maxVector));
+                            }
+                        });
+
+        MinMaxScalerModel model =
+                new MinMaxScalerModel().setModelData(tEnv.fromDataStream(modelData));
+        ReadWriteUtils.updateExistingParams(model, getParamMap());
+        return model;
+    }
+
+    /**
+     * A stream operator to compute the min and max values in each partition of the input bounded
+     * data stream.
+     */
+    private static class MinMaxReduceFunctionOperator extends AbstractStreamOperator<DenseVector>
+            implements OneInputStreamOperator<DenseVector, DenseVector>, BoundedOneInput {
+        private ListState<DenseVector> minState;
+        private ListState<DenseVector> maxState;
+
+        private DenseVector minVector;
+        private DenseVector maxVector;
+
+        @Override
+        public void endInput() {
+            if (minVector != null) {
+                output.collect(new StreamRecord<>(minVector));
+            }
+            if (maxVector != null) {
+                output.collect(new StreamRecord<>(maxVector));
+            }
+        }
+
+        @Override
+        public void processElement(StreamRecord<DenseVector> streamRecord) {
+            DenseVector currentValue = streamRecord.getValue();
+            if (minVector == null) {
+                int vecSize = currentValue.size();
+                minVector = new DenseVector(vecSize);
+                maxVector = new DenseVector(vecSize);
+                System.arraycopy(currentValue.values, 0, minVector.values, 0, vecSize);
+                System.arraycopy(currentValue.values, 0, maxVector.values, 0, vecSize);
+
+            } else {
+                for (int i = 0; i < currentValue.size(); ++i) {
+                    minVector.values[i] = Math.min(minVector.values[i], currentValue.values[i]);
+                    maxVector.values[i] = Math.max(maxVector.values[i], currentValue.values[i]);
+                }
+            }
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            minState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "minState",
+                                            getOperatorConfig()
+                                                    .getTypeSerializerIn(
+                                                            0, getClass().getClassLoader())));
+            maxState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "maxState",
+                                            getOperatorConfig()
+                                                    .getTypeSerializerIn(
+                                                            0, getClass().getClassLoader())));
+            Iterator<DenseVector> minIterator = minState.get().iterator();
+            Iterator<DenseVector> maxIterator = maxState.get().iterator();
+            if (minIterator.hasNext()) {

Review comment:
       Have we checked the states or used `getUniqueElement` here? I saw zhipeng93's comment has been resolved here.

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {

Review comment:
       This for loop might also be replaced by BLAS operations.

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {

Review comment:
       Maybe we can make MinMaxScaler implements `HasHandleInvalid`, as the behavior to throw exception corresponds to `HasHandleInvalid.ERROR_INVALID`.
   
   Besides, if the feature vector exists, but its dimension is different from that of maxVector, or its max/min value exceeds maxVector/minVector's range, then maybe we should also throw exception.

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {
+                    if ((minVector.values[i] - maxVector.values[i]) != 0.0) {
+                        outputVector.values[i] =
+                                (feature.values[i] - minVector.values[i])
+                                                / (maxVector.values[i] - minVector.values[i])

Review comment:
       It seems that there is no direct usage of `maxVector`. Instead, we only use `maxVector.values[i] - minVector.values[i]`. So maybe we can just keep something like `spanVector` instead of `maxVector` in these operators and in model data.

##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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(trainData);
+        trainDataTable = tEnv.fromDataStream(dataStream, schema).as("features");
+        DataStream<Row> predDataStream = env.fromCollection(predictRows);
+        predictDataTable = tEnv.fromDataStream(predDataStream, schema).as("features");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol, DenseVector expected)
+            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(1, result.size());
+        assertEquals(expected, 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(trainDataTable.as("test_features"));
+        Table output = model.transform(predictDataTable.as("test_features"))[0];
+        assertEquals(
+                Arrays.asList("test_features", "test_output"),
+                output.getResolvedSchema().getColumnNames());
+    }
+
+    @Test
+    public void testFewerDistinctPointsThanCluster() throws Exception {

Review comment:
       Maybe we can refine this naming, as there is no concept of "cluster" in MinMaxScaler.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829014907



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {
+                    if ((minVector.values[i] - maxVector.values[i]) != 0.0) {
+                        outputVector.values[i] =
+                                (feature.values[i] - minVector.values[i])
+                                                / (maxVector.values[i] - minVector.values[i])

Review comment:
       I think spanVector maybe not very good. 
   1. In Spark and alink, maxVector and minVector are saved for modelData. 
   2. If we spanVector, code may be not clear as minVector and maxVector used. 




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829018999



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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> features =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                (MapFunction<Row, DenseVector>)
+                                        value -> (DenseVector) value.getField(featureCol));
+        DataStream<DenseVector> minMaxValues =
+                features.transform(
+                                "reduceInEachPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .transform(
+                                "reduceInFinalPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .setParallelism(1);
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(
+                        minMaxValues,
+                        new RichMapPartitionFunction<DenseVector, MinMaxScalerModelData>() {
+                            @Override
+                            public void mapPartition(
+                                    Iterable<DenseVector> values,
+                                    Collector<MinMaxScalerModelData> out) {
+                                Iterator<DenseVector> iter = values.iterator();
+                                DenseVector minVector = iter.next();
+                                DenseVector maxVector = iter.next();
+                                out.collect(new MinMaxScalerModelData(minVector, maxVector));
+                            }
+                        });
+
+        MinMaxScalerModel model =
+                new MinMaxScalerModel().setModelData(tEnv.fromDataStream(modelData));
+        ReadWriteUtils.updateExistingParams(model, getParamMap());
+        return model;
+    }
+
+    /**
+     * A stream operator to compute the min and max values in each partition of the input bounded
+     * data stream.
+     */
+    private static class MinMaxReduceFunctionOperator extends AbstractStreamOperator<DenseVector>
+            implements OneInputStreamOperator<DenseVector, DenseVector>, BoundedOneInput {
+        private ListState<DenseVector> minState;
+        private ListState<DenseVector> maxState;
+
+        private DenseVector minVector;
+        private DenseVector maxVector;
+
+        @Override
+        public void endInput() {
+            if (minVector != null) {
+                output.collect(new StreamRecord<>(minVector));
+            }
+            if (maxVector != null) {
+                output.collect(new StreamRecord<>(maxVector));
+            }
+        }
+
+        @Override
+        public void processElement(StreamRecord<DenseVector> streamRecord) {
+            DenseVector currentValue = streamRecord.getValue();
+            if (minVector == null) {
+                int vecSize = currentValue.size();
+                minVector = new DenseVector(vecSize);
+                maxVector = new DenseVector(vecSize);
+                System.arraycopy(currentValue.values, 0, minVector.values, 0, vecSize);
+                System.arraycopy(currentValue.values, 0, maxVector.values, 0, vecSize);
+
+            } else {
+                for (int i = 0; i < currentValue.size(); ++i) {
+                    minVector.values[i] = Math.min(minVector.values[i], currentValue.values[i]);

Review comment:
       As I Know, blas has no function like minVector or maxVector.
   If user blas1 function(minVector is blas1 function), loop may not be avoided, too.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829643000



##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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(trainData);
+        trainDataTable = tEnv.fromDataStream(dataStream, schema).as("features");
+        DataStream<Row> predDataStream = env.fromCollection(predictRows);
+        predictDataTable = tEnv.fromDataStream(predDataStream, schema).as("features");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol, DenseVector expected)
+            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(1, result.size());
+        assertEquals(expected, 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(trainDataTable.as("test_features"));
+        Table output = model.transform(predictDataTable.as("test_features"))[0];
+        assertEquals(
+                Arrays.asList("test_features", "test_output"),
+                output.getResolvedSchema().getColumnNames());
+    }
+
+    @Test
+    public void testFewerDistinctPointsThanCluster() throws Exception {

Review comment:
       KMeans is a clustering algorithm, so I understand that there is the concept of cluster. But in MinMaxScaler there is no such "cluster" thing. I think it would be more proper if we rename it to `testMinMaxEquals()`.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r823413985



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.RichMapPartitionFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.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.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.Map;
+
+/** An Estimator which implements the MinMaxScaler algorithm. */
+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);
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment();
+        DataStream<Tuple2<DenseVector, DenseVector>> minMaxVectors =
+                computeMinMaxVectors(tEnv.toDataStream(inputs[0]), getFeaturesCol());
+        DataStream<MinMaxScalerModelData> modelData = genModelData(minMaxVectors);
+        MinMaxScalerModel model =
+                new MinMaxScalerModel().setModelData(tEnv.fromDataStream(modelData));
+        ReadWriteUtils.updateExistingParams(model, getParamMap());
+        return model;
+    }
+
+    @Override
+    public Map<Param<?>, Object> getParamMap() {
+        return paramMap;
+    }
+
+    @Override
+    public void save(String path) throws IOException {
+        ReadWriteUtils.saveMetadata(this, path);
+    }
+
+    public static MinMaxScaler load(StreamExecutionEnvironment env, String path)
+            throws IOException {
+        return ReadWriteUtils.loadStageParam(path);
+    }
+
+    /**
+     * Generates minMax scaler model data.
+     *
+     * @param minMaxVectors Input distributed minMaxVectors.
+     * @return MinMax scaler model data.
+     */
+    private static DataStream<MinMaxScalerModelData> genModelData(
+            DataStream<Tuple2<DenseVector, DenseVector>> minMaxVectors) {
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(
+                        minMaxVectors,
+                        new RichMapPartitionFunction<
+                                Tuple2<DenseVector, DenseVector>, MinMaxScalerModelData>() {
+                            @Override
+                            public void mapPartition(
+                                    Iterable<Tuple2<DenseVector, DenseVector>> dataPoints,
+                                    Collector<MinMaxScalerModelData> out) {
+                                DenseVector minVector = null;
+                                DenseVector maxVector = null;
+                                int vecSize = 0;
+                                for (Tuple2<DenseVector, DenseVector> dataPoint : dataPoints) {
+                                    if (maxVector == null) {
+                                        vecSize = dataPoint.f0.size();
+                                        maxVector = dataPoint.f1;
+                                        minVector = dataPoint.f0;
+                                    }
+                                    for (int i = 0; i < vecSize; ++i) {

Review comment:
       ok




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r823425316



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerParams.java
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.ml.common.param.HasFeaturesCol;
+import org.apache.flink.ml.common.param.HasOutputCol;
+import org.apache.flink.ml.param.DoubleParam;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+
+/**
+ * Params for {@link MinMaxScaler}.
+ *
+ * @param <T> The class type of this instance.
+ */
+public interface MinMaxScalerParams<T> extends HasFeaturesCol<T>, HasOutputCol<T> {
+    Param<Double> MAX =
+            new DoubleParam(
+                    "max", "Upper bound after transformation.", 1.0, ParamValidators.notNull());
+
+    default Double getMax() {
+        return get(MAX);
+    }
+
+    default T setMax(Double value) {
+        return set(MAX, value);
+    }
+
+    Param<Double> MIN =
+            new DoubleParam(
+                    "min", "Lower bound after transformation.", 0.0, ParamValidators.notNull());
+
+    default Double getMIN() {
+        return get(MIN);
+    }
+
+    default T setMIN(Double value) {

Review comment:
       ditto.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829680935



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {

Review comment:
       When maxVector[i] == minVector[i],
   If featureValue[i] != maxVector[i], above code may not get the same result as original code.
   In real world, data for train and predict may be different, featureValue[i] != maxVector[i] is existed. 




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829908054



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {
+                    if ((minVector.values[i] - maxVector.values[i]) != 0.0) {
+                        outputVector.values[i] =
+                                (feature.values[i] - minVector.values[i])
+                                                / (maxVector.values[i] - minVector.values[i])

Review comment:
       Sounds good.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r826538436



##########
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:
       testFewerDistinctPointsThanCluster()  test this condition. Parallel is 4 and only 1 record trained.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] zhipeng93 commented on pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler

Posted by GitBox <gi...@apache.org>.
zhipeng93 commented on pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#issuecomment-1074967174


   Thanks @weibozhao for the PR and @lindong28 @yunfengzhou-hub for the review. Merging now.


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r831696152



##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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, 400))));
+    private static final List<Row> predictData =
+            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);
+        trainDataTable = tEnv.fromDataStream(env.fromCollection(trainData)).as("features");
+        predictDataTable = tEnv.fromDataStream(env.fromCollection(predictData)).as("features");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol, DenseVector expected)
+            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(1, result.size());
+        assertEquals(expected, result.get(0));
+    }
+
+    @Test
+    public void testParam() {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        assertEquals("features", minMaxScaler.getFeaturesCol());
+        assertEquals("prediction", minMaxScaler.getPredictionCol());
+        assertEquals(1.0, minMaxScaler.getMax(), 0.0001);
+        assertEquals(0.0, minMaxScaler.getMin(), 0.0001);
+        minMaxScaler
+                .setFeaturesCol("test_features")
+                .setPredictionCol("test_output")
+                .setMax(4.0)
+                .setMin(1.0);
+        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()
+                        .setFeaturesCol("test_features")
+                        .setPredictionCol("test_output")
+                        .setMin(1.0)
+                        .setMax(4.0);
+
+        MinMaxScalerModel model = minMaxScaler.fit(trainDataTable.as("test_features"));
+        Table output = model.transform(predictDataTable.as("test_features"))[0];
+        assertEquals(
+                Arrays.asList("test_features", "test_output"),
+                output.getResolvedSchema().getColumnNames());
+    }
+
+    @Test
+    public void testFewerSamplesThanParallel() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScalerModel model = minMaxScaler.fit(predictDataTable);
+        Table result = model.transform(predictDataTable)[0];
+        verifyPredictionResult(result, minMaxScaler.getPredictionCol(), Vectors.dense(0.5, 0.5));
+    }
+
+    @Test
+    public void testMaxValueEqualsMinValueButPredictValueNotEquals() throws Exception {
+        List<Row> userDefineData =
+                new ArrayList<>(Collections.singletonList(Row.of(Vectors.dense(30.0, 50.0))));
+        Table userDefineDataTable =
+                tEnv.fromDataStream(env.fromCollection(userDefineData)).as("features");
+        MinMaxScaler minMaxScaler = new MinMaxScaler().setMax(10.0).setMin(0.0);
+        MinMaxScalerModel model = minMaxScaler.fit(predictDataTable);
+        Table result = model.transform(userDefineDataTable)[0];
+        verifyPredictionResult(result, minMaxScaler.getPredictionCol(), Vectors.dense(5.0, 5.0));
+    }
+
+    @Test
+    public void testFitAndPredict() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScalerModel minMaxScalerModel = minMaxScaler.fit(trainDataTable);
+        Table output = minMaxScalerModel.transform(predictDataTable)[0];
+        verifyPredictionResult(output, minMaxScaler.getPredictionCol(), Vectors.dense(0.75, 0.225));
+    }
+
+    @Test
+    public void testSaveLoadAndPredict() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScaler loadedMinMaxScaler =
+                StageTestUtils.saveAndReload(
+                        env, minMaxScaler, tempFolder.newFolder().getAbsolutePath());
+        MinMaxScalerModel minMaxScalerModel = loadedMinMaxScaler.fit(trainDataTable);
+        minMaxScalerModel =
+                StageTestUtils.saveAndReload(
+                        env, minMaxScalerModel, tempFolder.newFolder().getAbsolutePath());
+        assertEquals(
+                Arrays.asList("minVector", "maxVector"),
+                minMaxScalerModel.getModelData()[0].getResolvedSchema().getColumnNames());
+        Table output = minMaxScalerModel.transform(predictDataTable)[0];
+        verifyPredictionResult(output, minMaxScaler.getPredictionCol(), Vectors.dense(0.75, 0.225));
+    }
+
+    @Test
+    public void testGetModelData() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScalerModel minMaxScalerModel = minMaxScaler.fit(trainDataTable);
+        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 =

Review comment:
       OK




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 edited a comment on pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
lindong28 edited a comment on pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#issuecomment-1017123834


   Thanks for the PR.
   
   Before we continue to add more and more algorithm, what's our plan to make sure these algorithm's implementation could meet our performance target? I am concerned that if we add more algorithms before we have a benchmark plan, we might end up refactoring added algorithms in a unnecessary manner, which is probably not a good long term solution.


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r826540199



##########
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:
       OK




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829632681



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {

Review comment:
       Got it. If this is the case, then I think MinMaxScaler might not need this parameter either.
   
   What do you think of the other part of the question above?
   > if the feature vector exists, but its dimension is different from that of maxVector, or its max/min value exceeds maxVector/minVector's range, then maybe we should also throw exception.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
lindong28 commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829628875



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.iteration.operator.OperatorStateUtils;
+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> features =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                (MapFunction<Row, DenseVector>)
+                                        value -> (DenseVector) value.getField(featureCol));
+        DataStream<DenseVector> minMaxValues =
+                features.transform(
+                                "reduceInEachPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .transform(
+                                "reduceInFinalPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .setParallelism(1);
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(
+                        minMaxValues,
+                        new RichMapPartitionFunction<DenseVector, MinMaxScalerModelData>() {
+                            @Override
+                            public void mapPartition(
+                                    Iterable<DenseVector> values,
+                                    Collector<MinMaxScalerModelData> out) {
+                                Iterator<DenseVector> iter = values.iterator();
+                                DenseVector minVector = iter.next();
+                                DenseVector maxVector = iter.next();
+                                out.collect(new MinMaxScalerModelData(minVector, maxVector));
+                            }
+                        });
+
+        MinMaxScalerModel model =
+                new MinMaxScalerModel().setModelData(tEnv.fromDataStream(modelData));
+        ReadWriteUtils.updateExistingParams(model, getParamMap());
+        return model;
+    }
+
+    /**
+     * A stream operator to compute the min and max values in each partition of the input bounded
+     * data stream.
+     */
+    private static class MinMaxReduceFunctionOperator extends AbstractStreamOperator<DenseVector>
+            implements OneInputStreamOperator<DenseVector, DenseVector>, BoundedOneInput {
+        private ListState<DenseVector> minState;
+        private ListState<DenseVector> maxState;
+
+        private DenseVector minVector;
+        private DenseVector maxVector;
+
+        @Override
+        public void endInput() {
+            if (minVector != null) {
+                output.collect(new StreamRecord<>(minVector));
+            }
+            if (maxVector != null) {
+                output.collect(new StreamRecord<>(maxVector));
+            }
+        }
+
+        @Override
+        public void processElement(StreamRecord<DenseVector> streamRecord) {
+            DenseVector currentValue = streamRecord.getValue();
+            if (minVector == null) {
+                int vecSize = currentValue.size();
+                minVector = new DenseVector(vecSize);
+                maxVector = new DenseVector(vecSize);
+                System.arraycopy(currentValue.values, 0, minVector.values, 0, vecSize);
+                System.arraycopy(currentValue.values, 0, maxVector.values, 0, vecSize);
+
+            } else {
+                for (int i = 0; i < currentValue.size(); ++i) {

Review comment:
       Would it be useful to check that `currentValue.size() = maxVector.size()` here? 
   
   Maybe follow NaiveBayes.java line 256 for example.

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.iteration.operator.OperatorStateUtils;
+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> features =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                (MapFunction<Row, DenseVector>)
+                                        value -> (DenseVector) value.getField(featureCol));
+        DataStream<DenseVector> minMaxValues =
+                features.transform(
+                                "reduceInEachPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .transform(
+                                "reduceInFinalPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .setParallelism(1);
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(
+                        minMaxValues,
+                        new RichMapPartitionFunction<DenseVector, MinMaxScalerModelData>() {
+                            @Override
+                            public void mapPartition(
+                                    Iterable<DenseVector> values,
+                                    Collector<MinMaxScalerModelData> out) {
+                                Iterator<DenseVector> iter = values.iterator();
+                                DenseVector minVector = iter.next();
+                                DenseVector maxVector = iter.next();
+                                out.collect(new MinMaxScalerModelData(minVector, maxVector));
+                            }
+                        });
+
+        MinMaxScalerModel model =
+                new MinMaxScalerModel().setModelData(tEnv.fromDataStream(modelData));
+        ReadWriteUtils.updateExistingParams(model, getParamMap());
+        return model;
+    }
+
+    /**
+     * A stream operator to compute the min and max values in each partition of the input bounded
+     * data stream.
+     */
+    private static class MinMaxReduceFunctionOperator extends AbstractStreamOperator<DenseVector>
+            implements OneInputStreamOperator<DenseVector, DenseVector>, BoundedOneInput {
+        private ListState<DenseVector> minState;
+        private ListState<DenseVector> maxState;
+
+        private DenseVector minVector;
+        private DenseVector maxVector;
+
+        @Override
+        public void endInput() {
+            if (minVector != null) {

Review comment:
       Would it be simpler to just do the following:
   ```
               if (minVector != null) {
                   output.collect(new StreamRecord<>(minVector));
                   output.collect(new StreamRecord<>(maxVector));
               }
   ```

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.iteration.operator.OperatorStateUtils;
+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).

Review comment:
       Since the link provided here does not explain the use of min/max parameter, it does not exactly describe what this class does.
   
   Should we provide more detailed algorithm explanation here similar to Spark's MinMaxScaler Javadoc?

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());

Review comment:
       nits: would it be simpler to rename `outputVector` as `output` for simplicity and consistency with `feature`?




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829706772



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {

Review comment:
       OK, I have add exception.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
lindong28 commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829628875



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.iteration.operator.OperatorStateUtils;
+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> features =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                (MapFunction<Row, DenseVector>)
+                                        value -> (DenseVector) value.getField(featureCol));
+        DataStream<DenseVector> minMaxValues =
+                features.transform(
+                                "reduceInEachPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .transform(
+                                "reduceInFinalPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .setParallelism(1);
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(
+                        minMaxValues,
+                        new RichMapPartitionFunction<DenseVector, MinMaxScalerModelData>() {
+                            @Override
+                            public void mapPartition(
+                                    Iterable<DenseVector> values,
+                                    Collector<MinMaxScalerModelData> out) {
+                                Iterator<DenseVector> iter = values.iterator();
+                                DenseVector minVector = iter.next();
+                                DenseVector maxVector = iter.next();
+                                out.collect(new MinMaxScalerModelData(minVector, maxVector));
+                            }
+                        });
+
+        MinMaxScalerModel model =
+                new MinMaxScalerModel().setModelData(tEnv.fromDataStream(modelData));
+        ReadWriteUtils.updateExistingParams(model, getParamMap());
+        return model;
+    }
+
+    /**
+     * A stream operator to compute the min and max values in each partition of the input bounded
+     * data stream.
+     */
+    private static class MinMaxReduceFunctionOperator extends AbstractStreamOperator<DenseVector>
+            implements OneInputStreamOperator<DenseVector, DenseVector>, BoundedOneInput {
+        private ListState<DenseVector> minState;
+        private ListState<DenseVector> maxState;
+
+        private DenseVector minVector;
+        private DenseVector maxVector;
+
+        @Override
+        public void endInput() {
+            if (minVector != null) {
+                output.collect(new StreamRecord<>(minVector));
+            }
+            if (maxVector != null) {
+                output.collect(new StreamRecord<>(maxVector));
+            }
+        }
+
+        @Override
+        public void processElement(StreamRecord<DenseVector> streamRecord) {
+            DenseVector currentValue = streamRecord.getValue();
+            if (minVector == null) {
+                int vecSize = currentValue.size();
+                minVector = new DenseVector(vecSize);
+                maxVector = new DenseVector(vecSize);
+                System.arraycopy(currentValue.values, 0, minVector.values, 0, vecSize);
+                System.arraycopy(currentValue.values, 0, maxVector.values, 0, vecSize);
+
+            } else {
+                for (int i = 0; i < currentValue.size(); ++i) {

Review comment:
       Would it be useful to check that `currentValue.size() = maxVector.size()` here? 
   
   Maybe follow NaiveBayes.java line 256 for example.

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.iteration.operator.OperatorStateUtils;
+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> features =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                (MapFunction<Row, DenseVector>)
+                                        value -> (DenseVector) value.getField(featureCol));
+        DataStream<DenseVector> minMaxValues =
+                features.transform(
+                                "reduceInEachPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .transform(
+                                "reduceInFinalPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .setParallelism(1);
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(
+                        minMaxValues,
+                        new RichMapPartitionFunction<DenseVector, MinMaxScalerModelData>() {
+                            @Override
+                            public void mapPartition(
+                                    Iterable<DenseVector> values,
+                                    Collector<MinMaxScalerModelData> out) {
+                                Iterator<DenseVector> iter = values.iterator();
+                                DenseVector minVector = iter.next();
+                                DenseVector maxVector = iter.next();
+                                out.collect(new MinMaxScalerModelData(minVector, maxVector));
+                            }
+                        });
+
+        MinMaxScalerModel model =
+                new MinMaxScalerModel().setModelData(tEnv.fromDataStream(modelData));
+        ReadWriteUtils.updateExistingParams(model, getParamMap());
+        return model;
+    }
+
+    /**
+     * A stream operator to compute the min and max values in each partition of the input bounded
+     * data stream.
+     */
+    private static class MinMaxReduceFunctionOperator extends AbstractStreamOperator<DenseVector>
+            implements OneInputStreamOperator<DenseVector, DenseVector>, BoundedOneInput {
+        private ListState<DenseVector> minState;
+        private ListState<DenseVector> maxState;
+
+        private DenseVector minVector;
+        private DenseVector maxVector;
+
+        @Override
+        public void endInput() {
+            if (minVector != null) {

Review comment:
       Would it be simpler to just do the following:
   ```
               if (minVector != null) {
                   output.collect(new StreamRecord<>(minVector));
                   output.collect(new StreamRecord<>(maxVector));
               }
   ```

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.iteration.operator.OperatorStateUtils;
+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).

Review comment:
       Since the link provided here does not explain the use of min/max parameter, it does not exactly describe what this class does.
   
   Should we provide more detailed algorithm explanation here similar to Spark's MinMaxScaler Javadoc?

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());

Review comment:
       nits: would it be simpler to rename `outputVector` as `output` for simplicity and consistency with `feature`?

##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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(trainData);
+        trainDataTable = tEnv.fromDataStream(dataStream, schema).as("features");
+        DataStream<Row> predDataStream = env.fromCollection(predictRows);
+        predictDataTable = tEnv.fromDataStream(predDataStream, schema).as("features");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol, DenseVector expected)
+            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(1, result.size());
+        assertEquals(expected, 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(trainDataTable.as("test_features"));
+        Table output = model.transform(predictDataTable.as("test_features"))[0];
+        assertEquals(
+                Arrays.asList("test_features", "test_output"),
+                output.getResolvedSchema().getColumnNames());
+    }
+
+    @Test
+    public void testFewerDistinctPointsThanCluster() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScalerModel model = minMaxScaler.fit(predictDataTable);
+        Table result = model.transform(predictDataTable)[0];
+        verifyPredictionResult(result, minMaxScaler.getPredictionCol(), Vectors.dense(0.5, 0.5));
+    }
+
+    @Test
+    public void testFitAndPredict() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScalerModel minMaxScalerModel = minMaxScaler.fit(trainDataTable);
+        Table output = minMaxScalerModel.transform(predictDataTable)[0];
+        verifyPredictionResult(output, minMaxScaler.getPredictionCol(), Vectors.dense(0.75, 0.3));
+    }
+
+    @Test
+    public void testSaveLoadAndPredict() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScaler loadedMinMaxScaler =
+                StageTestUtils.saveAndReload(
+                        env, minMaxScaler, tempFolder.newFolder().getAbsolutePath());
+        MinMaxScalerModel minMaxScalerModel = loadedMinMaxScaler.fit(trainDataTable);
+        minMaxScalerModel =
+                StageTestUtils.saveAndReload(
+                        env, minMaxScalerModel, tempFolder.newFolder().getAbsolutePath());
+        assertEquals(
+                Arrays.asList("minVector", "maxVector"),
+                minMaxScalerModel.getModelData()[0].getResolvedSchema().getColumnNames());
+        Table output = minMaxScalerModel.transform(predictDataTable)[0];
+        verifyPredictionResult(output, minMaxScaler.getPredictionCol(), Vectors.dense(0.75, 0.3));
+    }
+
+    @Test
+    public void testModelSaveLoadAndPredict() throws Exception {

Review comment:
       It seems that all functionalities covered in this test is already covered by `testSaveLoadAndPredict`. Could we remove this test for simplicity?

##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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(trainData);
+        trainDataTable = tEnv.fromDataStream(dataStream, schema).as("features");
+        DataStream<Row> predDataStream = env.fromCollection(predictRows);
+        predictDataTable = tEnv.fromDataStream(predDataStream, schema).as("features");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol, DenseVector expected)
+            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(1, result.size());
+        assertEquals(expected, 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());

Review comment:
       nits: could we move `prediction` to be right after "features" for a bit more consistency, i.e. column name features are close to each other?
   
   Same for other `assertEquals` usages and `testFeaturePredictionParam()`.

##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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(trainData);
+        trainDataTable = tEnv.fromDataStream(dataStream, schema).as("features");
+        DataStream<Row> predDataStream = env.fromCollection(predictRows);

Review comment:
       It appears that we don't need to explicitly create the schema. Would it be simpler to use the code below?
   
   ```
           trainDataTable = tEnv.fromDataStream(env.fromCollection(trainData)).as("features");
           predictDataTable = tEnv.fromDataStream(env.fromCollection(predictRows)).as("features");
   ```

##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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 =

Review comment:
       Could we have multiple elements instead of just one element in the prediction stream? In general this could help provide more test coverage.
   
   And could we rename `predictRows` as `predictData` for consistency with `trainData`?




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
lindong28 commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r831080421



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModelData.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.serialization.Encoder;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.src.reader.SimpleStreamFormat;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.linalg.typeinfo.DenseVectorSerializer;
+import org.apache.flink.streaming.api.datastream.DataStream;
+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 java.io.EOFException;
+import java.io.IOException;
+import java.io.OutputStream;
+
+/**
+ * Model data of {@link MinMaxScalerModel}.
+ *
+ * <p>This class also provides methods to convert model data from Table to a data stream, and
+ * classes to save/load model data.
+ */
+public class MinMaxScalerModelData {
+    public DenseVector minVector;
+
+    public DenseVector maxVector;
+
+    public MinMaxScalerModelData() {}
+
+    public MinMaxScalerModelData(DenseVector minVector, DenseVector maxVector) {
+        this.minVector = minVector;
+        this.maxVector = maxVector;
+    }
+
+    /**
+     * Converts the table model to a data stream.
+     *
+     * @param modelDataTable The table model data.
+     * @return The data stream model data.
+     */
+    public static DataStream<MinMaxScalerModelData> getModelDataStream(Table modelDataTable) {
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) modelDataTable).getTableEnvironment();
+        return tEnv.toDataStream(modelDataTable)
+                .map(
+                        x ->
+                                new MinMaxScalerModelData(
+                                        (DenseVector) x.getField(0), (DenseVector) x.getField(1)));
+    }
+
+    /** Encoder for {@link MinMaxScalerModelData}. */
+    public static class ModelDataEncoder implements Encoder<MinMaxScalerModelData> {
+        @Override
+        public void encode(MinMaxScalerModelData minMaxScalerModelData, OutputStream outputStream)

Review comment:
       Super nits: all other classes (except `KnnModelData`) uses `modelData` as the input parameter name of this method. Any chance we can do the same here for consistency and shorten the code a little bit?
   
   And maybe update KnnModelData as well?

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModelData.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.serialization.Encoder;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.src.reader.SimpleStreamFormat;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.linalg.typeinfo.DenseVectorSerializer;
+import org.apache.flink.streaming.api.datastream.DataStream;
+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 java.io.EOFException;
+import java.io.IOException;
+import java.io.OutputStream;
+
+/**
+ * Model data of {@link MinMaxScalerModel}.
+ *
+ * <p>This class also provides methods to convert model data from Table to a data stream, and
+ * classes to save/load model data.
+ */
+public class MinMaxScalerModelData {
+    public DenseVector minVector;
+
+    public DenseVector maxVector;
+
+    public MinMaxScalerModelData() {}
+
+    public MinMaxScalerModelData(DenseVector minVector, DenseVector maxVector) {
+        this.minVector = minVector;
+        this.maxVector = maxVector;
+    }
+
+    /**
+     * Converts the table model to a data stream.
+     *
+     * @param modelDataTable The table model data.
+     * @return The data stream model data.
+     */
+    public static DataStream<MinMaxScalerModelData> getModelDataStream(Table modelDataTable) {
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) modelDataTable).getTableEnvironment();
+        return tEnv.toDataStream(modelDataTable)
+                .map(
+                        x ->
+                                new MinMaxScalerModelData(
+                                        (DenseVector) x.getField(0), (DenseVector) x.getField(1)));
+    }
+
+    /** Encoder for {@link MinMaxScalerModelData}. */
+    public static class ModelDataEncoder implements Encoder<MinMaxScalerModelData> {
+        @Override
+        public void encode(MinMaxScalerModelData minMaxScalerModelData, OutputStream outputStream)

Review comment:
       Super nits: all other classes (except `KnnModelData`) uses `modelData` as the input parameter name of this method. Any chance we can do the same here for consistency and shorten the code a little bit?
   
   And maybe update KnnModelData as well for consistency?




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 edited a comment on pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
lindong28 edited a comment on pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#issuecomment-1017123834


   Thanks for the PR.
   
   Before we continue to more and more algorithm, what's our plan to make sure these algorithm's implementation could meet our performance target? I am concerned that if we add more algorithms before we have a benchmark plan, we might end up refactoring added algorithms in a unnecessary manner, which is probably not a good long term solution.


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r823413341



##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,206 @@
+/*
+ * 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());
+        for (DenseVector t2 : result) {
+            assertEquals(Vectors.dense(0.75, 0.3), t2);
+        }
+    }
+
+    @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("output", minMaxScaler.getOutputCol());
+        minMaxScaler
+                .setFeaturesCol("test_features")
+                .setMax(4.0)
+                .setMIN(0.0)

Review comment:
       OK




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r823414741



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerParams.java
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.ml.common.param.HasFeaturesCol;
+import org.apache.flink.ml.common.param.HasOutputCol;
+import org.apache.flink.ml.param.DoubleParam;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+
+/**
+ * Params for {@link MinMaxScaler}.
+ *
+ * @param <T> The class type of this instance.
+ */
+public interface MinMaxScalerParams<T> extends HasFeaturesCol<T>, HasOutputCol<T> {
+    Param<Double> MAX =
+            new DoubleParam(
+                    "max", "Upper bound after transformation.", 1.0, ParamValidators.notNull());
+
+    default Double getMax() {
+        return get(MAX);
+    }
+
+    default T setMax(Double value) {
+        return set(MAX, value);
+    }
+
+    Param<Double> MIN =
+            new DoubleParam(
+                    "min", "Lower bound after transformation.", 0.0, ParamValidators.notNull());
+
+    default Double getMIN() {

Review comment:
       I define api as alink and spark.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r823412958



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/common/param/HasOutputCol.java
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.common.param;
+
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+import org.apache.flink.ml.param.StringParam;
+import org.apache.flink.ml.param.WithParams;
+
+/** Interface for the shared outputCol param. */
+public interface HasOutputCol<T> extends WithParams<T> {

Review comment:
       PredictionCol may be better for ml algo. For feature engineer and data proc, outputCol may be better.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r823402845



##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,206 @@
+/*
+ * 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());
+        for (DenseVector t2 : result) {
+            assertEquals(Vectors.dense(0.75, 0.3), t2);
+        }
+    }
+
+    @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("output", minMaxScaler.getOutputCol());
+        minMaxScaler
+                .setFeaturesCol("test_features")
+                .setMax(4.0)
+                .setMIN(0.0)
+                .setOutputCol("test_output");
+        assertEquals("test_features", minMaxScaler.getFeaturesCol());
+        assertEquals(0.0, minMaxScaler.getMIN(), 0.0001);
+        assertEquals(4.0, minMaxScaler.getMax(), 0.0001);
+        assertEquals("test_output", minMaxScaler.getOutputCol());
+    }
+
+    @Test
+    public void testFeaturePredictionParam() {
+        MinMaxScaler minMaxScaler =
+                new MinMaxScaler()
+                        .setMIN(0.0)
+                        .setMax(4.0)
+                        .setFeaturesCol("test_features")
+                        .setOutputCol("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() {

Review comment:
       This test checks when some worker has no data. I think it is needed. 




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r823401050



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.RichMapPartitionFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.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.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.Map;
+
+/** An Estimator which implements the MinMaxScaler algorithm. */
+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);
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment();
+        DataStream<Tuple2<DenseVector, DenseVector>> minMaxVectors =
+                computeMinMaxVectors(tEnv.toDataStream(inputs[0]), getFeaturesCol());
+        DataStream<MinMaxScalerModelData> modelData = genModelData(minMaxVectors);
+        MinMaxScalerModel model =
+                new MinMaxScalerModel().setModelData(tEnv.fromDataStream(modelData));
+        ReadWriteUtils.updateExistingParams(model, getParamMap());
+        return model;
+    }
+
+    @Override
+    public Map<Param<?>, Object> getParamMap() {
+        return paramMap;
+    }
+
+    @Override
+    public void save(String path) throws IOException {
+        ReadWriteUtils.saveMetadata(this, path);
+    }
+
+    public static MinMaxScaler load(StreamExecutionEnvironment env, String path)
+            throws IOException {
+        return ReadWriteUtils.loadStageParam(path);
+    }
+
+    /**
+     * Generates minMax scaler model data.
+     *
+     * @param minMaxVectors Input distributed minMaxVectors.
+     * @return MinMax scaler model data.
+     */
+    private static DataStream<MinMaxScalerModelData> genModelData(
+            DataStream<Tuple2<DenseVector, DenseVector>> minMaxVectors) {
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(
+                        minMaxVectors,
+                        new RichMapPartitionFunction<
+                                Tuple2<DenseVector, DenseVector>, MinMaxScalerModelData>() {
+                            @Override
+                            public void mapPartition(
+                                    Iterable<Tuple2<DenseVector, DenseVector>> dataPoints,
+                                    Collector<MinMaxScalerModelData> out) {
+                                DenseVector minVector = null;
+                                DenseVector maxVector = null;
+                                int vecSize = 0;
+                                for (Tuple2<DenseVector, DenseVector> dataPoint : dataPoints) {
+                                    if (maxVector == null) {
+                                        vecSize = dataPoint.f0.size();
+                                        maxVector = dataPoint.f1;
+                                        minVector = dataPoint.f0;
+                                    }
+                                    for (int i = 0; i < vecSize; ++i) {
+                                        minVector.values[i] =
+                                                Math.min(
+                                                        dataPoint.f0.values[i],
+                                                        minVector.values[i]);
+                                        maxVector.values[i] =
+                                                Math.max(
+                                                        dataPoint.f1.values[i],
+                                                        maxVector.values[i]);
+                                    }
+                                }
+                                out.collect(new MinMaxScalerModelData(minVector, maxVector));
+                            }
+                        });
+        modelData.getTransformation().setParallelism(1);
+        return modelData;
+    }
+
+    /**
+     * Computes max and min values of features.
+     *
+     * @param inputData Input data.
+     * @param featureCol Feature column name.
+     * @return Max and min values of features.
+     */
+    private DataStream<Tuple2<DenseVector, DenseVector>> computeMinMaxVectors(

Review comment:
       computeMinMaxVectors execute in parallel mode, genModelData reduce distributed minMax values to one. 




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r823414741



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerParams.java
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.ml.common.param.HasFeaturesCol;
+import org.apache.flink.ml.common.param.HasOutputCol;
+import org.apache.flink.ml.param.DoubleParam;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+
+/**
+ * Params for {@link MinMaxScaler}.
+ *
+ * @param <T> The class type of this instance.
+ */
+public interface MinMaxScalerParams<T> extends HasFeaturesCol<T>, HasOutputCol<T> {
+    Param<Double> MAX =
+            new DoubleParam(
+                    "max", "Upper bound after transformation.", 1.0, ParamValidators.notNull());
+
+    default Double getMax() {
+        return get(MAX);
+    }
+
+    default T setMax(Double value) {
+        return set(MAX, value);
+    }
+
+    Param<Double> MIN =
+            new DoubleParam(
+                    "min", "Lower bound after transformation.", 0.0, ParamValidators.notNull());
+
+    default Double getMIN() {

Review comment:
       OK




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r823412958



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/common/param/HasOutputCol.java
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.common.param;
+
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+import org.apache.flink.ml.param.StringParam;
+import org.apache.flink.ml.param.WithParams;
+
+/** Interface for the shared outputCol param. */
+public interface HasOutputCol<T> extends WithParams<T> {

Review comment:
       PredictionCol may be better for ml algo. For feature engineer and data proc, outputCol may be better.

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/common/param/HasOutputCol.java
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.common.param;
+
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+import org.apache.flink.ml.param.StringParam;
+import org.apache.flink.ml.param.WithParams;
+
+/** Interface for the shared outputCol param. */
+public interface HasOutputCol<T> extends WithParams<T> {

Review comment:
       OK




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r826541447



##########
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:
       OK




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r826538436



##########
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:
       I have refine it. I check the vector before collect and save to checkpoint.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829015961



##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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(trainData);
+        trainDataTable = tEnv.fromDataStream(dataStream, schema).as("features");
+        DataStream<Row> predDataStream = env.fromCollection(predictRows);
+        predictDataTable = tEnv.fromDataStream(predDataStream, schema).as("features");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol, DenseVector expected)
+            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(1, result.size());
+        assertEquals(expected, 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(trainDataTable.as("test_features"));
+        Table output = model.transform(predictDataTable.as("test_features"))[0];
+        assertEquals(
+                Arrays.asList("test_features", "test_output"),
+                output.getResolvedSchema().getColumnNames());
+    }
+
+    @Test
+    public void testFewerDistinctPointsThanCluster() throws Exception {

Review comment:
       Kmeans and knn use this name already, I just take it from other ut.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829675465



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.iteration.operator.OperatorStateUtils;
+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).

Review comment:
       OK




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829680935



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {

Review comment:
       If featureValue[i] != maxVector[i], above code may not get the same result as original code.
   In read world, data for train and predict are different, featureValue[i] != maxVector[i] is existed. 




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829729279



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {

Review comment:
       I don‘t think so. 
   If featureValue[i] != maxVector[i],  predict data may have mo problem. For train data defines the rule and we can't constrain predict data follow this rule. 
   
   For example, in train data one feature is people's age, all feature values in the train data is 20 years old. If predict data has a sample which feature is 21 years old which will be scaled to 0.5*(max - min). 
     




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r831711819



##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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 =

Review comment:
       fixed already.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r831694755



##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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, 400))));
+    private static final List<Row> predictData =
+            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);
+        trainDataTable = tEnv.fromDataStream(env.fromCollection(trainData)).as("features");
+        predictDataTable = tEnv.fromDataStream(env.fromCollection(predictData)).as("features");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol, DenseVector expected)
+            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(1, result.size());
+        assertEquals(expected, result.get(0));
+    }
+
+    @Test
+    public void testParam() {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        assertEquals("features", minMaxScaler.getFeaturesCol());
+        assertEquals("prediction", minMaxScaler.getPredictionCol());
+        assertEquals(1.0, minMaxScaler.getMax(), 0.0001);
+        assertEquals(0.0, minMaxScaler.getMin(), 0.0001);
+        minMaxScaler
+                .setFeaturesCol("test_features")
+                .setPredictionCol("test_output")
+                .setMax(4.0)
+                .setMin(1.0);
+        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()
+                        .setFeaturesCol("test_features")
+                        .setPredictionCol("test_output")
+                        .setMin(1.0)
+                        .setMax(4.0);
+
+        MinMaxScalerModel model = minMaxScaler.fit(trainDataTable.as("test_features"));
+        Table output = model.transform(predictDataTable.as("test_features"))[0];
+        assertEquals(
+                Arrays.asList("test_features", "test_output"),
+                output.getResolvedSchema().getColumnNames());
+    }
+
+    @Test
+    public void testFewerSamplesThanParallel() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScalerModel model = minMaxScaler.fit(predictDataTable);
+        Table result = model.transform(predictDataTable)[0];
+        verifyPredictionResult(result, minMaxScaler.getPredictionCol(), Vectors.dense(0.5, 0.5));
+    }
+
+    @Test
+    public void testMaxValueEqualsMinValueButPredictValueNotEquals() throws Exception {
+        List<Row> userDefineData =
+                new ArrayList<>(Collections.singletonList(Row.of(Vectors.dense(30.0, 50.0))));
+        Table userDefineDataTable =
+                tEnv.fromDataStream(env.fromCollection(userDefineData)).as("features");
+        MinMaxScaler minMaxScaler = new MinMaxScaler().setMax(10.0).setMin(0.0);
+        MinMaxScalerModel model = minMaxScaler.fit(predictDataTable);
+        Table result = model.transform(userDefineDataTable)[0];
+        verifyPredictionResult(result, minMaxScaler.getPredictionCol(), Vectors.dense(5.0, 5.0));
+    }
+
+    @Test
+    public void testFitAndPredict() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScalerModel minMaxScalerModel = minMaxScaler.fit(trainDataTable);
+        Table output = minMaxScalerModel.transform(predictDataTable)[0];
+        verifyPredictionResult(output, minMaxScaler.getPredictionCol(), Vectors.dense(0.75, 0.225));
+    }
+
+    @Test
+    public void testSaveLoadAndPredict() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScaler loadedMinMaxScaler =
+                StageTestUtils.saveAndReload(
+                        env, minMaxScaler, tempFolder.newFolder().getAbsolutePath());
+        MinMaxScalerModel minMaxScalerModel = loadedMinMaxScaler.fit(trainDataTable);
+        minMaxScalerModel =

Review comment:
       Sounds good!




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r831811014



##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.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.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.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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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, 400))));
+    private static final List<Row> predictData =
+            new ArrayList<>(
+                    Arrays.asList(
+                            Row.of(Vectors.dense(150.0, 90.0)),
+                            Row.of(Vectors.dense(50.0, 40.0)),
+                            Row.of(Vectors.dense(100.0, 50.0))));
+
+    private static final List<DenseVector> expectedData =
+            new ArrayList<>(
+                    Arrays.asList(
+                            Vectors.dense(0.25, 0.1),
+                            Vectors.dense(0.5, 0.125),
+                            Vectors.dense(0.75, 0.225)));
+
+    @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);
+        trainDataTable = tEnv.fromDataStream(env.fromCollection(trainData)).as("features");
+        predictDataTable = tEnv.fromDataStream(env.fromCollection(predictData)).as("features");
+    }
+
+    private static void verifyPredictionResult(
+            Table output, String outputCol, List<DenseVector> expected) 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());
+
+        result.sort(
+                (first, second) -> {
+                    for (int x = 0; x < first.size(); x++) {
+                        int cmp = Double.compare(first.get(x), second.get(x));
+                        if (cmp != 0) {
+                            return cmp;
+                        }
+                    }
+                    return 0;
+                });
+        assertEquals(expected.size(), result.size());

Review comment:
       OK, I will remove it.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r831815901



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.common.typeinfo.TypeInformation;
+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.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(),
+                                TypeInformation.of(DenseVector.class)),
+                        ArrayUtils.addAll(inputTypeInfo.getFieldNames(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private final String broadcastKey;
+        private final double upperBound;
+        private final double lowerBound;
+        private DenseVector scaleVector;
+        private DenseVector offsetVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (scaleVector == null) {
+                MinMaxScalerModelData minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                DenseVector minVector = minMaxScalerModelData.minVector;
+                DenseVector maxVector = minMaxScalerModelData.maxVector;
+                scaleVector = new DenseVector(minVector.size());
+                offsetVector = new DenseVector(minVector.size());
+                for (int i = 0; i < maxVector.size(); ++i) {
+                    if ((minVector.values[i] - maxVector.values[i]) == 0.0) {

Review comment:
       I think, [3] is much better. I will refine the code later.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#issuecomment-1017107333


   @zhipeng93 @lindong28 @gaoyunhaii Can you help review this PR?


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829681672



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {
+                    if ((minVector.values[i] - maxVector.values[i]) != 0.0) {
+                        outputVector.values[i] =
+                                (feature.values[i] - minVector.values[i])
+                                                / (maxVector.values[i] - minVector.values[i])

Review comment:
       val scaleArray = Array.tabulate(numFeatures) { i =>
         val range = originalMax(i) - originalMin(i)




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829680935



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {

Review comment:
       When maxVector[i] == minVector[i],
   If featureValue[i] != maxVector[i], above code may not get the same result as original code.
   In real world, data for train and predict are different, featureValue[i] != maxVector[i] is existed. 




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829635833



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {
+                    if ((minVector.values[i] - maxVector.values[i]) != 0.0) {
+                        outputVector.values[i] =
+                                (feature.values[i] - minVector.values[i])
+                                                / (maxVector.values[i] - minVector.values[i])

Review comment:
       I'm not sure this is the case for Spark. In Spark's MinMaxScaler.scala I found the following code:
   ```scala
   values(i) = (values(i) - minArray(i)) * scaleArray(i) + minValue
   ```
   which shows that Spark uses `minArray` and `scaleArray`, instead of `minArray` and `maxArray`.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829650219



##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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(trainData);
+        trainDataTable = tEnv.fromDataStream(dataStream, schema).as("features");
+        DataStream<Row> predDataStream = env.fromCollection(predictRows);
+        predictDataTable = tEnv.fromDataStream(predDataStream, schema).as("features");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol, DenseVector expected)
+            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(1, result.size());
+        assertEquals(expected, 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(trainDataTable.as("test_features"));
+        Table output = model.transform(predictDataTable.as("test_features"))[0];
+        assertEquals(
+                Arrays.asList("test_features", "test_output"),
+                output.getResolvedSchema().getColumnNames());
+    }
+
+    @Test
+    public void testFewerDistinctPointsThanCluster() throws Exception {

Review comment:
       OK,I got it. I will refine it later.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] zhipeng93 commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
zhipeng93 commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r828785312



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,197 @@
+/*
+ * 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> features =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                (MapFunction<Row, DenseVector>)
+                                        value -> (DenseVector) value.getField(featureCol));
+        DataStream<DenseVector> minMaxValues =
+                features.transform(
+                                "reduceInEachPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .transform(
+                                "reduceInFinalPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .setParallelism(1);
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(
+                        minMaxValues,
+                        new RichMapPartitionFunction<DenseVector, MinMaxScalerModelData>() {
+                            @Override
+                            public void mapPartition(
+                                    Iterable<DenseVector> values,
+                                    Collector<MinMaxScalerModelData> out) {
+                                Iterator<DenseVector> iter = values.iterator();
+                                DenseVector minVector = iter.next();
+                                DenseVector maxVector = iter.next();
+                                out.collect(new MinMaxScalerModelData(minVector, maxVector));
+                            }
+                        });
+
+        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> minState;
+        private ListState<DenseVector> maxState;
+
+        private DenseVector minVector;
+        private DenseVector maxVector;
+
+        @Override
+        public void endInput() {
+            output.collect(new StreamRecord<>(minVector));
+            output.collect(new StreamRecord<>(maxVector));
+        }
+
+        @Override
+        public void processElement(StreamRecord<DenseVector> streamRecord) {
+            DenseVector currentValue = streamRecord.getValue();
+            if (minVector == null) {
+                int vecSize = currentValue.size();
+                minVector = new DenseVector(vecSize);
+                maxVector = new DenseVector(vecSize);
+                System.arraycopy(currentValue.values, 0, minVector.values, 0, vecSize);
+                System.arraycopy(currentValue.values, 0, maxVector.values, 0, vecSize);
+
+            } else {
+                for (int i = 0; i < currentValue.size(); ++i) {
+                    minVector.values[i] = Math.min(minVector.values[i], currentValue.values[i]);
+                    maxVector.values[i] = Math.max(maxVector.values[i], currentValue.values[i]);
+                }
+            }
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            minState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "minVectorState",

Review comment:
       nit: minVectorState -> minState

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,197 @@
+/*
+ * 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> features =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                (MapFunction<Row, DenseVector>)
+                                        value -> (DenseVector) value.getField(featureCol));
+        DataStream<DenseVector> minMaxValues =
+                features.transform(
+                                "reduceInEachPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .transform(
+                                "reduceInFinalPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .setParallelism(1);
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(
+                        minMaxValues,
+                        new RichMapPartitionFunction<DenseVector, MinMaxScalerModelData>() {
+                            @Override
+                            public void mapPartition(
+                                    Iterable<DenseVector> values,
+                                    Collector<MinMaxScalerModelData> out) {
+                                Iterator<DenseVector> iter = values.iterator();
+                                DenseVector minVector = iter.next();
+                                DenseVector maxVector = iter.next();
+                                out.collect(new MinMaxScalerModelData(minVector, maxVector));
+                            }
+                        });
+
+        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> minState;
+        private ListState<DenseVector> maxState;
+
+        private DenseVector minVector;
+        private DenseVector maxVector;
+
+        @Override
+        public void endInput() {
+            output.collect(new StreamRecord<>(minVector));
+            output.collect(new StreamRecord<>(maxVector));
+        }
+
+        @Override
+        public void processElement(StreamRecord<DenseVector> streamRecord) {
+            DenseVector currentValue = streamRecord.getValue();
+            if (minVector == null) {
+                int vecSize = currentValue.size();
+                minVector = new DenseVector(vecSize);
+                maxVector = new DenseVector(vecSize);
+                System.arraycopy(currentValue.values, 0, minVector.values, 0, vecSize);
+                System.arraycopy(currentValue.values, 0, maxVector.values, 0, vecSize);
+
+            } else {
+                for (int i = 0; i < currentValue.size(); ++i) {
+                    minVector.values[i] = Math.min(minVector.values[i], currentValue.values[i]);
+                    maxVector.values[i] = Math.max(maxVector.values[i], currentValue.values[i]);
+                }
+            }
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            minState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "minVectorState",
+                                            getOperatorConfig()
+                                                    .getTypeSerializerIn(
+                                                            0, getClass().getClassLoader())));
+            maxState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "maxVectorState",

Review comment:
       ditto.

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,197 @@
+/*
+ * 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> features =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                (MapFunction<Row, DenseVector>)
+                                        value -> (DenseVector) value.getField(featureCol));
+        DataStream<DenseVector> minMaxValues =
+                features.transform(
+                                "reduceInEachPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .transform(
+                                "reduceInFinalPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .setParallelism(1);
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(
+                        minMaxValues,
+                        new RichMapPartitionFunction<DenseVector, MinMaxScalerModelData>() {
+                            @Override
+                            public void mapPartition(
+                                    Iterable<DenseVector> values,
+                                    Collector<MinMaxScalerModelData> out) {
+                                Iterator<DenseVector> iter = values.iterator();
+                                DenseVector minVector = iter.next();
+                                DenseVector maxVector = iter.next();
+                                out.collect(new MinMaxScalerModelData(minVector, maxVector));
+                            }
+                        });
+
+        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

Review comment:
       min max vectors --> ...min and max values

##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,206 @@
+/*
+ * 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());
+        for (DenseVector t2 : result) {
+            assertEquals(Vectors.dense(0.75, 0.3), t2);
+        }
+    }
+
+    @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("output", minMaxScaler.getOutputCol());
+        minMaxScaler
+                .setFeaturesCol("test_features")
+                .setMax(4.0)
+                .setMIN(0.0)
+                .setOutputCol("test_output");
+        assertEquals("test_features", minMaxScaler.getFeaturesCol());
+        assertEquals(0.0, minMaxScaler.getMIN(), 0.0001);
+        assertEquals(4.0, minMaxScaler.getMax(), 0.0001);
+        assertEquals("test_output", minMaxScaler.getOutputCol());
+    }
+
+    @Test
+    public void testFeaturePredictionParam() {
+        MinMaxScaler minMaxScaler =
+                new MinMaxScaler()
+                        .setMIN(0.0)
+                        .setMax(4.0)
+                        .setFeaturesCol("test_features")
+                        .setOutputCol("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() {

Review comment:
       Could you also verify the execution result here?

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,197 @@
+/*
+ * 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> features =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                (MapFunction<Row, DenseVector>)
+                                        value -> (DenseVector) value.getField(featureCol));
+        DataStream<DenseVector> minMaxValues =
+                features.transform(
+                                "reduceInEachPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .transform(
+                                "reduceInFinalPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .setParallelism(1);
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(
+                        minMaxValues,
+                        new RichMapPartitionFunction<DenseVector, MinMaxScalerModelData>() {
+                            @Override
+                            public void mapPartition(
+                                    Iterable<DenseVector> values,
+                                    Collector<MinMaxScalerModelData> out) {
+                                Iterator<DenseVector> iter = values.iterator();
+                                DenseVector minVector = iter.next();
+                                DenseVector maxVector = iter.next();
+                                out.collect(new MinMaxScalerModelData(minVector, maxVector));
+                            }
+                        });
+
+        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> minState;
+        private ListState<DenseVector> maxState;
+
+        private DenseVector minVector;
+        private DenseVector maxVector;
+
+        @Override
+        public void endInput() {
+            output.collect(new StreamRecord<>(minVector));

Review comment:
       If there is no data in one partition, it throw a NPE here.

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        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> {

Review comment:
       Is `PredictOutputFunction` a better name for `PredictLabelFunction`, since there is no label here?

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModelData.java
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.serialization.Encoder;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.src.reader.SimpleStreamFormat;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.linalg.typeinfo.DenseVectorSerializer;
+import org.apache.flink.streaming.api.datastream.DataStream;
+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 java.io.EOFException;
+import java.io.IOException;
+import java.io.OutputStream;
+
+/**
+ * Model data of {@link MinMaxScalerModel}.
+ *
+ * <p>This class also provides methods to convert model data from Table to a data stream, and
+ * classes to save/load model data.
+ */
+public class MinMaxScalerModelData {
+    public DenseVector minVector;
+
+    public DenseVector maxVector;
+
+    public MinMaxScalerModelData() {}
+
+    public MinMaxScalerModelData(DenseVector minVector, DenseVector maxVector) {
+        this.minVector = minVector;
+        this.maxVector = maxVector;
+    }
+
+    /**
+     * Converts the table model to a data stream.
+     *
+     * @param modelDataTable The table model data.
+     * @return The data stream model data.
+     */
+    public static DataStream<MinMaxScalerModelData> getModelDataStream(Table modelDataTable) {
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) modelDataTable).getTableEnvironment();
+        return tEnv.toDataStream(modelDataTable)
+                .map(
+                        x ->
+                                new MinMaxScalerModelData(
+                                        (DenseVector) x.getField(0), (DenseVector) x.getField(1)));
+    }
+
+    /** Encoder for {@link MinMaxScalerModelData}. */
+    public static class ModelDataEncoder implements Encoder<MinMaxScalerModelData> {
+        @Override
+        public void encode(MinMaxScalerModelData minMaxScalerModelData, OutputStream outputStream)
+                throws IOException {
+            DataOutputView dataOutputView = new DataOutputViewStreamWrapper(outputStream);
+            DenseVectorSerializer.INSTANCE.serialize(
+                    minMaxScalerModelData.minVector, dataOutputView);
+            DenseVectorSerializer.INSTANCE.serialize(
+                    minMaxScalerModelData.maxVector, dataOutputView);
+        }
+    }
+
+    /** Decoder for {@link MinMaxScalerModelData}. */
+    public static class ModelDataDecoder extends SimpleStreamFormat<MinMaxScalerModelData> {
+        @Override
+        public Reader<MinMaxScalerModelData> createReader(
+                Configuration config, FSDataInputStream stream) {
+            return new Reader<MinMaxScalerModelData>() {
+
+                private final DataInputView source = new DataInputViewStreamWrapper(stream);

Review comment:
       nit: this could be intialized in `read()`.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829675125



##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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(trainData);
+        trainDataTable = tEnv.fromDataStream(dataStream, schema).as("features");
+        DataStream<Row> predDataStream = env.fromCollection(predictRows);
+        predictDataTable = tEnv.fromDataStream(predDataStream, schema).as("features");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol, DenseVector expected)
+            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(1, result.size());
+        assertEquals(expected, 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(trainDataTable.as("test_features"));
+        Table output = model.transform(predictDataTable.as("test_features"))[0];
+        assertEquals(
+                Arrays.asList("test_features", "test_output"),
+                output.getResolvedSchema().getColumnNames());
+    }
+
+    @Test
+    public void testFewerDistinctPointsThanCluster() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScalerModel model = minMaxScaler.fit(predictDataTable);
+        Table result = model.transform(predictDataTable)[0];
+        verifyPredictionResult(result, minMaxScaler.getPredictionCol(), Vectors.dense(0.5, 0.5));
+    }
+
+    @Test
+    public void testFitAndPredict() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScalerModel minMaxScalerModel = minMaxScaler.fit(trainDataTable);
+        Table output = minMaxScalerModel.transform(predictDataTable)[0];
+        verifyPredictionResult(output, minMaxScaler.getPredictionCol(), Vectors.dense(0.75, 0.3));
+    }
+
+    @Test
+    public void testSaveLoadAndPredict() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScaler loadedMinMaxScaler =
+                StageTestUtils.saveAndReload(
+                        env, minMaxScaler, tempFolder.newFolder().getAbsolutePath());
+        MinMaxScalerModel minMaxScalerModel = loadedMinMaxScaler.fit(trainDataTable);
+        minMaxScalerModel =
+                StageTestUtils.saveAndReload(
+                        env, minMaxScalerModel, tempFolder.newFolder().getAbsolutePath());
+        assertEquals(
+                Arrays.asList("minVector", "maxVector"),
+                minMaxScalerModel.getModelData()[0].getResolvedSchema().getColumnNames());
+        Table output = minMaxScalerModel.transform(predictDataTable)[0];
+        verifyPredictionResult(output, minMaxScaler.getPredictionCol(), Vectors.dense(0.75, 0.3));
+    }
+
+    @Test
+    public void testModelSaveLoadAndPredict() throws Exception {

Review comment:
       OK




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829846075



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {

Review comment:
       Got it. Then how about this one?
   ```java
           @Override
           public Row map(Row row) {
               if (minMaxScalerModelData == null) {
                   minMaxScalerModelData =
                           (MinMaxScalerModelData)
                                   getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
                   maxVector = minMaxScalerModelData.maxVector;
                   minVector = minMaxScalerModelData.minVector;
                   scaleVector = new DenseVector(minVector.size());
                   offsetVector = new DenseVector(minVector.size());
                   for (int i = 0; i < maxVector.size(); ++i) {
                       if ((minVector.values[i] - maxVector.values[i]) == 0.0) {
                           scaleVector.values[i] = Double.POSITIVE_INFINITY;
                           offsetVector.values[i] = 0.5;
                       } else {
                           scaleVector.values[i] = maxVector.values[i] - minVector.values[i];
                           offsetVector.values[i] = 0.0;
                       }
                   }
               }
               DenseVector feature = (DenseVector) row.getField(featureCol);
               DenseVector outputVector = new DenseVector(maxVector.size());
               if (feature != null) {
                   for (int i = 0; i < maxVector.size(); ++i) {
                       outputVector.values[i] =
                               ((feature.values[i] - minVector.values[i])
                                                       / scaleVector.values[i]
                                                       + offsetVector.values[i])
                                               * (upperBound - lowerBound)
                                       + lowerBound;
                   }
                   return Row.join(row, Row.of(outputVector));
               } else {
                   throw new RuntimeException("Feature value is null, please check your input data.");
               }
           }
   ```
   
   This code should meet the requirements without using if conditions.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829877070



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {

Review comment:
       Even better:
   ```java
   
           @Override
           public Row map(Row row) {
               if (minMaxScalerModelData == null) {
                   minMaxScalerModelData =
                           (MinMaxScalerModelData)
                                   getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
                   maxVector = minMaxScalerModelData.maxVector;
                   minVector = minMaxScalerModelData.minVector;
                   scaleVector = new DenseVector(minVector.size());
                   offsetVector = new DenseVector(minVector.size());
                   for (int i = 0; i < maxVector.size(); ++i) {
                       if ((minVector.values[i] - maxVector.values[i]) == 0.0) {
                           scaleVector.values[i] = 0.0;
                           offsetVector.values[i] = (upperBound + lowerBound) / 2;
                       } else {
                           scaleVector.values[i] = (upperBound - lowerBound) / (maxVector.values[i] - minVector.values[i]);
                           offsetVector.values[i] = lowerBound - minVector.values[i] * scaleVector.values[i];
                       }
                   }
               }
               DenseVector feature = (DenseVector) row.getField(featureCol);
               DenseVector outputVector = new DenseVector(maxVector.size());
               if (feature != null) {
                   for (int i = 0; i < maxVector.size(); ++i) {
                       outputVector.values[i] = feature.values[i] * scaleVector.values[i] + offsetVector.values[i];
                   }
                   return Row.join(row, Row.of(outputVector));
               } else {
                   throw new RuntimeException("Feature value is null, please check your input data.");
               }
           }
   ```
   
   With this implementation, there would be only one multiplication and one addition in the for loop, which can definitely be replaced by `BLAS.axpy`. It would cause some precision loss (less then 1e-16 on this PR's test data), but I think the performance improvement worth it.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r823425316



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerParams.java
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.ml.common.param.HasFeaturesCol;
+import org.apache.flink.ml.common.param.HasOutputCol;
+import org.apache.flink.ml.param.DoubleParam;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+
+/**
+ * Params for {@link MinMaxScaler}.
+ *
+ * @param <T> The class type of this instance.
+ */
+public interface MinMaxScalerParams<T> extends HasFeaturesCol<T>, HasOutputCol<T> {
+    Param<Double> MAX =
+            new DoubleParam(
+                    "max", "Upper bound after transformation.", 1.0, ParamValidators.notNull());
+
+    default Double getMax() {
+        return get(MAX);
+    }
+
+    default T setMax(Double value) {
+        return set(MAX, value);
+    }
+
+    Param<Double> MIN =
+            new DoubleParam(
+                    "min", "Lower bound after transformation.", 0.0, ParamValidators.notNull());
+
+    default Double getMIN() {
+        return get(MIN);
+    }
+
+    default T setMIN(Double value) {

Review comment:
       ditto.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r826539650



##########
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:
       OK




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r831697137



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerParams.java
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.ml.common.param.HasFeaturesCol;
+import org.apache.flink.ml.common.param.HasPredictionCol;
+import org.apache.flink.ml.param.DoubleParam;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+
+/**
+ * Params for {@link MinMaxScaler}.
+ *
+ * @param <T> The class type of this instance.
+ */
+public interface MinMaxScalerParams<T> extends HasFeaturesCol<T>, HasPredictionCol<T> {
+    Param<Double> MAX =
+            new DoubleParam(
+                    "max", "Upper bound after transformation.", 1.0, ParamValidators.notNull());

Review comment:
       I agree with it.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
lindong28 commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r831052100



##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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, 400))));
+    private static final List<Row> predictData =
+            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);
+        trainDataTable = tEnv.fromDataStream(env.fromCollection(trainData)).as("features");
+        predictDataTable = tEnv.fromDataStream(env.fromCollection(predictData)).as("features");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol, DenseVector expected)
+            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(1, result.size());
+        assertEquals(expected, result.get(0));
+    }
+
+    @Test
+    public void testParam() {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        assertEquals("features", minMaxScaler.getFeaturesCol());
+        assertEquals("prediction", minMaxScaler.getPredictionCol());
+        assertEquals(1.0, minMaxScaler.getMax(), 0.0001);
+        assertEquals(0.0, minMaxScaler.getMin(), 0.0001);
+        minMaxScaler
+                .setFeaturesCol("test_features")
+                .setPredictionCol("test_output")
+                .setMax(4.0)
+                .setMin(1.0);
+        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()
+                        .setFeaturesCol("test_features")
+                        .setPredictionCol("test_output")
+                        .setMin(1.0)
+                        .setMax(4.0);
+
+        MinMaxScalerModel model = minMaxScaler.fit(trainDataTable.as("test_features"));
+        Table output = model.transform(predictDataTable.as("test_features"))[0];
+        assertEquals(
+                Arrays.asList("test_features", "test_output"),
+                output.getResolvedSchema().getColumnNames());
+    }
+
+    @Test
+    public void testFewerSamplesThanParallel() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScalerModel model = minMaxScaler.fit(predictDataTable);
+        Table result = model.transform(predictDataTable)[0];
+        verifyPredictionResult(result, minMaxScaler.getPredictionCol(), Vectors.dense(0.5, 0.5));
+    }
+
+    @Test
+    public void testMaxValueEqualsMinValueButPredictValueNotEquals() throws Exception {
+        List<Row> userDefineData =
+                new ArrayList<>(Collections.singletonList(Row.of(Vectors.dense(30.0, 50.0))));
+        Table userDefineDataTable =
+                tEnv.fromDataStream(env.fromCollection(userDefineData)).as("features");
+        MinMaxScaler minMaxScaler = new MinMaxScaler().setMax(10.0).setMin(0.0);
+        MinMaxScalerModel model = minMaxScaler.fit(predictDataTable);
+        Table result = model.transform(userDefineDataTable)[0];
+        verifyPredictionResult(result, minMaxScaler.getPredictionCol(), Vectors.dense(5.0, 5.0));
+    }
+
+    @Test
+    public void testFitAndPredict() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScalerModel minMaxScalerModel = minMaxScaler.fit(trainDataTable);
+        Table output = minMaxScalerModel.transform(predictDataTable)[0];
+        verifyPredictionResult(output, minMaxScaler.getPredictionCol(), Vectors.dense(0.75, 0.225));
+    }
+
+    @Test
+    public void testSaveLoadAndPredict() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScaler loadedMinMaxScaler =
+                StageTestUtils.saveAndReload(
+                        env, minMaxScaler, tempFolder.newFolder().getAbsolutePath());
+        MinMaxScalerModel minMaxScalerModel = loadedMinMaxScaler.fit(trainDataTable);
+        minMaxScalerModel =

Review comment:
       nits: should we replace `minMaxScalerModel = ...` with `MinMaxScaler loadedMinMaxScalerModel = ...` so that the name is consistent with `loadedKmeans` used above? 
   
   This would also be consistent with the naming pattern used in other tests (e.g. `KMeansTest::testSaveLoadAndPredict`).
   
   If the name is too long, feel free to simplify the name to `model` and `loadedModel`.
   

##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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, 400))));
+    private static final List<Row> predictData =
+            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);
+        trainDataTable = tEnv.fromDataStream(env.fromCollection(trainData)).as("features");
+        predictDataTable = tEnv.fromDataStream(env.fromCollection(predictData)).as("features");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol, DenseVector expected)
+            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(1, result.size());
+        assertEquals(expected, result.get(0));
+    }
+
+    @Test
+    public void testParam() {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        assertEquals("features", minMaxScaler.getFeaturesCol());
+        assertEquals("prediction", minMaxScaler.getPredictionCol());
+        assertEquals(1.0, minMaxScaler.getMax(), 0.0001);
+        assertEquals(0.0, minMaxScaler.getMin(), 0.0001);
+        minMaxScaler
+                .setFeaturesCol("test_features")
+                .setPredictionCol("test_output")
+                .setMax(4.0)
+                .setMin(1.0);
+        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()
+                        .setFeaturesCol("test_features")
+                        .setPredictionCol("test_output")
+                        .setMin(1.0)
+                        .setMax(4.0);
+
+        MinMaxScalerModel model = minMaxScaler.fit(trainDataTable.as("test_features"));
+        Table output = model.transform(predictDataTable.as("test_features"))[0];
+        assertEquals(
+                Arrays.asList("test_features", "test_output"),
+                output.getResolvedSchema().getColumnNames());
+    }
+
+    @Test
+    public void testFewerSamplesThanParallel() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScalerModel model = minMaxScaler.fit(predictDataTable);
+        Table result = model.transform(predictDataTable)[0];
+        verifyPredictionResult(result, minMaxScaler.getPredictionCol(), Vectors.dense(0.5, 0.5));
+    }
+
+    @Test
+    public void testMaxValueEqualsMinValueButPredictValueNotEquals() throws Exception {
+        List<Row> userDefineData =
+                new ArrayList<>(Collections.singletonList(Row.of(Vectors.dense(30.0, 50.0))));
+        Table userDefineDataTable =
+                tEnv.fromDataStream(env.fromCollection(userDefineData)).as("features");
+        MinMaxScaler minMaxScaler = new MinMaxScaler().setMax(10.0).setMin(0.0);
+        MinMaxScalerModel model = minMaxScaler.fit(predictDataTable);
+        Table result = model.transform(userDefineDataTable)[0];
+        verifyPredictionResult(result, minMaxScaler.getPredictionCol(), Vectors.dense(5.0, 5.0));
+    }
+
+    @Test
+    public void testFitAndPredict() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScalerModel minMaxScalerModel = minMaxScaler.fit(trainDataTable);
+        Table output = minMaxScalerModel.transform(predictDataTable)[0];
+        verifyPredictionResult(output, minMaxScaler.getPredictionCol(), Vectors.dense(0.75, 0.225));
+    }
+
+    @Test
+    public void testSaveLoadAndPredict() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScaler loadedMinMaxScaler =
+                StageTestUtils.saveAndReload(
+                        env, minMaxScaler, tempFolder.newFolder().getAbsolutePath());
+        MinMaxScalerModel minMaxScalerModel = loadedMinMaxScaler.fit(trainDataTable);
+        minMaxScalerModel =
+                StageTestUtils.saveAndReload(
+                        env, minMaxScalerModel, tempFolder.newFolder().getAbsolutePath());
+        assertEquals(
+                Arrays.asList("minVector", "maxVector"),
+                minMaxScalerModel.getModelData()[0].getResolvedSchema().getColumnNames());
+        Table output = minMaxScalerModel.transform(predictDataTable)[0];
+        verifyPredictionResult(output, minMaxScaler.getPredictionCol(), Vectors.dense(0.75, 0.225));
+    }
+
+    @Test
+    public void testGetModelData() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScalerModel minMaxScalerModel = minMaxScaler.fit(trainDataTable);
+        Table modelData = minMaxScalerModel.getModelData()[0];
+        DataStream<Row> output = tEnv.toDataStream(modelData);
+        assertEquals("minVector", modelData.getResolvedSchema().getColumnNames().get(0));

Review comment:
       nits: could we replace these two lines with the following one line? This would make the code a bit simpler and also more extensible if in the future we want to test schemas with 3+ fields.
   
   ```
   assertEquals(
           Arrays.asList("minVector", "maxVector"),
           modelData.getResolvedSchema().getColumnNames());
   ```

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.iteration.operator.OperatorStateUtils;
+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. This algorithm rescales feature values
+ * to a common range [min, max] which defined by user.
+ *
+ * <blockquote>
+ *
+ * $$ Rescaled(value) = \frac{value - E_{min}}{E_{max} - E_{min}} * (max - min) + min $$
+ *
+ * </blockquote>
+ *
+ * <p>For the case \(E_{max} == E_{min}\), \(Rescaled(value) = 0.5 * (max + min)\).
+ *
+ * <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> features =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                (MapFunction<Row, DenseVector>)
+                                        value -> (DenseVector) value.getField(featureCol));
+        DataStream<DenseVector> minMaxValues =
+                features.transform(
+                                "reduceInEachPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .transform(
+                                "reduceInFinalPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .setParallelism(1);
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(
+                        minMaxValues,
+                        new RichMapPartitionFunction<DenseVector, MinMaxScalerModelData>() {
+                            @Override
+                            public void mapPartition(
+                                    Iterable<DenseVector> values,
+                                    Collector<MinMaxScalerModelData> out) {
+                                Iterator<DenseVector> iter = values.iterator();
+                                DenseVector minVector = iter.next();
+                                DenseVector maxVector = iter.next();
+                                out.collect(new MinMaxScalerModelData(minVector, maxVector));
+                            }
+                        });
+
+        MinMaxScalerModel model =
+                new MinMaxScalerModel().setModelData(tEnv.fromDataStream(modelData));
+        ReadWriteUtils.updateExistingParams(model, getParamMap());
+        return model;
+    }
+
+    /**
+     * A stream operator to compute the min and max values in each partition of the input bounded
+     * data stream.
+     */
+    private static class MinMaxReduceFunctionOperator extends AbstractStreamOperator<DenseVector>
+            implements OneInputStreamOperator<DenseVector, DenseVector>, BoundedOneInput {
+        private ListState<DenseVector> minState;
+        private ListState<DenseVector> maxState;
+
+        private DenseVector minVector;
+        private DenseVector maxVector;
+
+        @Override
+        public void endInput() {
+            if (minVector != null) {
+                output.collect(new StreamRecord<>(minVector));
+                output.collect(new StreamRecord<>(maxVector));
+            }
+        }
+
+        @Override
+        public void processElement(StreamRecord<DenseVector> streamRecord) {
+            DenseVector currentValue = streamRecord.getValue();
+            if (minVector == null) {
+                int vecSize = currentValue.size();
+                minVector = new DenseVector(vecSize);
+                maxVector = new DenseVector(vecSize);
+                System.arraycopy(currentValue.values, 0, minVector.values, 0, vecSize);
+                System.arraycopy(currentValue.values, 0, maxVector.values, 0, vecSize);
+            } else {
+                Preconditions.checkArgument(
+                        currentValue.size() == maxVector.size(),
+                        "CurrentValue should has same size with maxVector.");
+                for (int i = 0; i < currentValue.size(); ++i) {
+                    minVector.values[i] = Math.min(minVector.values[i], currentValue.values[i]);
+                    maxVector.values[i] = Math.max(maxVector.values[i], currentValue.values[i]);
+                }
+            }
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            minState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(

Review comment:
       Hmm.. instead of using `getOperatorConfig().getTypeSerializerIn(...)`, would it be better to use the code below for simplicity and consistency with other usages of `getListState(...)` in Flink ML?
   
   ```
   new ListStateDescriptor<>("minState", TypeInformation.of(DenseVector.class))
   ```

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,189 @@
+/*
+ * 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)),

Review comment:
       Hmm.. is there any reason in particular to use `ExternalTypeInfo`?
   
   Would it be better to use `TypeInformation.of(DenseVector.class)` for consistency with e.g. `LogisticRegressionModel::transform(...)`?
   

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerParams.java
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.ml.common.param.HasFeaturesCol;
+import org.apache.flink.ml.common.param.HasPredictionCol;
+import org.apache.flink.ml.param.DoubleParam;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+
+/**
+ * Params for {@link MinMaxScaler}.
+ *
+ * @param <T> The class type of this instance.
+ */
+public interface MinMaxScalerParams<T> extends HasFeaturesCol<T>, HasPredictionCol<T> {
+    Param<Double> MAX =
+            new DoubleParam(
+                    "max", "Upper bound after transformation.", 1.0, ParamValidators.notNull());

Review comment:
       Spark uses `upper bound of the output feature range` as the description for this parameter. It useful to explicitly specify what this upper bound applies to (i.e. `the output`). In comparison, `Upper bound after transformation` does not explicitly specify this information.
   
   Do you think it would be a bit better to use Spark's description here?
   
   Same for the `min` parameter.

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerParams.java
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.ml.common.param.HasFeaturesCol;
+import org.apache.flink.ml.common.param.HasPredictionCol;
+import org.apache.flink.ml.param.DoubleParam;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+
+/**
+ * Params for {@link MinMaxScaler}.
+ *
+ * @param <T> The class type of this instance.
+ */
+public interface MinMaxScalerParams<T> extends HasFeaturesCol<T>, HasPredictionCol<T> {
+    Param<Double> MAX =
+            new DoubleParam(
+                    "max", "Upper bound after transformation.", 1.0, ParamValidators.notNull());
+
+    default Double getMax() {
+        return get(MAX);
+    }
+
+    default T setMax(Double value) {
+        return set(MAX, value);
+    }
+
+    Param<Double> MIN =

Review comment:
       nits: Could we move the definition of this MIN parameter to be above MAX?
   
   This would be more consistent with the Java coding style, where we typically declare all class member variables before declaring methods.
   
   And it would also make it easier for users to get all the parameters defined in this class.

##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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, 400))));
+    private static final List<Row> predictData =
+            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);
+        trainDataTable = tEnv.fromDataStream(env.fromCollection(trainData)).as("features");
+        predictDataTable = tEnv.fromDataStream(env.fromCollection(predictData)).as("features");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol, DenseVector expected)
+            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(1, result.size());
+        assertEquals(expected, result.get(0));
+    }
+
+    @Test
+    public void testParam() {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        assertEquals("features", minMaxScaler.getFeaturesCol());
+        assertEquals("prediction", minMaxScaler.getPredictionCol());
+        assertEquals(1.0, minMaxScaler.getMax(), 0.0001);
+        assertEquals(0.0, minMaxScaler.getMin(), 0.0001);
+        minMaxScaler
+                .setFeaturesCol("test_features")
+                .setPredictionCol("test_output")
+                .setMax(4.0)

Review comment:
       Super nits: Given that we already use min before max at line 111, any chance we can consistently use `*min*` before `*max*` for a bit extra readability and consistency?
   
   There are a few other places where we use max before min. It will be update them for consistency.
   
   Feel free to leave it as is and I can submit a followup PR to make this change.

##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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, 400))));
+    private static final List<Row> predictData =
+            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);
+        trainDataTable = tEnv.fromDataStream(env.fromCollection(trainData)).as("features");
+        predictDataTable = tEnv.fromDataStream(env.fromCollection(predictData)).as("features");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol, DenseVector expected)
+            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(1, result.size());
+        assertEquals(expected, result.get(0));
+    }
+
+    @Test
+    public void testParam() {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        assertEquals("features", minMaxScaler.getFeaturesCol());
+        assertEquals("prediction", minMaxScaler.getPredictionCol());
+        assertEquals(1.0, minMaxScaler.getMax(), 0.0001);
+        assertEquals(0.0, minMaxScaler.getMin(), 0.0001);
+        minMaxScaler
+                .setFeaturesCol("test_features")
+                .setPredictionCol("test_output")
+                .setMax(4.0)
+                .setMin(1.0);
+        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()
+                        .setFeaturesCol("test_features")
+                        .setPredictionCol("test_output")
+                        .setMin(1.0)
+                        .setMax(4.0);
+
+        MinMaxScalerModel model = minMaxScaler.fit(trainDataTable.as("test_features"));
+        Table output = model.transform(predictDataTable.as("test_features"))[0];
+        assertEquals(
+                Arrays.asList("test_features", "test_output"),
+                output.getResolvedSchema().getColumnNames());
+    }
+
+    @Test
+    public void testFewerSamplesThanParallel() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScalerModel model = minMaxScaler.fit(predictDataTable);
+        Table result = model.transform(predictDataTable)[0];
+        verifyPredictionResult(result, minMaxScaler.getPredictionCol(), Vectors.dense(0.5, 0.5));
+    }
+
+    @Test
+    public void testMaxValueEqualsMinValueButPredictValueNotEquals() throws Exception {
+        List<Row> userDefineData =
+                new ArrayList<>(Collections.singletonList(Row.of(Vectors.dense(30.0, 50.0))));
+        Table userDefineDataTable =
+                tEnv.fromDataStream(env.fromCollection(userDefineData)).as("features");
+        MinMaxScaler minMaxScaler = new MinMaxScaler().setMax(10.0).setMin(0.0);
+        MinMaxScalerModel model = minMaxScaler.fit(predictDataTable);
+        Table result = model.transform(userDefineDataTable)[0];
+        verifyPredictionResult(result, minMaxScaler.getPredictionCol(), Vectors.dense(5.0, 5.0));
+    }
+
+    @Test
+    public void testFitAndPredict() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScalerModel minMaxScalerModel = minMaxScaler.fit(trainDataTable);
+        Table output = minMaxScalerModel.transform(predictDataTable)[0];
+        verifyPredictionResult(output, minMaxScaler.getPredictionCol(), Vectors.dense(0.75, 0.225));
+    }
+
+    @Test
+    public void testSaveLoadAndPredict() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScaler loadedMinMaxScaler =
+                StageTestUtils.saveAndReload(
+                        env, minMaxScaler, tempFolder.newFolder().getAbsolutePath());
+        MinMaxScalerModel minMaxScalerModel = loadedMinMaxScaler.fit(trainDataTable);
+        minMaxScalerModel =
+                StageTestUtils.saveAndReload(
+                        env, minMaxScalerModel, tempFolder.newFolder().getAbsolutePath());
+        assertEquals(
+                Arrays.asList("minVector", "maxVector"),
+                minMaxScalerModel.getModelData()[0].getResolvedSchema().getColumnNames());
+        Table output = minMaxScalerModel.transform(predictDataTable)[0];
+        verifyPredictionResult(output, minMaxScaler.getPredictionCol(), Vectors.dense(0.75, 0.225));
+    }
+
+    @Test
+    public void testGetModelData() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScalerModel minMaxScalerModel = minMaxScaler.fit(trainDataTable);
+        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 =

Review comment:
       Hmm... it is redundant to instantiate a `MinMaxScalerModelData` with the expected min/max vectors and then invoke assertEquals on these two class member variables.
   
   Would it be simpler to do the following directly?
   
   ```
   assertEquals(new DenseVector(new double[] {0.0, 0.0}), modelRows.get(0).getField(0));
   assertEquals(new DenseVector(new double[] {200.0, 400.0}), modelRows.get(0).getField(1));
   ```

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModelData.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.serialization.Encoder;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.src.reader.SimpleStreamFormat;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.linalg.typeinfo.DenseVectorSerializer;
+import org.apache.flink.streaming.api.datastream.DataStream;
+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 java.io.EOFException;
+import java.io.IOException;
+import java.io.OutputStream;
+
+/**
+ * Model data of {@link MinMaxScalerModel}.
+ *
+ * <p>This class also provides methods to convert model data from Table to a data stream, and
+ * classes to save/load model data.
+ */
+public class MinMaxScalerModelData {
+    public DenseVector minVector;
+
+    public DenseVector maxVector;
+
+    public MinMaxScalerModelData() {}
+
+    public MinMaxScalerModelData(DenseVector minVector, DenseVector maxVector) {
+        this.minVector = minVector;
+        this.maxVector = maxVector;
+    }
+
+    /**
+     * Converts the table model to a data stream.
+     *
+     * @param modelDataTable The table model data.
+     * @return The data stream model data.
+     */
+    public static DataStream<MinMaxScalerModelData> getModelDataStream(Table modelDataTable) {
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) modelDataTable).getTableEnvironment();
+        return tEnv.toDataStream(modelDataTable)
+                .map(
+                        x ->
+                                new MinMaxScalerModelData(
+                                        (DenseVector) x.getField(0), (DenseVector) x.getField(1)));
+    }
+
+    /** Encoder for {@link MinMaxScalerModelData}. */
+    public static class ModelDataEncoder implements Encoder<MinMaxScalerModelData> {
+        @Override
+        public void encode(MinMaxScalerModelData minMaxScalerModelData, OutputStream outputStream)

Review comment:
       Super nits: all other classes (except `KnnModelData`) uses `modelData` as the input parameter name of this method. Any chance we can do the same here for consistency and shorten the code a little bit?

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,189 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private final String broadcastKey;
+        private MinMaxScalerModelData minMaxScalerModelData;

Review comment:
       Given that the code already stores `scaleVector` and `offsetVector`, do we still need to keep `minMaxScalerModelData` as a class member variable here?
   
   Same for `minVector`.

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.iteration.operator.OperatorStateUtils;
+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. This algorithm rescales feature values
+ * to a common range [min, max] which defined by user.
+ *
+ * <blockquote>
+ *
+ * $$ Rescaled(value) = \frac{value - E_{min}}{E_{max} - E_{min}} * (max - min) + min $$
+ *
+ * </blockquote>
+ *
+ * <p>For the case \(E_{max} == E_{min}\), \(Rescaled(value) = 0.5 * (max + min)\).
+ *
+ * <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> features =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                (MapFunction<Row, DenseVector>)
+                                        value -> (DenseVector) value.getField(featureCol));
+        DataStream<DenseVector> minMaxValues =
+                features.transform(
+                                "reduceInEachPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .transform(
+                                "reduceInFinalPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .setParallelism(1);
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(
+                        minMaxValues,
+                        new RichMapPartitionFunction<DenseVector, MinMaxScalerModelData>() {
+                            @Override
+                            public void mapPartition(
+                                    Iterable<DenseVector> values,
+                                    Collector<MinMaxScalerModelData> out) {
+                                Iterator<DenseVector> iter = values.iterator();
+                                DenseVector minVector = iter.next();
+                                DenseVector maxVector = iter.next();
+                                out.collect(new MinMaxScalerModelData(minVector, maxVector));
+                            }
+                        });
+
+        MinMaxScalerModel model =
+                new MinMaxScalerModel().setModelData(tEnv.fromDataStream(modelData));
+        ReadWriteUtils.updateExistingParams(model, getParamMap());
+        return model;
+    }
+
+    /**
+     * A stream operator to compute the min and max values in each partition of the input bounded
+     * data stream.
+     */
+    private static class MinMaxReduceFunctionOperator extends AbstractStreamOperator<DenseVector>
+            implements OneInputStreamOperator<DenseVector, DenseVector>, BoundedOneInput {
+        private ListState<DenseVector> minState;
+        private ListState<DenseVector> maxState;
+
+        private DenseVector minVector;
+        private DenseVector maxVector;
+
+        @Override
+        public void endInput() {
+            if (minVector != null) {
+                output.collect(new StreamRecord<>(minVector));
+                output.collect(new StreamRecord<>(maxVector));
+            }
+        }
+
+        @Override
+        public void processElement(StreamRecord<DenseVector> streamRecord) {
+            DenseVector currentValue = streamRecord.getValue();
+            if (minVector == null) {
+                int vecSize = currentValue.size();
+                minVector = new DenseVector(vecSize);
+                maxVector = new DenseVector(vecSize);
+                System.arraycopy(currentValue.values, 0, minVector.values, 0, vecSize);
+                System.arraycopy(currentValue.values, 0, maxVector.values, 0, vecSize);
+            } else {
+                Preconditions.checkArgument(
+                        currentValue.size() == maxVector.size(),
+                        "CurrentValue should has same size with maxVector.");
+                for (int i = 0; i < currentValue.size(); ++i) {
+                    minVector.values[i] = Math.min(minVector.values[i], currentValue.values[i]);
+                    maxVector.values[i] = Math.max(maxVector.values[i], currentValue.values[i]);
+                }
+            }
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            minState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "minState",
+                                            getOperatorConfig()
+                                                    .getTypeSerializerIn(
+                                                            0, getClass().getClassLoader())));
+            maxState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "maxState",
+                                            getOperatorConfig()
+                                                    .getTypeSerializerIn(
+                                                            0, getClass().getClassLoader())));
+
+            OperatorStateUtils.getUniqueElement(minState, "minState").ifPresent(x -> minVector = x);
+            OperatorStateUtils.getUniqueElement(maxState, "maxState").ifPresent(x -> maxVector = x);
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            minState.clear();
+            maxState.clear();
+            if (minVector != null) {
+                minState.add(minVector);
+            }
+            if (maxVector != null) {

Review comment:
       nits: instead of using two `if` statements, would it be simpler to do the following?
   
   ```
   if (minVector != null) {
       minState.add(minVector);
       maxState.add(maxVector);
   }
   ```




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] zhipeng93 closed pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler

Posted by GitBox <gi...@apache.org>.
zhipeng93 closed pull request #54:
URL: https://github.com/apache/flink-ml/pull/54


   


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r823407289



##########
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;
+            this.min = min;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {

Review comment:
       OK
   




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r823410761



##########
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:
       Spark uses setMax() and setMin(), alink also do as spark. min and max has the meaning of lowerBound and upperBound in this algo.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r823401050



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.RichMapPartitionFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.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.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.Map;
+
+/** An Estimator which implements the MinMaxScaler algorithm. */
+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);
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment();
+        DataStream<Tuple2<DenseVector, DenseVector>> minMaxVectors =
+                computeMinMaxVectors(tEnv.toDataStream(inputs[0]), getFeaturesCol());
+        DataStream<MinMaxScalerModelData> modelData = genModelData(minMaxVectors);
+        MinMaxScalerModel model =
+                new MinMaxScalerModel().setModelData(tEnv.fromDataStream(modelData));
+        ReadWriteUtils.updateExistingParams(model, getParamMap());
+        return model;
+    }
+
+    @Override
+    public Map<Param<?>, Object> getParamMap() {
+        return paramMap;
+    }
+
+    @Override
+    public void save(String path) throws IOException {
+        ReadWriteUtils.saveMetadata(this, path);
+    }
+
+    public static MinMaxScaler load(StreamExecutionEnvironment env, String path)
+            throws IOException {
+        return ReadWriteUtils.loadStageParam(path);
+    }
+
+    /**
+     * Generates minMax scaler model data.
+     *
+     * @param minMaxVectors Input distributed minMaxVectors.
+     * @return MinMax scaler model data.
+     */
+    private static DataStream<MinMaxScalerModelData> genModelData(
+            DataStream<Tuple2<DenseVector, DenseVector>> minMaxVectors) {
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(
+                        minMaxVectors,
+                        new RichMapPartitionFunction<
+                                Tuple2<DenseVector, DenseVector>, MinMaxScalerModelData>() {
+                            @Override
+                            public void mapPartition(
+                                    Iterable<Tuple2<DenseVector, DenseVector>> dataPoints,
+                                    Collector<MinMaxScalerModelData> out) {
+                                DenseVector minVector = null;
+                                DenseVector maxVector = null;
+                                int vecSize = 0;
+                                for (Tuple2<DenseVector, DenseVector> dataPoint : dataPoints) {
+                                    if (maxVector == null) {
+                                        vecSize = dataPoint.f0.size();
+                                        maxVector = dataPoint.f1;
+                                        minVector = dataPoint.f0;
+                                    }
+                                    for (int i = 0; i < vecSize; ++i) {
+                                        minVector.values[i] =
+                                                Math.min(
+                                                        dataPoint.f0.values[i],
+                                                        minVector.values[i]);
+                                        maxVector.values[i] =
+                                                Math.max(
+                                                        dataPoint.f1.values[i],
+                                                        maxVector.values[i]);
+                                    }
+                                }
+                                out.collect(new MinMaxScalerModelData(minVector, maxVector));
+                            }
+                        });
+        modelData.getTransformation().setParallelism(1);
+        return modelData;
+    }
+
+    /**
+     * Computes max and min values of features.
+     *
+     * @param inputData Input data.
+     * @param featureCol Feature column name.
+     * @return Max and min values of features.
+     */
+    private DataStream<Tuple2<DenseVector, DenseVector>> computeMinMaxVectors(

Review comment:
       I think it's better.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r826537120



##########
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:
       OK




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r826536751



##########
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:
       OK




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
lindong28 commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829647411



##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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(trainData);
+        trainDataTable = tEnv.fromDataStream(dataStream, schema).as("features");
+        DataStream<Row> predDataStream = env.fromCollection(predictRows);
+        predictDataTable = tEnv.fromDataStream(predDataStream, schema).as("features");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol, DenseVector expected)
+            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(1, result.size());
+        assertEquals(expected, 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(trainDataTable.as("test_features"));
+        Table output = model.transform(predictDataTable.as("test_features"))[0];
+        assertEquals(
+                Arrays.asList("test_features", "test_output"),
+                output.getResolvedSchema().getColumnNames());
+    }
+
+    @Test
+    public void testFewerDistinctPointsThanCluster() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScalerModel model = minMaxScaler.fit(predictDataTable);
+        Table result = model.transform(predictDataTable)[0];
+        verifyPredictionResult(result, minMaxScaler.getPredictionCol(), Vectors.dense(0.5, 0.5));
+    }
+
+    @Test
+    public void testFitAndPredict() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScalerModel minMaxScalerModel = minMaxScaler.fit(trainDataTable);
+        Table output = minMaxScalerModel.transform(predictDataTable)[0];
+        verifyPredictionResult(output, minMaxScaler.getPredictionCol(), Vectors.dense(0.75, 0.3));
+    }
+
+    @Test
+    public void testSaveLoadAndPredict() throws Exception {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        MinMaxScaler loadedMinMaxScaler =
+                StageTestUtils.saveAndReload(
+                        env, minMaxScaler, tempFolder.newFolder().getAbsolutePath());
+        MinMaxScalerModel minMaxScalerModel = loadedMinMaxScaler.fit(trainDataTable);
+        minMaxScalerModel =
+                StageTestUtils.saveAndReload(
+                        env, minMaxScalerModel, tempFolder.newFolder().getAbsolutePath());
+        assertEquals(
+                Arrays.asList("minVector", "maxVector"),
+                minMaxScalerModel.getModelData()[0].getResolvedSchema().getColumnNames());
+        Table output = minMaxScalerModel.transform(predictDataTable)[0];
+        verifyPredictionResult(output, minMaxScaler.getPredictionCol(), Vectors.dense(0.75, 0.3));
+    }
+
+    @Test
+    public void testModelSaveLoadAndPredict() throws Exception {

Review comment:
       It seems that all functionalities covered in this test is already covered by `testSaveLoadAndPredict`. Could we remove this test for simplicity?

##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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(trainData);
+        trainDataTable = tEnv.fromDataStream(dataStream, schema).as("features");
+        DataStream<Row> predDataStream = env.fromCollection(predictRows);
+        predictDataTable = tEnv.fromDataStream(predDataStream, schema).as("features");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol, DenseVector expected)
+            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(1, result.size());
+        assertEquals(expected, 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());

Review comment:
       nits: could we move `prediction` to be right after "features" for a bit more consistency, i.e. column name features are close to each other?
   
   Same for other `assertEquals` usages and `testFeaturePredictionParam()`.

##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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(trainData);
+        trainDataTable = tEnv.fromDataStream(dataStream, schema).as("features");
+        DataStream<Row> predDataStream = env.fromCollection(predictRows);

Review comment:
       It appears that we don't need to explicitly create the schema. Would it be simpler to use the code below?
   
   ```
           trainDataTable = tEnv.fromDataStream(env.fromCollection(trainData)).as("features");
           predictDataTable = tEnv.fromDataStream(env.fromCollection(predictRows)).as("features");
   ```

##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,208 @@
+/*
+ * 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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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 =

Review comment:
       Could we have multiple elements instead of just one element in the prediction stream? In general this could help provide more test coverage.
   
   And could we rename `predictRows` as `predictData` for consistency with `trainData`?




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829675971



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.iteration.operator.OperatorStateUtils;
+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> features =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                (MapFunction<Row, DenseVector>)
+                                        value -> (DenseVector) value.getField(featureCol));
+        DataStream<DenseVector> minMaxValues =
+                features.transform(
+                                "reduceInEachPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .transform(
+                                "reduceInFinalPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .setParallelism(1);
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(
+                        minMaxValues,
+                        new RichMapPartitionFunction<DenseVector, MinMaxScalerModelData>() {
+                            @Override
+                            public void mapPartition(
+                                    Iterable<DenseVector> values,
+                                    Collector<MinMaxScalerModelData> out) {
+                                Iterator<DenseVector> iter = values.iterator();
+                                DenseVector minVector = iter.next();
+                                DenseVector maxVector = iter.next();
+                                out.collect(new MinMaxScalerModelData(minVector, maxVector));
+                            }
+                        });
+
+        MinMaxScalerModel model =
+                new MinMaxScalerModel().setModelData(tEnv.fromDataStream(modelData));
+        ReadWriteUtils.updateExistingParams(model, getParamMap());
+        return model;
+    }
+
+    /**
+     * A stream operator to compute the min and max values in each partition of the input bounded
+     * data stream.
+     */
+    private static class MinMaxReduceFunctionOperator extends AbstractStreamOperator<DenseVector>
+            implements OneInputStreamOperator<DenseVector, DenseVector>, BoundedOneInput {
+        private ListState<DenseVector> minState;
+        private ListState<DenseVector> maxState;
+
+        private DenseVector minVector;
+        private DenseVector maxVector;
+
+        @Override
+        public void endInput() {
+            if (minVector != null) {

Review comment:
       OK




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r828829587



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,197 @@
+/*
+ * 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> features =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                (MapFunction<Row, DenseVector>)
+                                        value -> (DenseVector) value.getField(featureCol));
+        DataStream<DenseVector> minMaxValues =
+                features.transform(
+                                "reduceInEachPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .transform(
+                                "reduceInFinalPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .setParallelism(1);
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(
+                        minMaxValues,
+                        new RichMapPartitionFunction<DenseVector, MinMaxScalerModelData>() {
+                            @Override
+                            public void mapPartition(
+                                    Iterable<DenseVector> values,
+                                    Collector<MinMaxScalerModelData> out) {
+                                Iterator<DenseVector> iter = values.iterator();
+                                DenseVector minVector = iter.next();
+                                DenseVector maxVector = iter.next();
+                                out.collect(new MinMaxScalerModelData(minVector, maxVector));
+                            }
+                        });
+
+        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> minState;
+        private ListState<DenseVector> maxState;
+
+        private DenseVector minVector;
+        private DenseVector maxVector;
+
+        @Override
+        public void endInput() {
+            output.collect(new StreamRecord<>(minVector));

Review comment:
       OK




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829000929



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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> features =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                (MapFunction<Row, DenseVector>)
+                                        value -> (DenseVector) value.getField(featureCol));
+        DataStream<DenseVector> minMaxValues =
+                features.transform(
+                                "reduceInEachPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .transform(
+                                "reduceInFinalPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .setParallelism(1);
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(
+                        minMaxValues,
+                        new RichMapPartitionFunction<DenseVector, MinMaxScalerModelData>() {
+                            @Override
+                            public void mapPartition(
+                                    Iterable<DenseVector> values,
+                                    Collector<MinMaxScalerModelData> out) {
+                                Iterator<DenseVector> iter = values.iterator();
+                                DenseVector minVector = iter.next();
+                                DenseVector maxVector = iter.next();
+                                out.collect(new MinMaxScalerModelData(minVector, maxVector));
+                            }
+                        });
+
+        MinMaxScalerModel model =
+                new MinMaxScalerModel().setModelData(tEnv.fromDataStream(modelData));
+        ReadWriteUtils.updateExistingParams(model, getParamMap());
+        return model;
+    }
+
+    /**
+     * A stream operator to compute the min and max values in each partition of the input bounded
+     * data stream.
+     */
+    private static class MinMaxReduceFunctionOperator extends AbstractStreamOperator<DenseVector>
+            implements OneInputStreamOperator<DenseVector, DenseVector>, BoundedOneInput {
+        private ListState<DenseVector> minState;
+        private ListState<DenseVector> maxState;
+
+        private DenseVector minVector;
+        private DenseVector maxVector;
+
+        @Override
+        public void endInput() {
+            if (minVector != null) {
+                output.collect(new StreamRecord<>(minVector));
+            }
+            if (maxVector != null) {
+                output.collect(new StreamRecord<>(maxVector));
+            }
+        }
+
+        @Override
+        public void processElement(StreamRecord<DenseVector> streamRecord) {
+            DenseVector currentValue = streamRecord.getValue();
+            if (minVector == null) {
+                int vecSize = currentValue.size();
+                minVector = new DenseVector(vecSize);
+                maxVector = new DenseVector(vecSize);
+                System.arraycopy(currentValue.values, 0, minVector.values, 0, vecSize);
+                System.arraycopy(currentValue.values, 0, maxVector.values, 0, vecSize);
+
+            } else {
+                for (int i = 0; i < currentValue.size(); ++i) {
+                    minVector.values[i] = Math.min(minVector.values[i], currentValue.values[i]);
+                    maxVector.values[i] = Math.max(maxVector.values[i], currentValue.values[i]);
+                }
+            }
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            minState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "minState",
+                                            getOperatorConfig()
+                                                    .getTypeSerializerIn(
+                                                            0, getClass().getClassLoader())));
+            maxState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "maxState",
+                                            getOperatorConfig()
+                                                    .getTypeSerializerIn(
+                                                            0, getClass().getClassLoader())));
+            Iterator<DenseVector> minIterator = minState.get().iterator();
+            Iterator<DenseVector> maxIterator = maxState.get().iterator();
+            if (minIterator.hasNext()) {

Review comment:
       OK, I will refine it later.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829680935



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {

Review comment:
       When maxVector[i] == minVector[i],
   If featureValue[i] != maxVector[i], above code may not get the same result as original code.
   In read world, data for train and predict are different, featureValue[i] != maxVector[i] is existed. 




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler in FlinkML

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r829847006



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScalerModel.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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(), getPredictionCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, minMaxScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new PredictOutputFunction(
+                                            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 PredictOutputFunction extends RichMapFunction<Row, Row> {
+        private final String featureCol;
+        private MinMaxScalerModelData minMaxScalerModelData;
+        private final double upperBound;
+        private final double lowerBound;
+        private final String broadcastKey;
+        private DenseVector maxVector;
+        private DenseVector minVector;
+
+        public PredictOutputFunction(
+                String broadcastKey, double upperBound, double lowerBound, String featureCol) {
+            this.upperBound = upperBound;
+            this.lowerBound = lowerBound;
+            this.broadcastKey = broadcastKey;
+            this.featureCol = featureCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (minMaxScalerModelData == null) {
+                minMaxScalerModelData =
+                        (MinMaxScalerModelData)
+                                getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                maxVector = minMaxScalerModelData.maxVector;
+                minVector = minMaxScalerModelData.minVector;
+            }
+            DenseVector feature = (DenseVector) row.getField(featureCol);
+            DenseVector outputVector = new DenseVector(maxVector.size());
+            if (feature != null) {
+                for (int i = 0; i < maxVector.size(); ++i) {

Review comment:
       Besides, could you please add a test case about maxVector[i] == minVector[i] && featureValue[i] != maxVector[i]? I think it would be helpful to test these corner cases.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r831826228



##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,206 @@
+/*
+ * 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());
+        for (DenseVector t2 : result) {
+            assertEquals(Vectors.dense(0.75, 0.3), t2);
+        }
+    }
+
+    @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("output", minMaxScaler.getOutputCol());
+        minMaxScaler
+                .setFeaturesCol("test_features")
+                .setMax(4.0)
+                .setMIN(0.0)
+                .setOutputCol("test_output");
+        assertEquals("test_features", minMaxScaler.getFeaturesCol());
+        assertEquals(0.0, minMaxScaler.getMIN(), 0.0001);
+        assertEquals(4.0, minMaxScaler.getMax(), 0.0001);
+        assertEquals("test_output", minMaxScaler.getOutputCol());
+    }
+
+    @Test
+    public void testFeaturePredictionParam() {
+        MinMaxScaler minMaxScaler =
+                new MinMaxScaler()
+                        .setMIN(0.0)
+                        .setMax(4.0)
+                        .setFeaturesCol("test_features")
+                        .setOutputCol("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() {

Review comment:
       done




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r831699855



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/minmaxscaler/MinMaxScaler.java
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.iteration.operator.OperatorStateUtils;
+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. This algorithm rescales feature values
+ * to a common range [min, max] which defined by user.
+ *
+ * <blockquote>
+ *
+ * $$ Rescaled(value) = \frac{value - E_{min}}{E_{max} - E_{min}} * (max - min) + min $$
+ *
+ * </blockquote>
+ *
+ * <p>For the case \(E_{max} == E_{min}\), \(Rescaled(value) = 0.5 * (max + min)\).
+ *
+ * <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> features =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                (MapFunction<Row, DenseVector>)
+                                        value -> (DenseVector) value.getField(featureCol));
+        DataStream<DenseVector> minMaxValues =
+                features.transform(
+                                "reduceInEachPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .transform(
+                                "reduceInFinalPartition",
+                                features.getType(),
+                                new MinMaxReduceFunctionOperator())
+                        .setParallelism(1);
+        DataStream<MinMaxScalerModelData> modelData =
+                DataStreamUtils.mapPartition(
+                        minMaxValues,
+                        new RichMapPartitionFunction<DenseVector, MinMaxScalerModelData>() {
+                            @Override
+                            public void mapPartition(
+                                    Iterable<DenseVector> values,
+                                    Collector<MinMaxScalerModelData> out) {
+                                Iterator<DenseVector> iter = values.iterator();
+                                DenseVector minVector = iter.next();
+                                DenseVector maxVector = iter.next();
+                                out.collect(new MinMaxScalerModelData(minVector, maxVector));
+                            }
+                        });
+
+        MinMaxScalerModel model =
+                new MinMaxScalerModel().setModelData(tEnv.fromDataStream(modelData));
+        ReadWriteUtils.updateExistingParams(model, getParamMap());
+        return model;
+    }
+
+    /**
+     * A stream operator to compute the min and max values in each partition of the input bounded
+     * data stream.
+     */
+    private static class MinMaxReduceFunctionOperator extends AbstractStreamOperator<DenseVector>
+            implements OneInputStreamOperator<DenseVector, DenseVector>, BoundedOneInput {
+        private ListState<DenseVector> minState;
+        private ListState<DenseVector> maxState;
+
+        private DenseVector minVector;
+        private DenseVector maxVector;
+
+        @Override
+        public void endInput() {
+            if (minVector != null) {
+                output.collect(new StreamRecord<>(minVector));
+                output.collect(new StreamRecord<>(maxVector));
+            }
+        }
+
+        @Override
+        public void processElement(StreamRecord<DenseVector> streamRecord) {
+            DenseVector currentValue = streamRecord.getValue();
+            if (minVector == null) {
+                int vecSize = currentValue.size();
+                minVector = new DenseVector(vecSize);
+                maxVector = new DenseVector(vecSize);
+                System.arraycopy(currentValue.values, 0, minVector.values, 0, vecSize);
+                System.arraycopy(currentValue.values, 0, maxVector.values, 0, vecSize);
+            } else {
+                Preconditions.checkArgument(
+                        currentValue.size() == maxVector.size(),
+                        "CurrentValue should has same size with maxVector.");
+                for (int i = 0; i < currentValue.size(); ++i) {
+                    minVector.values[i] = Math.min(minVector.values[i], currentValue.values[i]);
+                    maxVector.values[i] = Math.max(maxVector.values[i], currentValue.values[i]);
+                }
+            }
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            minState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "minState",
+                                            getOperatorConfig()
+                                                    .getTypeSerializerIn(
+                                                            0, getClass().getClassLoader())));
+            maxState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "maxState",
+                                            getOperatorConfig()
+                                                    .getTypeSerializerIn(
+                                                            0, getClass().getClassLoader())));
+
+            OperatorStateUtils.getUniqueElement(minState, "minState").ifPresent(x -> minVector = x);
+            OperatorStateUtils.getUniqueElement(maxState, "maxState").ifPresent(x -> maxVector = x);
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            minState.clear();
+            maxState.clear();
+            if (minVector != null) {
+                minState.add(minVector);
+            }
+            if (maxVector != null) {

Review comment:
       OK




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] weibozhao commented on a change in pull request #54: [FLINK-25552] Add Estimator and Transformer for MinMaxScaler

Posted by GitBox <gi...@apache.org>.
weibozhao commented on a change in pull request #54:
URL: https://github.com/apache/flink-ml/pull/54#discussion_r831702806



##########
File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MinMaxScalerTest.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.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 trainDataTable;
+    private Table predictDataTable;
+    private static final List<Row> trainData =
+            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, 400))));
+    private static final List<Row> predictData =
+            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);
+        trainDataTable = tEnv.fromDataStream(env.fromCollection(trainData)).as("features");
+        predictDataTable = tEnv.fromDataStream(env.fromCollection(predictData)).as("features");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol, DenseVector expected)
+            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(1, result.size());
+        assertEquals(expected, result.get(0));
+    }
+
+    @Test
+    public void testParam() {
+        MinMaxScaler minMaxScaler = new MinMaxScaler();
+        assertEquals("features", minMaxScaler.getFeaturesCol());
+        assertEquals("prediction", minMaxScaler.getPredictionCol());
+        assertEquals(1.0, minMaxScaler.getMax(), 0.0001);
+        assertEquals(0.0, minMaxScaler.getMin(), 0.0001);
+        minMaxScaler
+                .setFeaturesCol("test_features")
+                .setPredictionCol("test_output")
+                .setMax(4.0)

Review comment:
       OK, I will check this cases and refine them.




-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org