yunfengzhou-hub commented on code in PR #142: URL: https://github.com/apache/flink-ml/pull/142#discussion_r948687937
########## flink-ml-lib/src/main/java/org/apache/flink/ml/feature/maxabsscaler/MaxAbsScaler.java: ########## @@ -0,0 +1,193 @@ +/* + * 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.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.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.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 = + maxAbsValues.map( + (MapFunction<Vector, MaxAbsScalerModelData>) + vector -> new MaxAbsScalerModelData((DenseVector) vector)); + + MaxAbsScalerModel model = + new MaxAbsScalerModel().setModelData(tEnv.fromDataStream(modelData)); + ReadWriteUtils.updateExistingParams(model, getParamMap()); + return model; + } + + /** + * A stream operator to compute the maximum absolute 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> maxAbsState; + private DenseVector maxAbsVector; + + @Override + public void endInput() { + if (maxAbsVector != null) { + output.collect(new StreamRecord<>(maxAbsVector)); + } + } + + @Override + public void processElement(StreamRecord<Vector> streamRecord) { + Vector currentValue = streamRecord.getValue(); + if (currentValue == null) { + throw new RuntimeException("Input column data cannot be null."); + } + if (maxAbsVector == null) { + int vecSize = currentValue.size(); + maxAbsVector = new DenseVector(vecSize); + if (currentValue instanceof DenseVector) { + double[] values = ((DenseVector) currentValue).values; + for (int i = 0; i < currentValue.size(); ++i) { + maxAbsVector.values[i] = Math.abs(values[i]); + } + } else { + int[] indices = ((SparseVector) currentValue).indices; + double[] values = ((SparseVector) currentValue).values; + for (int i = 0; i < indices.length; ++i) { + maxAbsVector.values[indices[i]] = + Math.max(maxAbsVector.values[indices[i]], Math.abs(values[i])); + } + } + } else { + Preconditions.checkArgument( + currentValue.size() == maxAbsVector.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) { + maxAbsVector.values[i] = + Math.max(maxAbsVector.values[i], Math.abs(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) { + maxAbsVector.values[indices[i]] = + Math.max(maxAbsVector.values[indices[i]], Math.abs(values[i])); + } + } else { + throw new RuntimeException( + "Input column type must be SparseVector or DenseVector. "); + } + } + } + + @Override + public void initializeState(StateInitializationContext context) throws Exception { + super.initializeState(context); + maxAbsState = + context.getOperatorStateStore() + .getListState( + new ListStateDescriptor<>( + "maxState", TypeInformation.of(DenseVector.class))); Review Comment: nit: `DenseVectorTypeInfo`. ########## flink-ml-lib/src/main/java/org/apache/flink/ml/feature/maxabsscaler/MaxAbsScaler.java: ########## @@ -0,0 +1,193 @@ +/* + * 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.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.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.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 = + maxAbsValues.map( + (MapFunction<Vector, MaxAbsScalerModelData>) + vector -> new MaxAbsScalerModelData((DenseVector) vector)); + + MaxAbsScalerModel model = + new MaxAbsScalerModel().setModelData(tEnv.fromDataStream(modelData)); + ReadWriteUtils.updateExistingParams(model, getParamMap()); + return model; + } + + /** + * A stream operator to compute the maximum absolute 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> maxAbsState; + private DenseVector maxAbsVector; + + @Override + public void endInput() { + if (maxAbsVector != null) { + output.collect(new StreamRecord<>(maxAbsVector)); + } + } + + @Override + public void processElement(StreamRecord<Vector> streamRecord) { + Vector currentValue = streamRecord.getValue(); + if (currentValue == null) { + throw new RuntimeException("Input column data cannot be null."); + } + if (maxAbsVector == null) { + int vecSize = currentValue.size(); + maxAbsVector = new DenseVector(vecSize); + if (currentValue instanceof DenseVector) { + double[] values = ((DenseVector) currentValue).values; + for (int i = 0; i < currentValue.size(); ++i) { + maxAbsVector.values[i] = Math.abs(values[i]); + } + } else { + int[] indices = ((SparseVector) currentValue).indices; + double[] values = ((SparseVector) currentValue).values; + for (int i = 0; i < indices.length; ++i) { + maxAbsVector.values[indices[i]] = + Math.max(maxAbsVector.values[indices[i]], Math.abs(values[i])); + } + } + } else { + Preconditions.checkArgument( + currentValue.size() == maxAbsVector.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) { + maxAbsVector.values[i] = + Math.max(maxAbsVector.values[i], Math.abs(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) { + maxAbsVector.values[indices[i]] = + Math.max(maxAbsVector.values[indices[i]], Math.abs(values[i])); + } + } else { + throw new RuntimeException( Review Comment: nit: Let's throw a more detailed exception here, like `IllegalArgumentException`. -- 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]
