lindong28 commented on a change in pull request #56: URL: https://github.com/apache/flink-ml/pull/56#discussion_r840982214
########## File path: flink-ml-lib/src/test/java/org/apache/flink/ml/feature/VectorAssemblerTest.java ########## @@ -0,0 +1,163 @@ +/* + * 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.common.param.HasHandleInvalid; +import org.apache.flink.ml.feature.vectorassembler.VectorAssembler; +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.StageTestUtils; +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.Assert; +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; +import static org.junit.Assert.assertNull; + +/** Tests VectorAssembler. */ +public class VectorAssemblerTest extends AbstractTestBase { + + private StreamTableEnvironment tEnv; + private Table inputDataTable; + + private static final List<Row> INPUT_DATA = + Arrays.asList( + Row.of( + 0, + Vectors.dense(2.1, 3.1), + 1.0, + Vectors.sparse(5, new int[] {3}, new double[] {1.0})), + Row.of( + 1, + Vectors.dense(2.1, 3.1), + 1.0, + Vectors.sparse( + 5, new int[] {4, 2, 3, 1}, new double[] {4.0, 2.0, 3.0, 1.0})), + Row.of(2, null, 1.0, null)); + + private static final SparseVector EXPECTED_OUTPUT_DATA_1 = + Vectors.sparse(8, new int[] {0, 1, 2, 6}, new double[] {2.1, 3.1, 1.0, 1.0}); + private static final DenseVector EXPECTED_OUTPUT_DATA_2 = + Vectors.dense(2.1, 3.1, 1.0, 0.0, 1.0, 2.0, 3.0, 4.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("id", "vec", "num", "sparseVec"); + } + + private void verifyOutputResult(Table output, String outputCol, int outputSize) + throws Exception { + DataStream<Row> dataStream = tEnv.toDataStream(output); + List<Row> results = IteratorUtils.toList(dataStream.executeAndCollect()); + assertEquals(outputSize, results.size()); + for (Row result : results) { + if (result.getField(0) == (Object) 0) { + assertEquals(EXPECTED_OUTPUT_DATA_1, result.getField(outputCol)); + } else if (result.getField(0) == (Object) 1) { + assertEquals(EXPECTED_OUTPUT_DATA_2, result.getField(outputCol)); + } else { + assertNull(result.getField(outputCol)); + } + } + } + + @Test + public void testParam() { + VectorAssembler vectorAssembler = new VectorAssembler(); + assertEquals(HasHandleInvalid.ERROR_INVALID, vectorAssembler.getHandleInvalid()); + assertEquals("output", vectorAssembler.getOutputCol()); + vectorAssembler + .setInputCols("vec", "num", "sparseVec") + .setOutputCol("assembledVec") + .setHandleInvalid(HasHandleInvalid.SKIP_INVALID); + assertArrayEquals(new String[] {"vec", "num", "sparseVec"}, vectorAssembler.getInputCols()); + assertEquals(HasHandleInvalid.SKIP_INVALID, vectorAssembler.getHandleInvalid()); + assertEquals("assembledVec", vectorAssembler.getOutputCol()); + } + + @Test + public void testTransform() throws Exception { Review comment: To make the test names consistent and match what they actually do, would we rename `testTransform` as `testKeepInvalid`, rename `testHandleInvalidOptions` as `testErrorInvalid`, and add a new test named `testSkipInvalid` which does not do save/load? ########## File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/vectorassembler/VectorAssembler.java ########## @@ -0,0 +1,183 @@ +/* + * 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.vectorassembler; + +import org.apache.flink.api.common.functions.FlatMapFunction; +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.common.param.HasHandleInvalid; +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.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.HashMap; +import java.util.LinkedHashMap; +import java.util.Map; + +/** + * A feature transformer that combines a given list of input columns into a vector column. Types of + * input columns must be either vector or numerical value. + */ +public class VectorAssembler + implements Transformer<VectorAssembler>, VectorAssemblerParams<VectorAssembler> { + private final Map<Param<?>, Object> paramMap = new HashMap<>(); + private static final double RATIO = 1.5; + + public VectorAssembler() { + 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()); + RowTypeInfo outputTypeInfo = + new RowTypeInfo( + ArrayUtils.addAll( + inputTypeInfo.getFieldTypes(), TypeInformation.of(Vector.class)), + ArrayUtils.addAll(inputTypeInfo.getFieldNames(), getOutputCol())); + DataStream<Row> output = + tEnv.toDataStream(inputs[0]) + .flatMap( + new AssemblerFunc(getInputCols(), getHandleInvalid()), + outputTypeInfo); + Table outputTable = tEnv.fromDataStream(output); + return new Table[] {outputTable}; + } + + private static class AssemblerFunc implements FlatMapFunction<Row, Row> { + private final String[] inputCols; + private final String handleInvalid; + + public AssemblerFunc(String[] inputCols, String handleInvalid) { + this.inputCols = inputCols; + this.handleInvalid = handleInvalid; + } + + @Override + public void flatMap(Row value, Collector<Row> out) throws Exception { + Object[] objects = new Object[inputCols.length]; + for (int i = 0; i < objects.length; ++i) { + objects[i] = value.getField(inputCols[i]); + } + Vector assembledVector = null; + try { + assembledVector = assemble(objects); + } catch (Exception e) { + switch (handleInvalid) { + case HasHandleInvalid.ERROR_INVALID: + throw e; + case HasHandleInvalid.SKIP_INVALID: + return; + case HasHandleInvalid.KEEP_INVALID: Review comment: When `keep` is specified for this transformer, do we keep all types of invalidity (e.g. missing column, types that are neither number nor vector), or just a selected list of invalid types? And what is the expected output of the transform(...) if it contains invalid input rows? For example, the output row's outputColumn is null? If we plan to keep only a selected types of `invalidity`, as it affects the algorithm behavior, we probably need to explicitly specify it in the Java doc of this class. ########## File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/vectorassembler/VectorAssembler.java ########## @@ -0,0 +1,183 @@ +/* + * 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.vectorassembler; + +import org.apache.flink.api.common.functions.FlatMapFunction; +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.common.param.HasHandleInvalid; +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.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.HashMap; +import java.util.LinkedHashMap; +import java.util.Map; + +/** + * A feature transformer that combines a given list of input columns into a vector column. Types of + * input columns must be either vector or numerical value. + */ +public class VectorAssembler + implements Transformer<VectorAssembler>, VectorAssemblerParams<VectorAssembler> { + private final Map<Param<?>, Object> paramMap = new HashMap<>(); + private static final double RATIO = 1.5; + + public VectorAssembler() { + 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()); + RowTypeInfo outputTypeInfo = + new RowTypeInfo( + ArrayUtils.addAll( + inputTypeInfo.getFieldTypes(), TypeInformation.of(Vector.class)), + ArrayUtils.addAll(inputTypeInfo.getFieldNames(), getOutputCol())); + DataStream<Row> output = + tEnv.toDataStream(inputs[0]) + .flatMap( + new AssemblerFunc(getInputCols(), getHandleInvalid()), + outputTypeInfo); + Table outputTable = tEnv.fromDataStream(output); + return new Table[] {outputTable}; + } + + private static class AssemblerFunc implements FlatMapFunction<Row, Row> { + private final String[] inputCols; + private final String handleInvalid; + + public AssemblerFunc(String[] inputCols, String handleInvalid) { + this.inputCols = inputCols; + this.handleInvalid = handleInvalid; + } + + @Override + public void flatMap(Row value, Collector<Row> out) throws Exception { + Object[] objects = new Object[inputCols.length]; + for (int i = 0; i < objects.length; ++i) { + objects[i] = value.getField(inputCols[i]); + } + Vector assembledVector = null; + try { + assembledVector = assemble(objects); + } catch (Exception e) { + switch (handleInvalid) { + case HasHandleInvalid.ERROR_INVALID: + throw e; + case HasHandleInvalid.SKIP_INVALID: + return; + case HasHandleInvalid.KEEP_INVALID: + out.collect(Row.join(value, Row.of(assembledVector))); + return; + default: Review comment: Should we do `throw new UnsupportedOperationException("handleInvalid=" + handleInvalid + " is not supported")` if none of the case above is matched? Note that if we don't plan to do anything in `default`, it would be simpler to just remove the `default` here. ########## File path: flink-ml-lib/src/main/java/org/apache/flink/ml/common/param/HasHandleInvalid.java ########## @@ -37,13 +37,14 @@ public interface HasHandleInvalid<T> extends WithParams<T> { String ERROR_INVALID = "error"; String SKIP_INVALID = "skip"; + String KEEP_INVALID = "keep"; Review comment: Can you also update the Java doc of this interface to explain what is the semantics of using `keep`? Given that this is a shared parameter, we need the description to be well-defined w.r.t. any estimator/transformer. For example, does `keep` makes sense for fit()? What is the expected behavior if it is applied for fit() and transform() respectively? Could this behavior be defined consistently across all stages, similar to the definition of `skip` and `error`? Or do we have to leave its definition to the specific class that inherits this parameter? -- 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