jiangxin369 commented on code in PR #158:
URL: https://github.com/apache/flink-ml/pull/158#discussion_r980656357


##########
flink-ml-lib/src/main/java/org/apache/flink/ml/feature/variancethresholdselector/VarianceThresholdSelectorModel.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.variancethresholdselector;
+
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.ml.api.Model;
+import org.apache.flink.ml.common.broadcast.BroadcastUtils;
+import org.apache.flink.ml.common.datastream.TableUtils;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.linalg.Vector;
+import org.apache.flink.ml.linalg.Vectors;
+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 removes low-variance data using the model data computed by 
{@link
+ * VarianceThresholdSelector}.
+ */
+public class VarianceThresholdSelectorModel
+        implements Model<VarianceThresholdSelectorModel>,
+                
VarianceThresholdSelectorParams<VarianceThresholdSelectorModel> {
+
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+    private Table modelDataTable;
+
+    public VarianceThresholdSelectorModel() {
+        ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+    }
+
+    @Override
+    public VarianceThresholdSelectorModel setModelData(Table... inputs) {
+        Preconditions.checkArgument(inputs.length == 1);
+        modelDataTable = inputs[0];
+        return this;
+    }
+
+    @Override
+    public Table[] getModelData() {
+        return new Table[] {modelDataTable};
+    }
+
+    @Override
+    public Map<Param<?>, Object> getParamMap() {
+        return paramMap;
+    }
+
+    @Override
+    public void save(String path) throws IOException {
+        ReadWriteUtils.saveMetadata(this, path);
+        ReadWriteUtils.saveModelData(
+                
VarianceThresholdSelectorModelData.getModelDataStream(modelDataTable),
+                path,
+                new VarianceThresholdSelectorModelData.ModelDataEncoder());
+    }
+
+    public static VarianceThresholdSelectorModel load(StreamTableEnvironment 
tEnv, String path)
+            throws IOException {
+        VarianceThresholdSelectorModel model = 
ReadWriteUtils.loadStageParam(path);
+        Table modelDataTable =
+                ReadWriteUtils.loadModelData(
+                        tEnv, path, new 
VarianceThresholdSelectorModelData.ModelDataDecoder());
+        return model.setModelData(modelDataTable);
+    }
+
+    @Override
+    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<VarianceThresholdSelectorModelData> 
varianceThresholdSelectorModel =
+                
VarianceThresholdSelectorModelData.getModelDataStream(modelDataTable);
+
+        final String broadcastModelKey = "broadcastModelKey";
+        RowTypeInfo inputTypeInfo = 
TableUtils.getRowTypeInfo(inputs[0].getResolvedSchema());
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        ArrayUtils.addAll(
+                                inputTypeInfo.getFieldTypes(),
+                                TypeInformation.of(DenseVector.class)),
+                        ArrayUtils.addAll(inputTypeInfo.getFieldNames(), 
getOutputCol()));
+
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(data),
+                        Collections.singletonMap(broadcastModelKey, 
varianceThresholdSelectorModel),
+                        inputList -> {
+                            DataStream input = inputList.get(0);
+                            return input.map(
+                                    new 
PredictOutputFunction(getFeaturesCol(), broadcastModelKey),
+                                    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 featureCol;
+        private final String broadcastKey;
+        private int expectedNumOfFeatures;
+        private int[] indices = null;
+
+        public PredictOutputFunction(String featureCol, String broadcastKey) {
+            this.featureCol = featureCol;
+            this.broadcastKey = broadcastKey;
+        }
+
+        @Override
+        public Row map(Row row) {
+            if (indices == null) {
+                VarianceThresholdSelectorModelData 
varianceThresholdSelectorModelData =
+                        (VarianceThresholdSelectorModelData)
+                                
getRuntimeContext().getBroadcastVariable(broadcastKey).get(0);
+                expectedNumOfFeatures = 
varianceThresholdSelectorModelData.numOfFeatures;
+                indices = varianceThresholdSelectorModelData.indices;
+            }
+
+            if (indices.length == 0) {
+                return Row.join(row, Row.of(Vectors.dense()));
+            } else {
+                DenseVector inputVec = ((Vector) 
row.getField(featureCol)).toDense();
+                Preconditions.checkArgument(

Review Comment:
   Good catch, I'll move it.



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

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

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

Reply via email to