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


##########
flink-ml-lib/src/main/java/org/apache/flink/ml/feature/hashingtf/HashingTF.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.hashingtf;
+
+import org.apache.flink.api.common.functions.MapFunction;
+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.Vectors;
+import org.apache.flink.ml.linalg.typeinfo.VectorTypeInfo;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.internal.TableImpl;
+import org.apache.flink.table.catalog.ResolvedSchema;
+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;
+
+import static 
org.apache.flink.shaded.guava30.com.google.common.hash.Hashing.murmur3_32;
+
+/**
+ * A Transformer that maps a sequence of terms(strings, numbers, booleans) to 
a sparse vector with a

Review Comment:
   nit: This should be "vector" instead of "sparse vector" corresponding to the 
`VectorTypeInfo.INSTANCE` used when generating `outputTypeInfo`.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/feature/hashingtf/HashingTF.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.hashingtf;
+
+import org.apache.flink.api.common.functions.MapFunction;
+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.Vectors;
+import org.apache.flink.ml.linalg.typeinfo.VectorTypeInfo;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.internal.TableImpl;
+import org.apache.flink.table.catalog.ResolvedSchema;
+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;
+
+import static 
org.apache.flink.shaded.guava30.com.google.common.hash.Hashing.murmur3_32;
+
+/**
+ * A Transformer that maps a sequence of terms(strings, numbers, booleans) to 
a sparse vector with a
+ * specified dimension using the hashing trick.
+ *
+ * <p>If multiple features are projected into the same column, the output 
values are accumulated by
+ * default. Users could also enforce all non-zero output values as 1 by 
setting {@link
+ * HashingTFParams#BINARY} as true.
+ *
+ * <p>For the hashing trick, see https://en.wikipedia.org/wiki/Feature_hashing 
for details.
+ */
+public class HashingTF implements Transformer<HashingTF>, 
HashingTFParams<HashingTF> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+
+    private static final 
org.apache.flink.shaded.guava30.com.google.common.hash.HashFunction
+            HASH_FUNC = murmur3_32(0);

Review Comment:
   Spark seems to use 42 as the hash seed[1]. Do you know why Spark chooses 
this value?
   
   If this value in Spark was just a personal choice, I agree that we can use 0 
for the seed in our implementation.
   
   [1] 
https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala#L143



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/feature/hashingtf/HashingTF.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.hashingtf;
+
+import org.apache.flink.api.common.functions.MapFunction;
+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.Vectors;
+import org.apache.flink.ml.linalg.typeinfo.VectorTypeInfo;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.internal.TableImpl;
+import org.apache.flink.table.catalog.ResolvedSchema;
+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;
+
+import static 
org.apache.flink.shaded.guava30.com.google.common.hash.Hashing.murmur3_32;
+
+/**
+ * A Transformer that maps a sequence of terms(strings, numbers, booleans) to 
a sparse vector with a
+ * specified dimension using the hashing trick.
+ *
+ * <p>If multiple features are projected into the same column, the output 
values are accumulated by
+ * default. Users could also enforce all non-zero output values as 1 by 
setting {@link
+ * HashingTFParams#BINARY} as true.
+ *
+ * <p>For the hashing trick, see https://en.wikipedia.org/wiki/Feature_hashing 
for details.
+ */
+public class HashingTF implements Transformer<HashingTF>, 
HashingTFParams<HashingTF> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+
+    private static final 
org.apache.flink.shaded.guava30.com.google.common.hash.HashFunction
+            HASH_FUNC = murmur3_32(0);
+
+    public HashingTF() {
+        ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+    }
+
+    @Override
+    public Table[] transform(Table... inputs) {
+        Preconditions.checkArgument(inputs.length == 1);
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) 
inputs[0]).getTableEnvironment();
+
+        ResolvedSchema tableSchema = inputs[0].getResolvedSchema();
+        RowTypeInfo inputTypeInfo = TableUtils.getRowTypeInfo(tableSchema);
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        ArrayUtils.addAll(inputTypeInfo.getFieldTypes(), 
VectorTypeInfo.INSTANCE),
+                        ArrayUtils.addAll(inputTypeInfo.getFieldNames(), 
getOutputCol()));
+
+        DataStream<Row> output =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                new HashTFFunction(getInputCol(), getBinary(), 
getNumFeatures()),
+                                outputTypeInfo);
+        return new Table[] {tEnv.fromDataStream(output)};
+    }
+
+    @Override
+    public void save(String path) throws IOException {
+        ReadWriteUtils.saveMetadata(this, path);
+    }
+
+    @Override
+    public Map<Param<?>, Object> getParamMap() {
+        return paramMap;
+    }
+
+    public static HashingTF load(StreamTableEnvironment tEnv, String path) 
throws IOException {
+        return ReadWriteUtils.loadStageParam(path);
+    }
+
+    /** The main logic of {@link HashingTF}, which converts the input to a 
sparse vector. */
+    public static class HashTFFunction implements MapFunction<Row, Row> {
+        private final String inputCol;
+        private final boolean binary;
+        private final int numFeatures;
+
+        public HashTFFunction(String inputCol, boolean binary, int 
numFeatures) {
+            this.inputCol = inputCol;
+            this.binary = binary;
+            this.numFeatures = numFeatures;
+        }
+
+        @Override
+        public Row map(Row row) throws Exception {
+            Object inputObj = row.getField(inputCol);
+
+            Iterable<Object> inputList;
+            if (inputObj.getClass().isArray()) {
+                inputList = Arrays.asList((Object[]) inputObj);
+            } else if (inputObj instanceof Iterable) {
+                inputList = (Iterable<Object>) inputObj;
+            } else {
+                throw new IllegalArgumentException(

Review Comment:
   It might be better to do this check in `transform()` method and throw 
exceptions as early as during compilation.



##########
flink-ml-python/pyflink/ml/lib/feature/tests/test_hashingtf.py:
##########
@@ -0,0 +1,115 @@
+################################################################################
+#  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.
+################################################################################
+import os
+
+from pyflink.common import Types
+
+from pyflink.ml.core.linalg import Vectors
+from pyflink.ml.lib.feature.hashingtf import HashingTF
+from pyflink.ml.tests.test_utils import PyFlinkMLTestCase
+
+
+class HashingTFTest(PyFlinkMLTestCase):
+    def setUp(self):
+        super(HashingTFTest, self).setUp()
+        self.input_data_table = self.t_env.from_data_stream(
+            self.env.from_collection([
+                (['HashingTFTest', 'Hashing', 'Term', 'Frequency', 'Test'],),
+                (['HashingTFTest', 'Hashing', 'Hashing', 'Test', 'Test'],),
+            ],
+                type_info=Types.ROW_NAMED(
+                    ["input", ],
+                    [Types.OBJECT_ARRAY(Types.STRING())])))
+
+        self.expected_output = [
+            Vectors.sparse(262144, [67564, 89917, 113827, 131486, 228971],
+                           [1.0, 1.0, 1.0, 1.0, 1.0]),
+            Vectors.sparse(262144, [67564, 131486, 228971], [1.0, 2.0, 2.0])
+        ]
+
+        self.expected_binary_output = [
+            Vectors.sparse(262144, [67564, 89917, 113827, 131486, 228971],
+                           [1.0, 1.0, 1.0, 1.0, 1.0]),
+            Vectors.sparse(262144, [67564, 131486, 228971], [1.0, 1.0, 1.0])
+        ]
+
+    def test_param(self):
+        hashing_tf = HashingTF()
+        self.assertEqual('input', hashing_tf.input_col)
+        self.assertFalse(hashing_tf.binary)
+        self.assertEqual(262144, hashing_tf.num_features)
+        self.assertEqual('output', hashing_tf.output_col)
+
+        hashing_tf.set_input_col("test_input_col") \
+            .set_binary(True) \
+            .set_num_features(1024) \
+            .set_output_col("test_output_col")
+
+        self.assertEqual('test_input_col', hashing_tf.input_col)
+        self.assertTrue(hashing_tf.binary)
+        self.assertEqual(1024, hashing_tf.num_features)
+        self.assertEqual('test_output_col', hashing_tf.output_col)
+
+    def test_output_schema(self):
+        hashing_tf = HashingTF()
+        input_data_table = self.t_env.from_data_stream(
+            self.env.from_collection([
+                ([''], ''),
+            ],
+                type_info=Types.ROW_NAMED(
+                    ['input', 'dummy_input'],
+                    [Types.OBJECT_ARRAY(Types.STRING()), Types.STRING()])))
+
+        output = hashing_tf \
+            .set_input_col('input') \
+            .set_output_col('output') \
+            .transform(input_data_table)[0]
+
+        self.assertEqual(
+            [hashing_tf.input_col, 'dummy_input', hashing_tf.output_col],
+            output.get_schema().get_field_names())
+
+    def verify_output_result(self, output_table, expected_output):
+        predicted_result = [result[1] for result in
+                            
self.t_env.to_data_stream(output_table).execute_and_collect()]
+        expected_output.sort(key=lambda x: x[89917])
+        predicted_result.sort(key=lambda x: x[89917])
+        self.assertEqual(len(expected_output), len(predicted_result))
+
+        for i in range(len(expected_output)):
+            self.assertEqual(expected_output[i], predicted_result[i])
+
+    def test_transform(self):
+        hashing_tf = HashingTF()
+
+        # Tests non-binary.
+        output = hashing_tf.transform(self.input_data_table)[0]
+        self.verify_output_result(output, self.expected_output)
+
+        # Tests non-binary.

Review Comment:
   nit: this should be binary.



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