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


##########
flink-ml-lib/src/test/java/org/apache/flink/ml/feature/BinarizerTest.java:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.restartstrategy.RestartStrategies;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.ml.feature.binarizer.Binarizer;
+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.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.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+
+import org.apache.commons.collections.IteratorUtils;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+/** Tests {@link Binarizer}. */
+public class BinarizerTest extends AbstractTestBase {
+
+    private StreamTableEnvironment tEnv;
+    private Table inputDataTable;
+
+    private static final List<Row> INPUT_DATA =
+            Arrays.asList(
+                    Row.of(
+                            1,
+                            Vectors.dense(1, 2),
+                            Vectors.sparse(17, new int[] {0, 3, 9}, new 
double[] {1.0, 2.0, 7.0})),
+                    Row.of(
+                            2,
+                            Vectors.dense(2, 1),
+                            Vectors.sparse(17, new int[] {0, 2, 14}, new 
double[] {5.0, 4.0, 1.0})),
+                    Row.of(
+                            3,
+                            Vectors.dense(5, 18),
+                            Vectors.sparse(
+                                    17, new int[] {0, 11, 12}, new double[] 
{2.0, 4.0, 4.0})));
+
+    private static final Double[] EXPECTED_VALUE_OUTPUT = new Double[] {0.0, 
1.0, 1.0};
+
+    private static final List<Vector> EXPECTED_DENSE_OUTPUT =
+            Arrays.asList(
+                    new DenseVector(new double[] {0.0, 1.0}),
+                    new DenseVector(new double[] {1.0, 0.0}),
+                    new DenseVector(new double[] {1.0, 1.0}));

Review Comment:
   nit: `Vectors.dense()` might be better, as it is consistent with others 
places to create vectors.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/feature/binarizer/Binarizer.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.binarizer;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.ml.api.Transformer;
+import org.apache.flink.ml.common.datastream.TableUtils;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.linalg.SparseVector;
+import org.apache.flink.ml.linalg.typeinfo.DenseVectorTypeInfo;
+import org.apache.flink.ml.linalg.typeinfo.SparseVectorTypeInfo;
+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.HashMap;
+import java.util.Map;
+
+/**
+ * A Transformer that binarizes the columns of continuous features by the 
given thresholds. The
+ * continuous features may be DenseVector, SparseVector, or Numerical Value.
+ */
+public class Binarizer implements Transformer<Binarizer>, 
BinarizerParams<Binarizer> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+
+    public Binarizer() {
+        ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+    }
+
+    @Override
+    public Table[] transform(Table... inputs) {
+        Preconditions.checkArgument(inputs.length == 1);
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) 
inputs[0]).getTableEnvironment();
+
+        RowTypeInfo inputTypeInfo = 
TableUtils.getRowTypeInfo(inputs[0].getResolvedSchema());
+        String[] inputCols = getInputCols();
+        TypeInformation<?>[] outputTypes = new 
TypeInformation[inputCols.length];
+
+        for (int i = 0; i < inputCols.length; ++i) {
+            int idx = inputTypeInfo.getFieldIndex(inputCols[i]);
+            if (inputTypeInfo.getFieldTypes()[idx] instanceof 
SparseVectorTypeInfo) {
+                outputTypes[i] = SparseVectorTypeInfo.INSTANCE;
+            } else if (inputTypeInfo.getFieldTypes()[idx] instanceof 
DenseVectorTypeInfo) {
+                outputTypes[i] = DenseVectorTypeInfo.INSTANCE;
+            } else {

Review Comment:
   nit: another possibility is that `inputTypeInfo.getFieldTypes()[idx] 
instanceof VectorTypeInfo`.



##########
flink-ml-examples/src/main/java/org/apache/flink/ml/examples/feature/BinarizerExample.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.examples.feature;
+
+import org.apache.flink.ml.feature.binarizer.Binarizer;
+import org.apache.flink.ml.linalg.Vectors;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.CloseableIterator;
+
+import java.util.Arrays;
+
+/** Simple program that creates an Binarizer instance and uses it for feature 
engineering. */
+public class BinarizerExample {
+    public static void main(String[] args) {
+        StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        // Generates input data.
+        DataStream<Row> inputStream =
+                env.fromElements(
+                        Row.of(
+                                1,
+                                Vectors.dense(1, 2),
+                                Vectors.sparse(
+                                        17, new int[] {0, 3, 9}, new double[] 
{1.0, 2.0, 7.0})),
+                        Row.of(
+                                2,
+                                Vectors.dense(2, 1),
+                                Vectors.sparse(
+                                        17, new int[] {0, 2, 14}, new double[] 
{5.0, 4.0, 1.0})),
+                        Row.of(
+                                3,
+                                Vectors.dense(5, 18),
+                                Vectors.sparse(
+                                        17, new int[] {0, 11, 12}, new 
double[] {2.0, 4.0, 4.0})));
+
+        Table inputTable = tEnv.fromDataStream(inputStream).as("f0", "f1", 
"f2");
+
+        // Creates an Binarizer object and initializes its parameters.

Review Comment:
   nit: "a Binarizer"



##########
flink-ml-examples/src/main/java/org/apache/flink/ml/examples/feature/BinarizerExample.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.examples.feature;
+
+import org.apache.flink.ml.feature.binarizer.Binarizer;
+import org.apache.flink.ml.linalg.Vectors;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.CloseableIterator;
+
+import java.util.Arrays;
+
+/** Simple program that creates an Binarizer instance and uses it for feature 
engineering. */

Review Comment:
   nit: "a Binarizer"



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/feature/binarizer/BinarizerParams.java:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.binarizer;
+
+import org.apache.flink.ml.common.param.HasInputCols;
+import org.apache.flink.ml.common.param.HasOutputCols;
+import org.apache.flink.ml.param.DoubleArrayParam;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+
+/**
+ * Params of {@link Binarizer}.
+ *
+ * @param <T> The class type of this instance.
+ */
+public interface BinarizerParams<T> extends HasInputCols<T>, HasOutputCols<T> {
+    Param<Double[]> THRESHOLDS =
+            new DoubleArrayParam(
+                    "thresholds",
+                    "Binarization thresholds, when the input continuous 
feature value is greater"
+                            + " than the threshold, it will be set to 1.0, 
else 0.0.",

Review Comment:
   How about changing the description to as follows? I think the following one 
is more consistent with the description in Spark, and is also clearer in 
grammar.
   
   > The thresholds used to binarize continuous features. Each threshold would 
be used against one input column. If the value of a continuous feature is 
greater than the threshold, it will be binarized to 1.0. If the value is equal 
to or less than the threshold, it will be binarized to 0.0.



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/feature/BinarizerTest.java:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.restartstrategy.RestartStrategies;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.ml.feature.binarizer.Binarizer;
+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.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.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+
+import org.apache.commons.collections.IteratorUtils;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+/** Tests {@link Binarizer}. */
+public class BinarizerTest extends AbstractTestBase {
+
+    private StreamTableEnvironment tEnv;
+    private Table inputDataTable;
+
+    private static final List<Row> INPUT_DATA =
+            Arrays.asList(
+                    Row.of(
+                            1,
+                            Vectors.dense(1, 2),
+                            Vectors.sparse(17, new int[] {0, 3, 9}, new 
double[] {1.0, 2.0, 7.0})),
+                    Row.of(
+                            2,
+                            Vectors.dense(2, 1),
+                            Vectors.sparse(17, new int[] {0, 2, 14}, new 
double[] {5.0, 4.0, 1.0})),
+                    Row.of(
+                            3,
+                            Vectors.dense(5, 18),
+                            Vectors.sparse(
+                                    17, new int[] {0, 11, 12}, new double[] 
{2.0, 4.0, 4.0})));
+
+    private static final Double[] EXPECTED_VALUE_OUTPUT = new Double[] {0.0, 
1.0, 1.0};
+
+    private static final List<Vector> EXPECTED_DENSE_OUTPUT =
+            Arrays.asList(
+                    new DenseVector(new double[] {0.0, 1.0}),
+                    new DenseVector(new double[] {1.0, 0.0}),
+                    new DenseVector(new double[] {1.0, 1.0}));
+
+    private static final List<Vector> EXPECTED_SPARSE_OUTPUT =
+            Arrays.asList(
+                    Vectors.sparse(17, new int[] {9}, new double[] {1.0}),
+                    Vectors.sparse(17, new int[] {0, 2}, new double[] {1.0, 
1.0}),
+                    Vectors.sparse(17, new int[] {11, 12}, new double[] {1.0, 
1.0}));
+
+    @Before
+    public void before() {
+        Configuration config = new Configuration();
+        
config.set(ExecutionCheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, 
true);
+        StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment(config);
+
+        env.setParallelism(4);
+        env.enableCheckpointing(100);
+        env.setRestartStrategy(RestartStrategies.noRestart());
+
+        tEnv = StreamTableEnvironment.create(env);
+        DataStream<Row> dataStream = env.fromCollection(INPUT_DATA);
+        inputDataTable = tEnv.fromDataStream(dataStream).as("f0", "f1", "f2");
+    }
+
+    private void verifyOutputResult(Table output, String[] outputCols) throws 
Exception {
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) 
output).getTableEnvironment();
+        DataStream<Row> stream = tEnv.toDataStream(output);
+
+        List<Row> results = IteratorUtils.toList(stream.executeAndCollect());
+        List<Double> values = new ArrayList<>(results.size());
+        List<Vector> sparseVec = new ArrayList<>(results.size());
+        List<Vector> denseVec = new ArrayList<>(results.size());

Review Comment:
   nit: Let's keep the naming rule consistent across these variables. For 
example, `doubleValues`, `denseVectorValues`, `sparseVectorValues`, or 
`numbers`, `denseVectors`, `sparseVectors`.



-- 
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