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


##########
flink-ml-lib/src/main/java/org/apache/flink/ml/feature/maxabsscaler/MaxAbsScaler.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.maxabsscaler;
+
+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.api.common.typeinfo.TypeInformation;
+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.linalg.SparseVector;
+import org.apache.flink.ml.linalg.Vector;
+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.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.Map;
+
+/**
+ * An Estimator which implements the MaxAbsScaler algorithm. This algorithm 
rescales feature values
+ * to the range [-1, 1] by dividing through the largest maximum absolute value 
in each feature. It
+ * does not shift/center the data and thus does not destroy any sparsity.
+ */
+public class MaxAbsScaler
+        implements Estimator<MaxAbsScaler, MaxAbsScalerModel>, 
MaxAbsScalerParams<MaxAbsScaler> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+
+    public MaxAbsScaler() {
+        ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+    }
+
+    @Override
+    public MaxAbsScalerModel fit(Table... inputs) {
+        Preconditions.checkArgument(inputs.length == 1);
+        final String inputCol = getInputCol();
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) 
inputs[0]).getTableEnvironment();
+        DataStream<Vector> inputData =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                (MapFunction<Row, Vector>)
+                                        value -> ((Vector) 
value.getField(inputCol)));
+        DataStream<Vector> maxAbsValues =
+                inputData
+                        .transform(
+                                "reduceInEachPartition",
+                                inputData.getType(),
+                                new MaxAbsReduceFunctionOperator())
+                        .transform(
+                                "reduceInFinalPartition",
+                                inputData.getType(),
+                                new MaxAbsReduceFunctionOperator())
+                        .setParallelism(1);
+        DataStream<MaxAbsScalerModelData> modelData =
+                DataStreamUtils.mapPartition(
+                        maxAbsValues,
+                        new RichMapPartitionFunction<Vector, 
MaxAbsScalerModelData>() {
+                            @Override
+                            public void mapPartition(
+                                    Iterable<Vector> values, 
Collector<MaxAbsScalerModelData> out) {
+                                DenseVector maxVector = (DenseVector) 
values.iterator().next();
+                                out.collect(new 
MaxAbsScalerModelData(maxVector));
+                            }
+                        });
+
+        MaxAbsScalerModel model =
+                new 
MaxAbsScalerModel().setModelData(tEnv.fromDataStream(modelData));
+        ReadWriteUtils.updateExistingParams(model, getParamMap());
+        return model;
+    }
+
+    /**
+     * A stream operator to compute the max values in each partition of the 
input bounded data
+     * stream.
+     */
+    private static class MaxAbsReduceFunctionOperator extends 
AbstractStreamOperator<Vector>
+            implements OneInputStreamOperator<Vector, Vector>, BoundedOneInput 
{
+        private ListState<DenseVector> maxState;
+        private DenseVector maxVector;
+
+        @Override
+        public void endInput() {
+            if (maxVector != null) {
+                output.collect(new StreamRecord<>(maxVector));
+            }
+        }
+
+        @Override
+        public void processElement(StreamRecord<Vector> streamRecord) {
+            Vector currentValue = streamRecord.getValue();
+            if (currentValue == null) {
+                throw new RuntimeException("Input column data cannot be 
null.");
+            }
+            if (maxVector == null) {
+                int vecSize = currentValue.size();
+                maxVector = new DenseVector(vecSize);
+                if (currentValue instanceof DenseVector) {
+                    System.arraycopy(
+                            ((DenseVector) currentValue).values, 0, 
maxVector.values, 0, vecSize);
+                } else {
+                    int[] indices = ((SparseVector) currentValue).indices;
+                    double[] values = ((SparseVector) currentValue).values;
+                    for (int i = 0; i < indices.length; ++i) {
+                        maxVector.values[indices[i]] =
+                                Math.max(maxVector.values[indices[i]], 
values[i]);
+                    }
+                }
+            } else {
+                Preconditions.checkArgument(
+                        currentValue.size() == maxVector.size(),
+                        "CurrentValue should has same size with maxVector.");
+                if (currentValue instanceof DenseVector) {
+                    double[] values = ((DenseVector) currentValue).values;
+                    for (int i = 0; i < currentValue.size(); ++i) {
+                        maxVector.values[i] = Math.max(maxVector.values[i], 
values[i]);
+                    }
+                } else if (currentValue instanceof SparseVector) {
+                    int[] indices = ((SparseVector) currentValue).indices;
+                    double[] values = ((SparseVector) currentValue).values;
+                    for (int i = 0; i < indices.length; ++i) {
+                        maxVector.values[indices[i]] =
+                                Math.max(maxVector.values[indices[i]], 
values[i]);
+                    }
+                } else {
+                    throw new RuntimeException(
+                            "Input column type must be SparseVector or 
DenseVector. ");
+                }
+            }
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")
+        public void initializeState(StateInitializationContext context) throws 
Exception {
+            super.initializeState(context);
+            maxState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "maxState", 
TypeInformation.of(DenseVector.class)));
+            OperatorStateUtils.getUniqueElement(maxState, 
"maxState").ifPresent(x -> maxVector = x);
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")

Review Comment:
   Some SuppressWarnings like this are redundant. Could you please check and 
remove them?



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/feature/maxabsscaler/MaxAbsScaler.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.maxabsscaler;
+
+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.api.common.typeinfo.TypeInformation;
+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.linalg.SparseVector;
+import org.apache.flink.ml.linalg.Vector;
+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.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.Map;
+
+/**
+ * An Estimator which implements the MaxAbsScaler algorithm. This algorithm 
rescales feature values
+ * to the range [-1, 1] by dividing through the largest maximum absolute value 
in each feature. It
+ * does not shift/center the data and thus does not destroy any sparsity.
+ */
+public class MaxAbsScaler
+        implements Estimator<MaxAbsScaler, MaxAbsScalerModel>, 
MaxAbsScalerParams<MaxAbsScaler> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+
+    public MaxAbsScaler() {
+        ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+    }
+
+    @Override
+    public MaxAbsScalerModel fit(Table... inputs) {
+        Preconditions.checkArgument(inputs.length == 1);
+        final String inputCol = getInputCol();
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) 
inputs[0]).getTableEnvironment();
+        DataStream<Vector> inputData =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                (MapFunction<Row, Vector>)
+                                        value -> ((Vector) 
value.getField(inputCol)));
+        DataStream<Vector> maxAbsValues =
+                inputData
+                        .transform(
+                                "reduceInEachPartition",
+                                inputData.getType(),
+                                new MaxAbsReduceFunctionOperator())
+                        .transform(
+                                "reduceInFinalPartition",
+                                inputData.getType(),
+                                new MaxAbsReduceFunctionOperator())
+                        .setParallelism(1);
+        DataStream<MaxAbsScalerModelData> modelData =
+                DataStreamUtils.mapPartition(

Review Comment:
   Given that the upstream operator would generate only one stream record, it 
seems unnecessary to use `DataStreamUtils.mapPartition` here. Maybe a simple 
`map` function is enough.



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MaxAbsScalerTest.java:
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.maxabsscaler.MaxAbsScaler;
+import org.apache.flink.ml.feature.maxabsscaler.MaxAbsScalerModel;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.linalg.Vector;
+import org.apache.flink.ml.linalg.Vectors;
+import org.apache.flink.ml.util.ReadWriteUtils;
+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.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.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.Collections;
+import java.util.List;
+
+import static 
org.apache.flink.test.util.TestBaseUtils.compareResultCollections;
+import static org.junit.Assert.assertEquals;
+
+/** Tests {@link MaxAbsScaler} and {@link MaxAbsScalerModel}. */
+public class MaxAbsScalerTest {
+    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+    private StreamTableEnvironment tEnv;
+    private StreamExecutionEnvironment env;
+    private Table trainDataTable;
+    private Table predictDataTable;
+    private Table trainSparseDataTable;
+    private Table predictSparseDataTable;
+    private static final List<Row> TRAIN_DATA =
+            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> PREDICT_DATA =
+            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<Row> TRAIN_SPARSE_DATA =
+            new ArrayList<>(
+                    Arrays.asList(
+                            Row.of(Vectors.sparse(4, new int[] {1, 3}, new 
double[] {4.0, 3.0})),
+                            Row.of(Vectors.sparse(4, new int[] {0, 2}, new 
double[] {2.0, 6.0})),
+                            Row.of(Vectors.sparse(4, new int[] {1, 2}, new 
double[] {1.0, 3.0})),
+                            Row.of(Vectors.sparse(4, new int[] {0, 1}, new 
double[] {2.0, 8.0})),
+                            Row.of(Vectors.sparse(4, new int[] {1, 3}, new 
double[] {1.0, 5.0}))));
+    private static final List<Row> PREDICT_SPARSE_DATA =
+            new ArrayList<>(
+                    Arrays.asList(
+                            Row.of(Vectors.sparse(4, new int[] {0, 1}, new 
double[] {2.0, 4.0})),
+                            Row.of(Vectors.sparse(4, new int[] {0, 2}, new 
double[] {1.0, 3.0})),
+                            Row.of(Vectors.sparse(4, new int[] {}, new 
double[] {})),
+                            Row.of(Vectors.sparse(4, new int[] {1, 3}, new 
double[] {1.0, 2.0}))));
+
+    private static final List<Vector> EXPECTED_DATA =
+            new ArrayList<>(
+                    Arrays.asList(
+                            Vectors.dense(0.25, 0.1),
+                            Vectors.dense(0.5, 0.125),
+                            Vectors.dense(0.75, 0.225)));
+
+    private static final List<Vector> EXPECTED_SPARSE_DATA =
+            new ArrayList<>(
+                    Arrays.asList(
+                            Vectors.sparse(4, new int[] {0, 1}, new double[] 
{1.0, 0.5}),
+                            Vectors.sparse(4, new int[] {0, 2}, new double[] 
{0.5, 0.5}),
+                            Vectors.sparse(4, new int[] {}, new double[] {}),
+                            Vectors.sparse(4, new int[] {1, 3}, new double[] 
{0.125, 0.4})));
+
+    @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("input");
+        predictDataTable = 
tEnv.fromDataStream(env.fromCollection(PREDICT_DATA)).as("input");
+        trainSparseDataTable =
+                
tEnv.fromDataStream(env.fromCollection(TRAIN_SPARSE_DATA)).as("input");
+        predictSparseDataTable =
+                
tEnv.fromDataStream(env.fromCollection(PREDICT_SPARSE_DATA)).as("input");
+    }
+
+    private static void verifyPredictionResult(Table output, String outputCol, 
boolean isSparse)
+            throws Exception {
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) 
output).getTableEnvironment();
+        DataStream<Vector> stream =
+                tEnv.toDataStream(output)
+                        .map((MapFunction<Row, Vector>) row -> 
row.getFieldAs(outputCol));
+
+        List<Vector> result = IteratorUtils.toList(stream.executeAndCollect());
+        if (isSparse) {
+            compareResultCollections(
+                    MaxAbsScalerTest.EXPECTED_SPARSE_DATA, result, 
TestUtils::compare);
+        } else {
+            compareResultCollections(MaxAbsScalerTest.EXPECTED_DATA, result, 
TestUtils::compare);
+        }
+    }
+
+    @Test
+    public void testParam() {
+        MaxAbsScaler maxAbsScaler = new MaxAbsScaler();
+        assertEquals("input", maxAbsScaler.getInputCol());
+        assertEquals("output", maxAbsScaler.getOutputCol());
+
+        maxAbsScaler.setInputCol("test_input").setOutputCol("test_output");
+        assertEquals("test_input", maxAbsScaler.getInputCol());
+        assertEquals("test_output", maxAbsScaler.getOutputCol());
+    }
+
+    @Test
+    public void testOutputSchema() {
+        MaxAbsScaler maxAbsScaler =
+                new 
MaxAbsScaler().setInputCol("test_input").setOutputCol("test_output");
+
+        MaxAbsScalerModel model = 
maxAbsScaler.fit(trainDataTable.as("test_input"));
+        Table output = model.transform(predictDataTable.as("test_input"))[0];
+        assertEquals(
+                Arrays.asList("test_input", "test_output"),
+                output.getResolvedSchema().getColumnNames());
+    }
+
+    @Test
+    public void testFitAndPredict() throws Exception {
+        MaxAbsScaler maxAbsScaler = new MaxAbsScaler();
+        MaxAbsScalerModel maxAbsScalerModel = maxAbsScaler.fit(trainDataTable);
+        Table output = maxAbsScalerModel.transform(predictDataTable)[0];
+        verifyPredictionResult(output, maxAbsScaler.getOutputCol(), false);
+    }
+
+    @Test
+    public void testFitDataWithNullValue() {
+        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((Object) null),
+                                Row.of(Vectors.dense(6.1, 8.1)),
+                                Row.of(Vectors.dense(200, 400))));
+
+        Table trainDataWithInvalidData =
+                tEnv.fromDataStream(env.fromCollection(trainData)).as("input");
+        try {
+            MaxAbsScaler maxAbsScaler = new MaxAbsScaler();
+            MaxAbsScalerModel maxAbsScalerModel = 
maxAbsScaler.fit(trainDataWithInvalidData);
+            IteratorUtils.toList(
+                    
tEnv.toDataStream(maxAbsScalerModel.getModelData()[0]).executeAndCollect());
+        } catch (Exception e) {
+            assertEquals(
+                    "Input column data cannot be null.",
+                    ExceptionUtils.getRootCause(e).getMessage());
+        }
+    }
+
+    @Test
+    public void testTransformDataWithNullValue() throws Exception {
+        List<Row> predictData =
+                new ArrayList<>(
+                        Arrays.asList(
+                                Row.of(Vectors.dense(0.0, 3.0)),
+                                Row.of((Object) null),
+                                Row.of((Object) null)));
+
+        Table predictDataWithInvalidData =
+                
tEnv.fromDataStream(env.fromCollection(predictData)).as("input");
+
+        MaxAbsScaler maxAbsScaler = new MaxAbsScaler();
+
+        MaxAbsScalerModel maxAbsScalerModel = maxAbsScaler.fit(trainDataTable);
+        Table output = 
maxAbsScalerModel.transform(predictDataWithInvalidData)[0];
+        List<Row> result = 
IteratorUtils.toList(tEnv.toDataStream(output).executeAndCollect());
+        for (Row row : result) {
+            if (row.getField("output") != null) {
+                double[] expected = new double[] {0.0, 0.0075};
+                DenseVector vec = row.getFieldAs("output");
+                for (int i = 0; i < vec.size(); ++i) {
+                    assertEquals(expected[i], vec.get(i), 1.0e-5);
+                }
+            }
+        }
+    }
+
+    @Test
+    public void testFitAndPredictSparse() throws Exception {
+        MaxAbsScaler maxAbsScaler = new MaxAbsScaler();
+        MaxAbsScalerModel maxAbsScalerModel = 
maxAbsScaler.fit(trainSparseDataTable);
+        Table output = maxAbsScalerModel.transform(predictSparseDataTable)[0];
+        verifyPredictionResult(output, maxAbsScaler.getOutputCol(), true);
+    }
+
+    @Test
+    public void testSaveLoadAndPredict() throws Exception {
+        MaxAbsScaler maxAbsScaler = new MaxAbsScaler();
+        MaxAbsScaler loadedMaxAbsScaler =
+                TestUtils.saveAndReload(
+                        tEnv, maxAbsScaler, 
tempFolder.newFolder().getAbsolutePath());
+        MaxAbsScalerModel model = loadedMaxAbsScaler.fit(trainDataTable);
+        MaxAbsScalerModel loadedModel =
+                TestUtils.saveAndReload(tEnv, model, 
tempFolder.newFolder().getAbsolutePath());
+        Table output = loadedModel.transform(predictDataTable)[0];
+        verifyPredictionResult(output, maxAbsScaler.getOutputCol(), false);
+    }
+
+    @Test
+    public void testGetModelData() throws Exception {
+        MaxAbsScaler maxAbsScaler = new MaxAbsScaler();
+        MaxAbsScalerModel maxAbsScalerModel = maxAbsScaler.fit(trainDataTable);
+        Table modelData = maxAbsScalerModel.getModelData()[0];
+        assertEquals(
+                Collections.singletonList("maxVector"),
+                modelData.getResolvedSchema().getColumnNames());
+        DataStream<Row> output = tEnv.toDataStream(modelData);
+        List<Row> modelRows = IteratorUtils.toList(output.executeAndCollect());
+        assertEquals(new DenseVector(new double[] {200.0, 400.0}), 
modelRows.get(0).getField(0));
+    }
+
+    @Test
+    public void testSetModelData() throws Exception {

Review Comment:
   Could you please add a test case that sets a model data that contains zero 
and negative values, and make sure the resulting behavior is the same as that 
in Spark?



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/feature/MaxAbsScalerTest.java:
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.maxabsscaler.MaxAbsScaler;
+import org.apache.flink.ml.feature.maxabsscaler.MaxAbsScalerModel;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.linalg.Vector;
+import org.apache.flink.ml.linalg.Vectors;
+import org.apache.flink.ml.util.ReadWriteUtils;
+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.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.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.Collections;
+import java.util.List;
+
+import static 
org.apache.flink.test.util.TestBaseUtils.compareResultCollections;
+import static org.junit.Assert.assertEquals;
+
+/** Tests {@link MaxAbsScaler} and {@link MaxAbsScalerModel}. */
+public class MaxAbsScalerTest {

Review Comment:
   Could you please add a test to verify the case when all values of input 
vectors are negative values? There might be a bug here?



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/feature/maxabsscaler/MaxAbsScaler.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.maxabsscaler;
+
+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.api.common.typeinfo.TypeInformation;
+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.linalg.SparseVector;
+import org.apache.flink.ml.linalg.Vector;
+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.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.Map;
+
+/**
+ * An Estimator which implements the MaxAbsScaler algorithm. This algorithm 
rescales feature values
+ * to the range [-1, 1] by dividing through the largest maximum absolute value 
in each feature. It
+ * does not shift/center the data and thus does not destroy any sparsity.
+ */
+public class MaxAbsScaler
+        implements Estimator<MaxAbsScaler, MaxAbsScalerModel>, 
MaxAbsScalerParams<MaxAbsScaler> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+
+    public MaxAbsScaler() {
+        ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+    }
+
+    @Override
+    public MaxAbsScalerModel fit(Table... inputs) {
+        Preconditions.checkArgument(inputs.length == 1);
+        final String inputCol = getInputCol();
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) 
inputs[0]).getTableEnvironment();
+        DataStream<Vector> inputData =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                (MapFunction<Row, Vector>)
+                                        value -> ((Vector) 
value.getField(inputCol)));
+        DataStream<Vector> maxAbsValues =
+                inputData
+                        .transform(
+                                "reduceInEachPartition",
+                                inputData.getType(),
+                                new MaxAbsReduceFunctionOperator())
+                        .transform(
+                                "reduceInFinalPartition",
+                                inputData.getType(),
+                                new MaxAbsReduceFunctionOperator())
+                        .setParallelism(1);

Review Comment:
   Would it be better to reuse `DataStreamUtils.reduce` for this part?



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/feature/maxabsscaler/MaxAbsScalerModel.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.maxabsscaler;
+
+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.BLAS;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.linalg.Vector;
+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.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * A Model which do a MaxAbsScaler operation using the model data computed by 
{@link MaxAbsScaler}.
+ */
+public class MaxAbsScalerModel
+        implements Model<MaxAbsScalerModel>, 
MaxAbsScalerParams<MaxAbsScalerModel> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+    private Table modelDataTable;
+
+    public MaxAbsScalerModel() {
+        ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+    }
+
+    @Override
+    public MaxAbsScalerModel 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<MaxAbsScalerModelData> maxAbsScalerModel =
+                MaxAbsScalerModelData.getModelDataStream(modelDataTable);
+        final String broadcastModelKey = "broadcastModelKey";
+        RowTypeInfo inputTypeInfo = 
TableUtils.getRowTypeInfo(inputs[0].getResolvedSchema());
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        ArrayUtils.addAll(
+                                inputTypeInfo.getFieldTypes(), 
TypeInformation.of(Vector.class)),
+                        ArrayUtils.addAll(inputTypeInfo.getFieldNames(), 
getOutputCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, 
maxAbsScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new 
PredictOutputFunction(broadcastModelKey, getInputCol()),
+                                    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(
+                MaxAbsScalerModelData.getModelDataStream(modelDataTable),
+                path,
+                new MaxAbsScalerModelData.ModelDataEncoder());
+    }
+
+    /**
+     * Loads model data from path.
+     *
+     * @param tEnv Stream table environment.
+     * @param path Model path.
+     * @return MaxAbsScalerModel model.
+     */
+    public static MaxAbsScalerModel load(StreamTableEnvironment tEnv, String 
path)
+            throws IOException {
+        MaxAbsScalerModel model = ReadWriteUtils.loadStageParam(path);
+        Table modelDataTable =
+                ReadWriteUtils.loadModelData(
+                        tEnv, path, new 
MaxAbsScalerModelData.ModelDataDecoder());
+        return model.setModelData(modelDataTable);
+    }
+
+    /** This operator loads model data and predicts result. */
+    private static class PredictOutputFunction extends RichMapFunction<Row, 
Row> {
+        private final String inputCol;
+        private final String broadcastKey;
+        private DenseVector scaleVector;
+
+        public PredictOutputFunction(String broadcastKey, String inputCol) {
+            this.broadcastKey = broadcastKey;
+            this.inputCol = inputCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (scaleVector == null) {
+                MaxAbsScalerModelData maxAbsScalerModelData =
+                        (MaxAbsScalerModelData)
+                                
getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                scaleVector = maxAbsScalerModelData.maxVector;

Review Comment:
   Could you please add document, support and tests for cases in which 
`maxVector` contains 0 values, which means `1.0 / scaleVector.values[i]` is 
invalid?



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/feature/maxabsscaler/MaxAbsScaler.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.maxabsscaler;
+
+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.api.common.typeinfo.TypeInformation;
+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.linalg.SparseVector;
+import org.apache.flink.ml.linalg.Vector;
+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.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.Map;
+
+/**
+ * An Estimator which implements the MaxAbsScaler algorithm. This algorithm 
rescales feature values
+ * to the range [-1, 1] by dividing through the largest maximum absolute value 
in each feature. It
+ * does not shift/center the data and thus does not destroy any sparsity.
+ */
+public class MaxAbsScaler
+        implements Estimator<MaxAbsScaler, MaxAbsScalerModel>, 
MaxAbsScalerParams<MaxAbsScaler> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+
+    public MaxAbsScaler() {
+        ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+    }
+
+    @Override
+    public MaxAbsScalerModel fit(Table... inputs) {
+        Preconditions.checkArgument(inputs.length == 1);
+        final String inputCol = getInputCol();
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) 
inputs[0]).getTableEnvironment();
+        DataStream<Vector> inputData =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                (MapFunction<Row, Vector>)
+                                        value -> ((Vector) 
value.getField(inputCol)));
+        DataStream<Vector> maxAbsValues =
+                inputData
+                        .transform(
+                                "reduceInEachPartition",
+                                inputData.getType(),
+                                new MaxAbsReduceFunctionOperator())
+                        .transform(
+                                "reduceInFinalPartition",
+                                inputData.getType(),
+                                new MaxAbsReduceFunctionOperator())
+                        .setParallelism(1);
+        DataStream<MaxAbsScalerModelData> modelData =
+                DataStreamUtils.mapPartition(
+                        maxAbsValues,
+                        new RichMapPartitionFunction<Vector, 
MaxAbsScalerModelData>() {
+                            @Override
+                            public void mapPartition(
+                                    Iterable<Vector> values, 
Collector<MaxAbsScalerModelData> out) {
+                                DenseVector maxVector = (DenseVector) 
values.iterator().next();
+                                out.collect(new 
MaxAbsScalerModelData(maxVector));
+                            }
+                        });
+
+        MaxAbsScalerModel model =
+                new 
MaxAbsScalerModel().setModelData(tEnv.fromDataStream(modelData));
+        ReadWriteUtils.updateExistingParams(model, getParamMap());
+        return model;
+    }
+
+    /**
+     * A stream operator to compute the max values in each partition of the 
input bounded data
+     * stream.
+     */
+    private static class MaxAbsReduceFunctionOperator extends 
AbstractStreamOperator<Vector>
+            implements OneInputStreamOperator<Vector, Vector>, BoundedOneInput 
{
+        private ListState<DenseVector> maxState;
+        private DenseVector maxVector;

Review Comment:
   If all input records are sparse vectors, this practice might waste much 
memory space. Is there any better way to improve this implementation?



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/feature/maxabsscaler/MaxAbsScalerModel.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.maxabsscaler;
+
+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.BLAS;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.linalg.Vector;
+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.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * A Model which do a MaxAbsScaler operation using the model data computed by 
{@link MaxAbsScaler}.
+ */
+public class MaxAbsScalerModel
+        implements Model<MaxAbsScalerModel>, 
MaxAbsScalerParams<MaxAbsScalerModel> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+    private Table modelDataTable;
+
+    public MaxAbsScalerModel() {
+        ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+    }
+
+    @Override
+    public MaxAbsScalerModel 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<MaxAbsScalerModelData> maxAbsScalerModel =
+                MaxAbsScalerModelData.getModelDataStream(modelDataTable);
+        final String broadcastModelKey = "broadcastModelKey";
+        RowTypeInfo inputTypeInfo = 
TableUtils.getRowTypeInfo(inputs[0].getResolvedSchema());
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        ArrayUtils.addAll(
+                                inputTypeInfo.getFieldTypes(), 
TypeInformation.of(Vector.class)),
+                        ArrayUtils.addAll(inputTypeInfo.getFieldNames(), 
getOutputCol()));
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, 
maxAbsScalerModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new 
PredictOutputFunction(broadcastModelKey, getInputCol()),
+                                    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(
+                MaxAbsScalerModelData.getModelDataStream(modelDataTable),
+                path,
+                new MaxAbsScalerModelData.ModelDataEncoder());
+    }
+
+    /**
+     * Loads model data from path.
+     *
+     * @param tEnv Stream table environment.
+     * @param path Model path.
+     * @return MaxAbsScalerModel model.
+     */
+    public static MaxAbsScalerModel load(StreamTableEnvironment tEnv, String 
path)
+            throws IOException {
+        MaxAbsScalerModel model = ReadWriteUtils.loadStageParam(path);
+        Table modelDataTable =
+                ReadWriteUtils.loadModelData(
+                        tEnv, path, new 
MaxAbsScalerModelData.ModelDataDecoder());
+        return model.setModelData(modelDataTable);
+    }
+
+    /** This operator loads model data and predicts result. */
+    private static class PredictOutputFunction extends RichMapFunction<Row, 
Row> {
+        private final String inputCol;
+        private final String broadcastKey;
+        private DenseVector scaleVector;
+
+        public PredictOutputFunction(String broadcastKey, String inputCol) {
+            this.broadcastKey = broadcastKey;
+            this.inputCol = inputCol;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (scaleVector == null) {
+                MaxAbsScalerModelData maxAbsScalerModelData =
+                        (MaxAbsScalerModelData)
+                                
getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                scaleVector = maxAbsScalerModelData.maxVector;
+                for (int i = 0; i < scaleVector.size(); ++i) {
+                    if (scaleVector.values[i] != 0) {
+                        scaleVector.values[i] = 1.0 / 
Math.abs(scaleVector.values[i]);

Review Comment:
   Spark's implementation is similar to `1.0 / scaleVector.values[i]`, instead 
of `1.0 / Math.abs(scaleVector.values[i])`. Would it be better to follow 
Spark's practice?



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to