yunfengzhou-hub commented on code in PR #172:
URL: https://github.com/apache/flink-ml/pull/172#discussion_r1022233623


##########
flink-ml-lib/src/test/java/org/apache/flink/ml/feature/RobustScalerTest.java:
##########
@@ -0,0 +1,318 @@
+/*
+ * 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.robustscaler.RobustScaler;
+import org.apache.flink.ml.feature.robustscaler.RobustScalerModel;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.linalg.SparseVector;
+import org.apache.flink.ml.linalg.Vectors;
+import org.apache.flink.ml.util.TestUtils;
+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.Expressions;
+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.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+
+import org.apache.commons.collections.IteratorUtils;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+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.List;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/** Tests {@link RobustScaler} and {@link RobustScalerModel}. */
+public class RobustScalerTest extends AbstractTestBase {
+    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+    private StreamExecutionEnvironment env;
+    private StreamTableEnvironment tEnv;
+    private Table trainDataTable;
+    private Table predictDataTable;
+
+    private static final List<Row> TRAIN_DATA =
+            new ArrayList<>(
+                    Arrays.asList(
+                            Row.of(0, Vectors.dense(0.0, 0.0)),
+                            Row.of(1, Vectors.dense(1.0, -1.0)),
+                            Row.of(2, Vectors.dense(2.0, -2.0)),
+                            Row.of(3, Vectors.dense(3.0, -3.0)),
+                            Row.of(4, Vectors.dense(4.0, -4.0)),
+                            Row.of(5, Vectors.dense(5.0, -5.0)),
+                            Row.of(6, Vectors.dense(6.0, -6.0)),
+                            Row.of(7, Vectors.dense(7.0, -7.0)),
+                            Row.of(8, Vectors.dense(8.0, -8.0))));
+    private static final List<Row> PREDICT_DATA =
+            new ArrayList<>(
+                    Arrays.asList(
+                            Row.of(Vectors.dense(3.0, -3.0)),
+                            Row.of(Vectors.dense(6.0, -6.0)),
+                            Row.of(Vectors.dense(99.0, -99.0))));
+    private static final double EPS = 1.0e-5;
+
+    private static final List<DenseVector> EXPECTED_OUTPUT =
+            new ArrayList<>(
+                    Arrays.asList(
+                            Vectors.dense(0.75, -0.75),
+                            Vectors.dense(1.5, -1.5),
+                            Vectors.dense(24.75, -24.75)));
+
+    @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(TRAIN_DATA)).as("id", "input");
+        predictDataTable = 
tEnv.fromDataStream(env.fromCollection(PREDICT_DATA)).as("input");
+    }
+
+    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());
+        compareResultCollections(expected, result, TestUtils::compare);
+    }
+
+    @Test
+    public void testParam() {
+        RobustScaler robustScaler = new RobustScaler();
+        assertEquals("input", robustScaler.getInputCol());
+        assertEquals("output", robustScaler.getOutputCol());
+        assertEquals(0.25, robustScaler.getLower(), EPS);
+        assertEquals(0.75, robustScaler.getUpper(), EPS);
+        assertEquals(0.001, robustScaler.getRelativeError(), EPS);
+        assertFalse(robustScaler.getWithCentering());
+        assertTrue(robustScaler.getWithScaling());
+
+        robustScaler
+                .setInputCol("test_input")
+                .setOutputCol("test_output")
+                .setLower(0.1)
+                .setUpper(0.9)
+                .setRelativeError(0.01)
+                .setWithCentering(true)
+                .setWithScaling(false);
+        assertEquals("test_input", robustScaler.getInputCol());
+        assertEquals("test_output", robustScaler.getOutputCol());
+        assertEquals(0.1, robustScaler.getLower(), EPS);
+        assertEquals(0.9, robustScaler.getUpper(), EPS);
+        assertEquals(0.01, robustScaler.getRelativeError(), EPS);
+        assertTrue(robustScaler.getWithCentering());
+        assertFalse(robustScaler.getWithScaling());
+    }
+
+    @Test
+    public void testOutputSchema() {
+        RobustScaler robustScaler = new RobustScaler();
+        RobustScalerModel model = robustScaler.fit(trainDataTable);
+        Table output = model.transform(trainDataTable)[0];
+        assertEquals(
+                Arrays.asList("id", "input", "output"),

Review Comment:
   Let's test against non-default column values.



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/feature/ImputerTest.java:
##########
@@ -175,9 +175,10 @@ public void testParam() {
         assertEquals(Double.NaN, imputer.getMissingValue(), EPS);
         assertEquals(0.001, imputer.getRelativeError(), EPS);
 
-        imputer.setMissingValue(0.0).setStrategy(MEDIAN);
+        imputer.setMissingValue(0.0).setStrategy(MEDIAN).setRelativeError(0.1);

Review Comment:
   Let's add test for all parameters of `Imputer`, same as other `testParam` 
methods.



##########
flink-ml-python/pyflink/ml/lib/feature/tests/test_robustscaler.py:
##########
@@ -0,0 +1,135 @@
+################################################################################
+#  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.
+################################################################################
+from typing import List
+
+from pyflink.common import Types
+from pyflink.ml.tests.test_utils import PyFlinkMLTestCase
+
+from pyflink.ml.core.linalg import Vectors, DenseVectorTypeInfo, DenseVector
+
+from pyflink.ml.lib.feature.robustscaler import RobustScaler
+from pyflink.table import Table
+
+
+class RobustScalerTest(PyFlinkMLTestCase):
+
+    def setUp(self):
+        super(RobustScalerTest, self).setUp()
+        self.train_table = self.t_env.from_data_stream(
+            self.env.from_collection([
+                (1, Vectors.dense(0.0, 0.0),),
+                (2, Vectors.dense(1.0, -1.0),),
+                (3, Vectors.dense(2.0, -2.0),),
+                (4, Vectors.dense(3.0, -3.0),),
+                (5, Vectors.dense(4.0, -4.0),),
+                (6, Vectors.dense(5.0, -5.0),),
+                (7, Vectors.dense(6.0, -6.0),),
+                (8, Vectors.dense(7.0, -7.0),),
+                (9, Vectors.dense(8.0, -8.0),),
+            ],
+                type_info=Types.ROW_NAMED(
+                    ['id', 'input'],
+                    [Types.INT(), DenseVectorTypeInfo()])
+            ))
+
+        self.predict_table = self.t_env.from_data_stream(
+            self.env.from_collection([
+                (Vectors.dense(3.0, -3.0),),
+                (Vectors.dense(6.0, -6.0),),
+                (Vectors.dense(99.0, -99.0),),
+            ],
+                type_info=Types.ROW_NAMED(
+                    ['input'],
+                    [DenseVectorTypeInfo()])
+            ))
+
+        self.expected_output = [
+            Vectors.dense(0.75, -0.75),
+            Vectors.dense(1.5, -1.5),
+            Vectors.dense(24.75, -24.75)]
+
+    def test_param(self):
+        robust_scaler = RobustScaler()
+        self.assertEqual("input", robust_scaler.input_col)
+        self.assertEqual("output", robust_scaler.output_col)
+        self.assertEqual(0.25, robust_scaler.lower)
+        self.assertEqual(0.75, robust_scaler.upper)
+        self.assertEqual(0.001, robust_scaler.relative_error)
+        self.assertFalse(robust_scaler.with_centering)
+        self.assertTrue(robust_scaler.with_scaling)
+
+        robust_scaler\
+            .set_input_col("test_input")\
+            .set_output_col("test_output")\
+            .set_lower(0.1)\
+            .set_upper(0.9)\
+            .set_relative_error(0.01)\
+            .set_with_centering(True)\
+            .set_with_scaling(False)
+
+        self.assertEqual("test_input", robust_scaler.input_col)
+        self.assertEqual("test_output", robust_scaler.output_col)
+        self.assertEqual(0.1, robust_scaler.lower)
+        self.assertEqual(0.9, robust_scaler.upper)
+        self.assertEqual(0.01, robust_scaler.relative_error)
+        self.assertTrue(robust_scaler.with_centering)
+        self.assertFalse(robust_scaler.with_scaling)
+
+    def test_output_schema(self):
+        robust_scaler = RobustScaler()
+        model = robust_scaler.fit(self.train_table)
+        output = model.transform(self.predict_table.alias('test_input'))[0]
+        self.assertEqual(
+            ['test_input', 'output'],

Review Comment:
   Let's test non-default output column names as well, and use `set_input_col` 
and `set_output_col` to specify the non-default values.



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/feature/RobustScalerTest.java:
##########
@@ -0,0 +1,318 @@
+/*
+ * 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.robustscaler.RobustScaler;
+import org.apache.flink.ml.feature.robustscaler.RobustScalerModel;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.linalg.SparseVector;
+import org.apache.flink.ml.linalg.Vectors;
+import org.apache.flink.ml.util.TestUtils;
+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.Expressions;
+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.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+
+import org.apache.commons.collections.IteratorUtils;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+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.List;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/** Tests {@link RobustScaler} and {@link RobustScalerModel}. */
+public class RobustScalerTest extends AbstractTestBase {
+    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+    private StreamExecutionEnvironment env;
+    private StreamTableEnvironment tEnv;
+    private Table trainDataTable;
+    private Table predictDataTable;
+
+    private static final List<Row> TRAIN_DATA =
+            new ArrayList<>(
+                    Arrays.asList(
+                            Row.of(0, Vectors.dense(0.0, 0.0)),
+                            Row.of(1, Vectors.dense(1.0, -1.0)),
+                            Row.of(2, Vectors.dense(2.0, -2.0)),
+                            Row.of(3, Vectors.dense(3.0, -3.0)),
+                            Row.of(4, Vectors.dense(4.0, -4.0)),
+                            Row.of(5, Vectors.dense(5.0, -5.0)),
+                            Row.of(6, Vectors.dense(6.0, -6.0)),
+                            Row.of(7, Vectors.dense(7.0, -7.0)),
+                            Row.of(8, Vectors.dense(8.0, -8.0))));
+    private static final List<Row> PREDICT_DATA =
+            new ArrayList<>(
+                    Arrays.asList(
+                            Row.of(Vectors.dense(3.0, -3.0)),
+                            Row.of(Vectors.dense(6.0, -6.0)),
+                            Row.of(Vectors.dense(99.0, -99.0))));
+    private static final double EPS = 1.0e-5;
+
+    private static final List<DenseVector> EXPECTED_OUTPUT =
+            new ArrayList<>(
+                    Arrays.asList(
+                            Vectors.dense(0.75, -0.75),
+                            Vectors.dense(1.5, -1.5),
+                            Vectors.dense(24.75, -24.75)));
+
+    @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(TRAIN_DATA)).as("id", "input");
+        predictDataTable = 
tEnv.fromDataStream(env.fromCollection(PREDICT_DATA)).as("input");
+    }
+
+    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());
+        compareResultCollections(expected, result, TestUtils::compare);
+    }
+
+    @Test
+    public void testParam() {
+        RobustScaler robustScaler = new RobustScaler();
+        assertEquals("input", robustScaler.getInputCol());
+        assertEquals("output", robustScaler.getOutputCol());
+        assertEquals(0.25, robustScaler.getLower(), EPS);
+        assertEquals(0.75, robustScaler.getUpper(), EPS);
+        assertEquals(0.001, robustScaler.getRelativeError(), EPS);
+        assertFalse(robustScaler.getWithCentering());
+        assertTrue(robustScaler.getWithScaling());
+
+        robustScaler
+                .setInputCol("test_input")
+                .setOutputCol("test_output")
+                .setLower(0.1)
+                .setUpper(0.9)
+                .setRelativeError(0.01)
+                .setWithCentering(true)
+                .setWithScaling(false);
+        assertEquals("test_input", robustScaler.getInputCol());
+        assertEquals("test_output", robustScaler.getOutputCol());
+        assertEquals(0.1, robustScaler.getLower(), EPS);
+        assertEquals(0.9, robustScaler.getUpper(), EPS);
+        assertEquals(0.01, robustScaler.getRelativeError(), EPS);
+        assertTrue(robustScaler.getWithCentering());
+        assertFalse(robustScaler.getWithScaling());
+    }
+
+    @Test
+    public void testOutputSchema() {
+        RobustScaler robustScaler = new RobustScaler();
+        RobustScalerModel model = robustScaler.fit(trainDataTable);
+        Table output = model.transform(trainDataTable)[0];
+        assertEquals(
+                Arrays.asList("id", "input", "output"),
+                output.getResolvedSchema().getColumnNames());
+    }
+
+    @Test
+    public void testFitAndPredict() throws Exception {
+        RobustScaler robustScaler = new RobustScaler();
+        RobustScalerModel model = robustScaler.fit(trainDataTable);
+        Table output = model.transform(predictDataTable)[0];
+        verifyPredictionResult(output, robustScaler.getOutputCol(), 
EXPECTED_OUTPUT);
+    }
+
+    @Test
+    public void testInputTypeConversion() throws Exception {
+        trainDataTable =
+                TestUtils.convertDataTypesToSparseInt(
+                        tEnv, trainDataTable.select(Expressions.$("input")));
+        predictDataTable = TestUtils.convertDataTypesToSparseInt(tEnv, 
predictDataTable);
+        assertArrayEquals(
+                new Class<?>[] {SparseVector.class}, 
TestUtils.getColumnDataTypes(trainDataTable));
+        assertArrayEquals(
+                new Class<?>[] {SparseVector.class},
+                TestUtils.getColumnDataTypes(predictDataTable));
+
+        RobustScaler robustScaler = new RobustScaler();
+        RobustScalerModel model = robustScaler.fit(trainDataTable);
+        Table output = model.transform(predictDataTable)[0];
+        verifyPredictionResult(output, robustScaler.getOutputCol(), 
EXPECTED_OUTPUT);
+    }
+
+    @Test
+    public void testSaveLoadAndPredict() throws Exception {
+        RobustScaler robustScaler = new RobustScaler();
+        RobustScaler loadedRobustScaler =
+                TestUtils.saveAndReload(
+                        tEnv, robustScaler, 
tempFolder.newFolder().getAbsolutePath());
+        RobustScalerModel model = loadedRobustScaler.fit(trainDataTable);
+        RobustScalerModel loadedModel =
+                TestUtils.saveAndReload(tEnv, model, 
tempFolder.newFolder().getAbsolutePath());
+        assertEquals(
+                Arrays.asList("medians", "ranges"),
+                model.getModelData()[0].getResolvedSchema().getColumnNames());
+        Table output = loadedModel.transform(predictDataTable)[0];
+        verifyPredictionResult(output, robustScaler.getOutputCol(), 
EXPECTED_OUTPUT);
+    }
+
+    @Test
+    public void testFitOnEmptyData() {
+        Table emptyTable =
+                tEnv.fromDataStream(env.fromCollection(TRAIN_DATA).filter(x -> 
x.getArity() == 0))
+                        .as("id", "input");
+        RobustScaler robustScaler = new RobustScaler();
+        RobustScalerModel model = robustScaler.fit(emptyTable);
+        Table modelDataTable = model.getModelData()[0];
+        try {
+            modelDataTable.execute().print();
+            fail();
+        } catch (Throwable e) {
+            assertEquals("The training set is empty.", 
ExceptionUtils.getRootCause(e).getMessage());
+        }
+    }
+
+    @Test
+    public void testWithCentering() throws Exception {
+        RobustScaler robustScaler = new RobustScaler().setWithCentering(true);
+        RobustScalerModel model = robustScaler.fit(trainDataTable);
+        Table output = model.transform(predictDataTable)[0];
+        List<DenseVector> expectedOutput =
+                new ArrayList<>(
+                        Arrays.asList(
+                                Vectors.dense(-0.25, 0.25),
+                                Vectors.dense(0.5, -0.5),
+                                Vectors.dense(23.75, -23.75)));
+        verifyPredictionResult(output, robustScaler.getOutputCol(), 
expectedOutput);
+    }
+
+    @Test
+    public void testWithoutScaling() throws Exception {
+        RobustScaler robustScaler = new 
RobustScaler().setWithCentering(true).setWithScaling(false);
+        RobustScalerModel model = robustScaler.fit(trainDataTable);
+        Table output = model.transform(predictDataTable)[0];
+        List<DenseVector> expectedOutput =
+                new ArrayList<>(
+                        Arrays.asList(
+                                Vectors.dense(-1, 1),
+                                Vectors.dense(2, -2),
+                                Vectors.dense(95, -95)));
+        verifyPredictionResult(output, robustScaler.getOutputCol(), 
expectedOutput);
+    }
+
+    @Test
+    public void testIncompatibleNumOfFeatures() {
+        RobustScaler robustScaler = new RobustScaler();
+        RobustScalerModel model = robustScaler.fit(trainDataTable);
+
+        List<Row> predictData =
+                new ArrayList<>(
+                        Arrays.asList(
+                                Row.of(Vectors.dense(1.0, 2.0, 3.0)),
+                                Row.of(Vectors.dense(-1.0, -2.0, -3.0))));
+        Table predictTable = 
tEnv.fromDataStream(env.fromCollection(predictData)).as("input");
+        Table output = model.transform(predictTable)[0];
+        try {
+            output.execute().print();
+            fail();
+        } catch (Throwable e) {
+            assertTrue(
+                    ExceptionUtils.getRootCause(e)
+                            .getMessage()
+                            .contains("Number of features must be"));
+        }
+    }
+
+    @Test
+    public void testZeroRange() throws Exception {
+        List<Row> trainData =
+                new ArrayList<>(
+                        Arrays.asList(
+                                Row.of(0, Vectors.dense(0.0, 0.0)),
+                                Row.of(1, Vectors.dense(1.0, 1.0)),
+                                Row.of(2, Vectors.dense(1.0, 1.0)),
+                                Row.of(3, Vectors.dense(1.0, 1.0)),
+                                Row.of(4, Vectors.dense(4.0, 4.0))));
+        List<DenseVector> expectedOutput =
+                new ArrayList<>(
+                        Arrays.asList(
+                                Vectors.dense(0.0, -0.0),
+                                Vectors.dense(0.0, -0.0),
+                                Vectors.dense(0.0, -0.0)));
+        Table trainTable = 
tEnv.fromDataStream(env.fromCollection(trainData)).as("id", "input");
+        RobustScaler robustScaler = new RobustScaler();
+        RobustScalerModel model = robustScaler.fit(trainTable);
+        Table output = model.transform(predictDataTable)[0];
+        verifyPredictionResult(output, robustScaler.getOutputCol(), 
expectedOutput);
+    }
+
+    @Test
+    public void testGetModelData() throws Exception {
+        RobustScaler robustScaler = new RobustScaler();
+        RobustScalerModel model = robustScaler.fit(trainDataTable);
+        Table modelData = model.getModelData()[0];
+        assertEquals(
+                Arrays.asList("medians", "ranges"), 
modelData.getResolvedSchema().getColumnNames());
+        DataStream<Row> output = tEnv.toDataStream(modelData);
+        List<Row> modelRows = IteratorUtils.toList(output.executeAndCollect());
+        DenseVector medians = (DenseVector) modelRows.get(0).getField(0);
+        DenseVector ranges = (DenseVector) modelRows.get(0).getField(1);
+
+        double[] expectedMedians = {4.0, -4.0};
+        double[] expectedRanges = {4.0, 4.0};
+        for (int i = 0; i < expectedMedians.length; i++) {
+            assertEquals(expectedMedians[i], medians.get(i), EPS);

Review Comment:
   `assertArrayEquals` might be better.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/feature/robustscaler/RobustScalerModel.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.robustscaler;
+
+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.BLAS;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.linalg.Vector;
+import org.apache.flink.ml.linalg.typeinfo.VectorTypeInfo;
+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.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.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+/** A Model which transforms data using the model data computed by {@link 
RobustScaler}. */
+public class RobustScalerModel
+        implements Model<RobustScalerModel>, 
RobustScalerModelParams<RobustScalerModel> {
+
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+    private Table modelDataTable;
+
+    public RobustScalerModel() {
+        ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public Table[] transform(Table... inputs) {
+        Preconditions.checkArgument(inputs.length == 1);
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) 
inputs[0]).getTableEnvironment();
+        DataStream<Row> inputStream = tEnv.toDataStream(inputs[0]);
+
+        RowTypeInfo inputTypeInfo = 
TableUtils.getRowTypeInfo(inputs[0].getResolvedSchema());
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        ArrayUtils.addAll(inputTypeInfo.getFieldTypes(), 
VectorTypeInfo.INSTANCE),
+                        ArrayUtils.addAll(inputTypeInfo.getFieldNames(), 
getOutputCol()));
+        final String broadcastModelKey = "broadcastModelKey";
+        DataStream<RobustScalerModelData> modelDataStream =
+                RobustScalerModelData.getModelDataStream(modelDataTable);
+
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(inputStream),
+                        Collections.singletonMap(broadcastModelKey, 
modelDataStream),
+                        inputList -> {
+                            DataStream inputData = inputList.get(0);
+                            return inputData.map(
+                                    new PredictOutputFunction(
+                                            broadcastModelKey,
+                                            getInputCol(),
+                                            getWithCentering(),
+                                            getWithScaling()),
+                                    outputTypeInfo);
+                        });
+
+        return new Table[] {tEnv.fromDataStream(output)};
+    }
+
+    /** This operator loads model data and predicts result. */
+    private static class PredictOutputFunction extends RichMapFunction<Row, 
Row> {
+        private final String broadcastModelKey;
+        private final String inputCol;
+        private final boolean withCentering;
+        private final boolean withScaling;
+
+        private DenseVector medians;
+        private DenseVector scales;
+
+        public PredictOutputFunction(
+                String broadcastModelKey,
+                String inputCol,
+                boolean withCentering,
+                boolean withScaling) {
+            this.broadcastModelKey = broadcastModelKey;
+            this.inputCol = inputCol;
+            this.withCentering = withCentering;
+            this.withScaling = withScaling;
+        }
+
+        @Override
+        public Row map(Row row) throws Exception {
+            if (medians == null) {
+                RobustScalerModelData modelData =
+                        (RobustScalerModelData)
+                                
getRuntimeContext().getBroadcastVariable(broadcastModelKey).get(0);
+                medians = modelData.medians;
+                scales =
+                        new DenseVector(
+                                Arrays.stream(modelData.ranges.values)
+                                        .map(range -> range == 0 ? 0 : 1 / 
range)
+                                        .toArray());
+            }
+            DenseVector outputVec = ((Vector) 
row.getField(inputCol)).clone().toDense();

Review Comment:
   `toDense()` might be redundant, as it is supposed that `BLAS.axpy` and 
`BLAS.hDot` can handle sparse vectors properly.



##########
flink-ml-python/pyflink/examples/ml/feature/robustscaler_example.py:
##########
@@ -0,0 +1,74 @@
+################################################################################
+#  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.
+################################################################################
+
+# Simple program that creates a RobustScaler instance and uses it for feature
+# engineering.
+
+from pyflink.common import Types
+from pyflink.datastream import StreamExecutionEnvironment
+from pyflink.table import StreamTableEnvironment
+
+from pyflink.ml.core.linalg import Vectors, DenseVectorTypeInfo
+
+from pyflink.ml.lib.feature.robustscaler import RobustScaler
+
+# Creates a new StreamExecutionEnvironment.
+env = StreamExecutionEnvironment.get_execution_environment()
+
+# Creates a StreamTableEnvironment.
+t_env = StreamTableEnvironment.create(env)
+
+# Generates input training and prediction data.
+train_data = t_env.from_data_stream(
+    env.from_collection([
+        (1, Vectors.dense(0.0, 0.0),),
+        (2, Vectors.dense(1.0, -1.0),),
+        (3, Vectors.dense(2.0, -2.0),),
+        (4, Vectors.dense(3.0, -3.0),),
+        (5, Vectors.dense(4.0, -4.0),),
+        (6, Vectors.dense(5.0, -5.0),),
+        (7, Vectors.dense(6.0, -6.0),),
+        (8, Vectors.dense(7.0, -7.0),),
+        (9, Vectors.dense(8.0, -8.0),),
+    ],
+        type_info=Types.ROW_NAMED(
+            ['id', 'input'],
+            [Types.INT(), DenseVectorTypeInfo()])
+    ))
+
+# Creates an RobustScaler object and initializes its parameters.
+robust_scaler = RobustScaler()\
+    .set_lower(0.25)\
+    .set_upper(0.75)\
+    .set_relative_error(0.001)\
+    .set_with_scaling(True)\
+    .set_with_centering(True)
+
+# Trains the RobustScaler Model.
+model = robust_scaler.fit(train_data)
+
+# Uses the RobustScaler Model for predictions.
+output = model.transform(train_data)[0]
+
+# Extracts and displays the results.

Review Comment:
   Comments in python examples uses the original form of the verbs and do not 
use uppercase for the first letter.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

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


Reply via email to