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


##########
flink-ml-lib/src/main/java/org/apache/flink/ml/feature/vectorslicer/VectorSlicerParams.java:
##########
@@ -0,0 +1,47 @@
+/*
+ * 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.vectorslicer;
+
+import org.apache.flink.ml.common.param.HasInputCol;
+import org.apache.flink.ml.common.param.HasOutputCol;
+import org.apache.flink.ml.param.IntArrayParam;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+
+/**
+ * Params of {@link VectorSlicer}.
+ *
+ * @param <T> The class type of this instance.
+ */
+public interface VectorSlicerParams<T> extends HasInputCol<T>, HasOutputCol<T> 
{
+    Param<Integer[]> INDICES =
+            new IntArrayParam(
+                    "indices",
+                    "An array of indices to select features from a vector 
column.",
+                    null,
+                    ParamValidators.numericalArrayGtEq(0));

Review Comment:
   I just found that in order to align the behavior of this algorithm with 
existing practices, we need to make sure that the input indices are not 
duplicated. Could you please improve the param validator with this function, 
add JavaDoc to explain the behavior when there are duplicated input indices, 
and add corresponding tests?
   
   Given that we need to verify the input integers are non-negative and 
non-duplicate, do you think it would be better to add a unique private param 
validator in this class to verify this specified case, instead of trying to add 
a common validator?



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/feature/vectorslicer/VectorSlicer.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.vectorslicer;
+
+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.DenseVector;
+import org.apache.flink.ml.linalg.SparseVector;
+import org.apache.flink.ml.linalg.Vector;
+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.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;
+
+/**
+ * VectorSlicer is a transformer that transforms a vector to a new one with a 
sub-array of the
+ * original features. It is useful for extracting features from a given 
vector. If the indices
+ * acquired from setIndices() are not in order, the indices of the result 
vector will be sorted. If

Review Comment:
   The existing practice in Spark seems not to re-order the given indices 
before slicing, which means if the provided indices are not in order, it will 
arrange the indices in the order given. Could you please modify the 
corresponding implementation, unit test and JavaDocs?



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/feature/VectorSlicerTest.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.vectorslicer.VectorSlicer;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.linalg.SparseVector;
+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.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.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+/** Tests {@link VectorSlicer}. */
+public class VectorSlicerTest extends AbstractTestBase {

Review Comment:
   Could you please add unit tests for the following corner cases?
   - The `indices` is an empty array.
   - The length of the vectors is not equal, but the max value in the `indices` 
is smaller than the minimum length.
   - The length of the vectors is not equal, and the max value in the `indices` 
is larger than the minimum length.



##########
flink-ml-python/pyflink/ml/lib/feature/vectorslicer.py:
##########
@@ -0,0 +1,71 @@
+################################################################################
+#  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.
+################################################################################
+
+from typing import Tuple
+from pyflink.ml.core.wrapper import JavaWithParams
+from pyflink.ml.core.param import IntArrayParam, ParamValidators
+from pyflink.ml.lib.feature.common import JavaFeatureTransformer
+from pyflink.ml.lib.param import HasInputCol, HasOutputCol, Param
+
+
+class _VectorSlicerParams(
+    JavaWithParams,
+    HasInputCol,
+    HasOutputCol
+):
+    """
+    Params for :class:`VectorSlicer`.
+    """
+
+    INDICES: Param[Tuple[int, ...]] = IntArrayParam(
+        "indices",
+        "An array of indices to select features from a vector column.",
+        None,
+        ParamValidators.numerical_array_gt_eq(0))
+
+    def __init__(self, java_params):
+        super(_VectorSlicerParams, self).__init__(java_params)
+
+    def set_indices(self, *ind: int):
+        return self.set(self.INDICES, ind)
+
+    def get_indices(self) -> Tuple[int, ...]:
+        return self.get(self.INDICES)
+
+    @property
+    def indices(self) -> Tuple[int, ...]:
+        return self.get_indices()
+
+
+class VectorSlicer(JavaFeatureTransformer, _VectorSlicerParams):
+    """
+    VectorSlicer is a transformer that transforms a vector to a new one with a 
sub-array of the
+    original features. It is useful for extracting features from a given 
vector. If the indices
+    acquired from setIndices() are not in order, the indices of the result 
vector will be sorted.

Review Comment:
   The comment here is inconsistent with that in JavaDoc.



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