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


##########
flink-ml-lib/src/test/java/org/apache/flink/ml/feature/VectorAssemblerTest.java:
##########
@@ -141,10 +140,11 @@ public void testErrorInvalid() {
             Table outputTable = vectorAssembler.transform(inputDataTable)[0];
             outputTable.execute().collect().next();
             Assert.fail("Expected IllegalArgumentException");
-        } catch (Exception e) {
-            assertEquals(
-                    "Input column value should not be null.",
-                    
e.getCause().getCause().getCause().getCause().getCause().getMessage());
+        } catch (Throwable e) {
+            while (e.getCause() != null) {
+                e = e.getCause();
+            }
+            assertEquals("Input column value should not be null.", 
e.getMessage());

Review Comment:
   Just found that we can directly use 
`org.apache.commons.lang3.exception.ExceptionUtils.getRootCauseMessage()` for 
such cases.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/feature/bucketizer/BucketizerParams.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.bucketizer;
+
+import org.apache.flink.ml.common.param.HasHandleInvalid;
+import org.apache.flink.ml.common.param.HasInputCols;
+import org.apache.flink.ml.common.param.HasOutputCols;
+import org.apache.flink.ml.param.DoubleArrayArrayParam;
+import org.apache.flink.ml.param.ParamValidator;
+import org.apache.flink.util.Preconditions;
+
+/**
+ * Params for {@link Bucketizer}.
+ *
+ * <p>The `keep` option of {@link HasHandleInvalid} means that we put the 
invalid data in the last
+ * bucket of the splits, whose index is the number of the buckets.
+ *
+ * @param <T> The class type of this instance.
+ */
+public interface BucketizerParams<T>
+        extends HasInputCols<T>, HasOutputCols<T>, HasHandleInvalid<T> {
+    /**
+     * The array of split points for mapping continuous features into buckets 
for multiple columns.
+     *
+     * <p>Each input column is supposed to be mapped into {numberOfSplitPoints 
- 1} buckets. A
+     * bucket is defined by two split points. For example, bucket(x,y) 
contains values in the range
+     * [x,y). An exception is that the last bucket also contains y. The array 
should contain at
+     * least three split points and be strictly increasing.
+     */
+    DoubleArrayArrayParam SPLITS_ARRAY =
+            new DoubleArrayArrayParam(
+                    "splitsArray",
+                    "Array of split points for mapping continuous features 
into buckets.",
+                    null,
+                    new SplitsArrayValidator());
+
+    default Double[][] getSplitsArray() {
+        return get(SPLITS_ARRAY);
+    }
+
+    default T setSplitsArray(Double[][] value) {
+        set(SPLITS_ARRAY, value);
+        return (T) this;
+    }
+
+    /** Param validator for splitsArray. */
+    class SplitsArrayValidator implements ParamValidator<Double[][]> {
+
+        @Override
+        public boolean validate(Double[][] splitsArray) {
+            Preconditions.checkNotNull(splitsArray);
+            Preconditions.checkArgument(splitsArray.length != 0);
+            for (Double[] splits : splitsArray) {
+                Preconditions.checkArgument(splits.length >= 3);
+                for (int j = 1; j < splits.length; j++) {
+                    Preconditions.checkArgument(splits[j] > splits[j - 1]);

Review Comment:
   Shall we return false so that the external infra can handle it, instead of 
directly throw exception?
   ```suggestion
   if (splits[j] <= splits[j-1]) {
       return false;
   }
   ```



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/feature/bucketizer/Bucketizer.java:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.bucketizer;
+
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+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.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.Collector;
+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;
+
+/**
+ * Bucketizer is a transformer that maps multiple columns of continuous 
features to multiple columns
+ * of discrete features, i.e., buckets IDs.
+ */
+public class Bucketizer implements Transformer<Bucketizer>, 
BucketizerParams<Bucketizer> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+
+    public Bucketizer() {
+        ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+    }
+
+    @Override
+    public Table[] transform(Table... inputs) {
+        Preconditions.checkArgument(inputs.length == 1);
+        String[] inputCols = getInputCols();
+        String[] outputCols = getOutputCols();
+        Double[][] splitsArray = getSplitsArray();
+        Preconditions.checkArgument(inputCols.length == outputCols.length);
+        Preconditions.checkArgument(inputCols.length == splitsArray.length);
+
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) 
inputs[0]).getTableEnvironment();
+
+        RowTypeInfo inputTypeInfo = 
TableUtils.getRowTypeInfo(inputs[0].getResolvedSchema());
+        TypeInformation<?>[] outputTypes = new 
TypeInformation[outputCols.length];
+        Arrays.fill(outputTypes, BasicTypeInfo.INT_TYPE_INFO);
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        ArrayUtils.addAll(inputTypeInfo.getFieldTypes(), 
outputTypes),
+                        ArrayUtils.addAll(inputTypeInfo.getFieldNames(), 
getOutputCols()));
+
+        DataStream<Row> result =
+                tEnv.toDataStream(inputs[0])
+                        .flatMap(
+                                new FindBucketFunction(inputCols, splitsArray, 
getHandleInvalid()),
+                                outputTypeInfo);
+        return new Table[] {tEnv.fromDataStream(result)};
+    }
+
+    /** Finds the bucket index for each continuous feature of an input data 
point. */
+    private static class FindBucketFunction implements FlatMapFunction<Row, 
Row> {
+        private final String[] inputCols;
+        private final String handleInvalid;
+        private final Double[][] splitsArray;
+
+        public FindBucketFunction(
+                String[] inputCols, Double[][] splitsArray, String 
handleInvalid) {
+            this.inputCols = inputCols;
+            this.splitsArray = splitsArray;
+            this.handleInvalid = handleInvalid;
+        }
+
+        @Override
+        public void flatMap(Row value, Collector<Row> out) {
+            Row outputRow = new Row(inputCols.length);
+
+            for (int i = 0; i < inputCols.length; i++) {
+                double feature = ((Number) 
value.getField(inputCols[i])).doubleValue();
+                Double[] splits = splitsArray[i];
+                boolean isInvalid = false;
+
+                if (!Double.isNaN(feature)) {
+                    int index = Arrays.binarySearch(splits, feature);
+                    if (index >= 0) {
+                        if (index == inputCols.length - 1) {
+                            index--;
+                        }
+                        outputRow.setField(i, index);
+                    } else {
+                        index = -index - 1;
+                        if (index == 0 || index == inputCols.length) {
+                            isInvalid = true;
+                        } else {
+                            outputRow.setField(i, index - 1);
+                        }
+                    }
+                } else {
+                    isInvalid = true;
+                }
+
+                if (isInvalid) {
+                    switch (handleInvalid) {
+                        case ERROR_INVALID:
+                            throw new RuntimeException(
+                                    "The input contains invalid value. See "
+                                            + HANDLE_INVALID
+                                            + " parameter for more options.");
+                        case SKIP_INVALID:
+                            return;
+                        case KEEP_INVALID:
+                            outputRow.setField(i, splits.length - 1);
+                            break;
+                        default:
+                            throw new IllegalStateException(

Review Comment:
   nit: `UnsupportedOperationException`



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