zhipeng93 commented on code in PR #156: URL: https://github.com/apache/flink-ml/pull/156#discussion_r1016126953
########## docs/content/docs/operators/feature/vectorassembler.md: ########## @@ -26,10 +26,20 @@ under the License. --> ## Vector Assembler - -Vector Assembler combines a given list of input columns into a vector column. -Types of input columns must be either vector or numerical value. - +A Transformer which combines a given list of input columns into a vector column. Input columns +would be numerics or vectors whose size is specified by the {@link #INPUT_SIZES} parameter. Review Comment: nit: `Input columns would be numerics` --> `Input Columns could be numerical or vectors whose sizes are specified by...` Same for the java docs in python and java source code. ########## flink-ml-lib/src/main/java/org/apache/flink/ml/feature/vectorassembler/VectorAssembler.java: ########## @@ -74,64 +85,109 @@ public Table[] transform(Table... inputs) { DataStream<Row> output = tEnv.toDataStream(inputs[0]) .flatMap( - new AssemblerFunc(getInputCols(), getHandleInvalid()), + new AssemblerFunction( + getInputCols(), getHandleInvalid(), getInputSizes()), outputTypeInfo); Table outputTable = tEnv.fromDataStream(output); return new Table[] {outputTable}; } - private static class AssemblerFunc implements FlatMapFunction<Row, Row> { + private static class AssemblerFunction implements FlatMapFunction<Row, Row> { private final String[] inputCols; private final String handleInvalid; + private final Integer[] inputSizes; + private final boolean keepInvalid; - public AssemblerFunc(String[] inputCols, String handleInvalid) { + public AssemblerFunction(String[] inputCols, String handleInvalid, Integer[] inputSizes) { this.inputCols = inputCols; this.handleInvalid = handleInvalid; + this.inputSizes = inputSizes; + keepInvalid = handleInvalid.equals(HasHandleInvalid.KEEP_INVALID); } @Override public void flatMap(Row value, Collector<Row> out) { int nnz = 0; int vectorSize = 0; try { - for (String inputCol : inputCols) { - Object object = value.getField(inputCol); - Preconditions.checkNotNull(object, "Input column value should not be null."); - if (object instanceof Number) { - nnz += 1; - vectorSize += 1; - } else if (object instanceof SparseVector) { - nnz += ((SparseVector) object).indices.length; - vectorSize += ((SparseVector) object).size(); - } else if (object instanceof DenseVector) { - nnz += ((DenseVector) object).size(); - vectorSize += ((DenseVector) object).size(); + for (int i = 0; i < inputCols.length; ++i) { Review Comment: This logic is a bit messy here. How about we split it into several functions? e.g., `assembleNullValues()` and `assembleNotNullValues()`? -- 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