You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2021/05/08 03:13:15 UTC

[GitHub] [iceberg] zhangjun0x01 opened a new pull request #2566: Flink : Vectorize read of orc format in flink

zhangjun0x01 opened a new pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566


   Vectorize read for flink, now I have completed the vectorized read in orc format,I will do parquet format later.


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r629012019



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java
##########
@@ -91,11 +102,77 @@ public void configure(Configuration parameters) {
 
   @Override
   public void open(FlinkInputSplit split) {
+    boolean enableVectorizedRead = readableConfig.get(FlinkTableOptions.ENABLE_VECTORIZED_READ);
+
+    if (enableVectorizedRead) {
+      if (useOrcVectorizedRead()) {

Review comment:
       There are other required requisition so that we could apply vectorized read:
   1.   All those files from the `CombinedScanTask` are data files,  if there is a delete file, the current deletions apply process are compared row by row which will disable the vectorized read actually. 
   2.  All the files from `CombinedScanTask` must be ORC files.
   3.  All the columns to read should all be primitives,  that means all the byte width should be the same size.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r630947942



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/ConstantColumnVectors.java
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import org.apache.flink.table.data.vector.BooleanColumnVector;
+import org.apache.flink.table.data.vector.BytesColumnVector;
+import org.apache.flink.table.data.vector.DecimalColumnVector;
+import org.apache.flink.table.data.vector.DoubleColumnVector;
+import org.apache.flink.table.data.vector.FloatColumnVector;
+import org.apache.flink.table.data.vector.IntColumnVector;
+import org.apache.flink.table.data.vector.LongColumnVector;
+import org.apache.flink.table.data.vector.TimestampColumnVector;
+
+class ConstantColumnVectors {
+  private ConstantColumnVectors() {
+  }
+
+  static IntOrcColumnVector ints(Object constant) {
+    return new IntOrcColumnVector(constant);
+  }
+
+  static LongOrcColumnVector longs(Object constant) {
+    return new LongOrcColumnVector(constant);
+  }
+
+  static BooleanOrcColumnVector booleans(Object constant) {
+    return new BooleanOrcColumnVector(constant);
+  }
+
+  static DoubleOrcColumnVector doubles(Object constant) {
+    return new DoubleOrcColumnVector(constant);
+  }
+
+  static FloatOrcColumnVector floats(Object constant) {
+    return new FloatOrcColumnVector(constant);
+  }
+
+  static DecimalOrcColumnVector decimals(Object constant) {
+    return new DecimalOrcColumnVector(constant);
+  }
+
+  static TimestampOrcColumnVector timestamps(Object constant) {
+    return new TimestampOrcColumnVector(constant);
+  }
+
+  static BytesOrcColumnVector bytes(Object constant) {
+    return new BytesOrcColumnVector(constant);
+  }
+
+  private static class IntOrcColumnVector implements IntColumnVector {
+
+    private Object constant;
+
+    private IntOrcColumnVector(Object constant) {
+      this.constant = constant;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return false;

Review comment:
       the code we get the `idToConstant` is 
   ```
       Map<Integer, ?> idToConstant = partitionSchema.columns().isEmpty() ? ImmutableMap.of() :
           PartitionUtil.constantsMap(task, RowDataUtil::convertConstant);
   ```
   so I think there should  be no `null` value in `idToConstant `, what do you think ?
   




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r630672091



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        FlinkRowColumnVector cv = (FlinkRowColumnVector) converter.convert(
+            new StructColumnVector(batch.size, batch.cols), batch.size, batchOffsetInFile);

Review comment:
       Looks like the `batchOffsetInFile` is used for metacolumn to get the correct row offset for a given row, mainly used for iceberg format v2.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r632356036



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        FlinkRowColumnVector cv = (FlinkRowColumnVector) converter.convert(
+            new StructColumnVector(batch.size, batch.cols), batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) {
+      return (vector, batchSize, batchOffsetInFile) -> {
+        if (vector instanceof LongColumnVector) {
+          return new OrcNoHiveLongVector((LongColumnVector) vector);
+        } else if (vector instanceof DoubleColumnVector) {
+          return new OrcNoHiveDoubleVector((DoubleColumnVector) vector);
+        } else if (vector instanceof BytesColumnVector) {
+          return new OrcNoHiveBytesVector((BytesColumnVector) vector);
+        } else if (vector instanceof DecimalColumnVector) {
+          return new OrcNoHiveDecimalVector((DecimalColumnVector) vector);
+        } else if (vector instanceof TimestampColumnVector) {
+          return new OrcNoHiveTimestampVector((TimestampColumnVector) vector);
+        } else {
+          throw new UnsupportedOperationException(
+              "Unsupported vector: " + vector.getClass().getName());
+        }
+      };
+    }
+  }
+
+  private static class LongOrcColumnVector implements org.apache.flink.table.data.vector.LongColumnVector {
+    private long batchOffsetInFile;
+
+    LongOrcColumnVector(long batchOffsetInFile) {
+      this.batchOffsetInFile = batchOffsetInFile;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return false;
+    }
+
+    @Override
+    public long getLong(int i) {
+      return batchOffsetInFile + i;
+    }
+  }
+
+  private static class StructConverter implements Converter {
+    private final Types.StructType structType;
+    private final List<Converter> fieldConverters;
+    private final Map<Integer, ?> idToConstant;
+
+    private StructConverter(Types.StructType structType, List<Converter> fieldConverters,
+                            Map<Integer, ?> idToConstant) {
+      this.structType = structType;
+      this.fieldConverters = fieldConverters;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector vector, int batchSize,
+                                long batchOffsetInFile) {
+      StructColumnVector structVector = (StructColumnVector) vector;
+      List<Types.NestedField> fields = structType.fields();
+      ColumnVector[] fieldVectors = new ColumnVector[fields.size()];
+      for (int pos = 0; pos < fields.size(); pos++) {
+        Types.NestedField field = fields.get(pos);
+        if (idToConstant.containsKey(field.fieldId())) {
+          fieldVectors[pos] = getConstantVectors(field.type(), idToConstant.get(field.fieldId()));
+        } else if (field.equals(MetadataColumns.ROW_POSITION)) {
+          fieldVectors[pos] = new LongOrcColumnVector(batchOffsetInFile);
+        } else {
+          fieldVectors[pos] = fieldConverters.get(pos)
+              .convert(structVector.fields[pos], batchSize, batchOffsetInFile);
+        }
+      }
+
+      return new FlinkRowColumnVector(fieldVectors, structVector);
+    }
+
+    private ColumnVector getConstantVectors(Type type, Object constant) {
+      if (type.isPrimitiveType()) {
+        Type.TypeID typeID = type.typeId();
+        switch (typeID) {
+          case INTEGER:
+          case DATE:
+          case TIME:
+            return ConstantColumnVectors.ints(constant);
+
+          case LONG:
+            return ConstantColumnVectors.longs(constant);
+
+          case BOOLEAN:
+            return ConstantColumnVectors.booleans(constant);
+
+          case DOUBLE:
+            return ConstantColumnVectors.doubles(constant);
+
+          case FLOAT:
+            return ConstantColumnVectors.floats(constant);
+
+          case DECIMAL:
+            return ConstantColumnVectors.decimals(constant);
+
+          case TIMESTAMP:
+            return ConstantColumnVectors.timestamps(constant);
+
+          case STRING:
+            return ConstantColumnVectors.bytes(constant);

Review comment:
       Okay,  I read the  [RowDataUtil#convertConstant](https://github.com/apache/iceberg/blob/90225d6c9413016d611e2ce5eff37db1bc1b4fc5/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java#L57),   the `STRING`'s data type is `StringData` and currently the `ConstantBytesColumnVector` could handle the `StringData`.  So it should be OK. I will mark this conversation as closed.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r635794913



##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/vectorized/TestVectorizedReads.java
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.math.BigDecimal;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.flink.FlinkCatalogTestBase;
+import org.apache.iceberg.flink.FlinkTableOptions;
+import org.apache.iceberg.flink.TestHelpers;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestVectorizedReads extends FlinkCatalogTestBase {

Review comment:
       I think we should add `MapColumnVector` in flink instead of iceberg,so I open  an issue in flink [FLINK-22591](https://issues.apache.org/jira/browse/FLINK-22591) , if  we add `MapColumnVector` in iceberg, and flink also add `MapColumnVector` in the futher, if the implementation is different, how do we resolve this conflict? 




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r630680833



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        FlinkRowColumnVector cv = (FlinkRowColumnVector) converter.convert(
+            new StructColumnVector(batch.size, batch.cols), batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) {
+      return (vector, batchSize, batchOffsetInFile) -> {
+        if (vector instanceof LongColumnVector) {
+          return new OrcNoHiveLongVector((LongColumnVector) vector);
+        } else if (vector instanceof DoubleColumnVector) {
+          return new OrcNoHiveDoubleVector((DoubleColumnVector) vector);
+        } else if (vector instanceof BytesColumnVector) {
+          return new OrcNoHiveBytesVector((BytesColumnVector) vector);
+        } else if (vector instanceof DecimalColumnVector) {
+          return new OrcNoHiveDecimalVector((DecimalColumnVector) vector);
+        } else if (vector instanceof TimestampColumnVector) {
+          return new OrcNoHiveTimestampVector((TimestampColumnVector) vector);
+        } else {
+          throw new UnsupportedOperationException(
+              "Unsupported vector: " + vector.getClass().getName());
+        }
+      };
+    }
+  }
+
+  private static class LongOrcColumnVector implements org.apache.flink.table.data.vector.LongColumnVector {
+    private long batchOffsetInFile;
+
+    LongOrcColumnVector(long batchOffsetInFile) {
+      this.batchOffsetInFile = batchOffsetInFile;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return false;
+    }
+
+    @Override
+    public long getLong(int i) {
+      return batchOffsetInFile + i;
+    }
+  }
+
+  private static class StructConverter implements Converter {
+    private final Types.StructType structType;
+    private final List<Converter> fieldConverters;
+    private final Map<Integer, ?> idToConstant;
+
+    private StructConverter(Types.StructType structType, List<Converter> fieldConverters,
+                            Map<Integer, ?> idToConstant) {
+      this.structType = structType;
+      this.fieldConverters = fieldConverters;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector vector, int batchSize,
+                                long batchOffsetInFile) {
+      StructColumnVector structVector = (StructColumnVector) vector;
+      List<Types.NestedField> fields = structType.fields();
+      ColumnVector[] fieldVectors = new ColumnVector[fields.size()];
+      for (int pos = 0; pos < fields.size(); pos++) {
+        Types.NestedField field = fields.get(pos);
+        if (idToConstant.containsKey(field.fieldId())) {
+          fieldVectors[pos] = getConstantVectors(field.type(), idToConstant.get(field.fieldId()));
+        } else if (field.equals(MetadataColumns.ROW_POSITION)) {
+          fieldVectors[pos] = new LongOrcColumnVector(batchOffsetInFile);
+        } else {
+          fieldVectors[pos] = fieldConverters.get(pos)
+              .convert(structVector.fields[pos], batchSize, batchOffsetInFile);
+        }
+      }
+
+      return new FlinkRowColumnVector(fieldVectors, structVector);
+    }
+
+    private ColumnVector getConstantVectors(Type type, Object constant) {
+      if (type.isPrimitiveType()) {

Review comment:
       We don't need this `type.isPrimitiveType()` any more because we've checked each data type in the following switch-case sentence.  We can throw an exception when we encounter non-primitive types that cannot be handled.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r630717897



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java
##########
@@ -91,11 +105,85 @@ public void configure(Configuration parameters) {
 
   @Override
   public void open(FlinkInputSplit split) {
+    boolean enableVectorizedRead = readableConfig.get(FlinkTableOptions.ENABLE_VECTORIZED_READ);
+
+    if (enableVectorizedRead) {
+      if (useOrcVectorizedRead(split.getTask())) {

Review comment:
       I think this check should be moved inside `BatchRowDateIterator`. Here the logic should be simple 
   ```
   if (enableVectorizedRead) {
     ... new BatchRowDataIterator
   } else {
      ... new RowDataIterator
   }




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r637651085



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/BatchRowDataIterator.java
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.iceberg.flink.source;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileContent;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.flink.data.RowDataUtil;
+import org.apache.iceberg.flink.data.vectorized.VectorizedFlinkOrcReaders;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PartitionUtil;
+import org.jetbrains.annotations.NotNull;
+
+class BatchRowDataIterator extends DataIterator<RowData> {
+
+  private final Schema tableSchema;
+  private final Schema projectedSchema;
+  private final String nameMapping;
+  private final boolean caseSensitive;
+  private final DataType[] dataTypes;
+
+  BatchRowDataIterator(CombinedScanTask task, FileIO io, EncryptionManager encryption, Schema tableSchema,
+                       Schema projectedSchema, String nameMapping, boolean caseSensitive, DataType[] dataTypes) {
+    super(task, io, encryption);
+    this.tableSchema = tableSchema;
+    this.projectedSchema = projectedSchema;
+    this.nameMapping = nameMapping;
+    this.caseSensitive = caseSensitive;
+    this.dataTypes = dataTypes;

Review comment:
       I was referring to the `useOrcVectorizedRead` method on the `DataType[]` that eventually calls `isVectorizationUnsupported` for each field to validate supported or not. The validation seems only applied at top-level fields.
   
   also a nit on the naming of `isVectorizationUnsupported`. Maybe rename and change it `isVectorizationSupported`?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r631586933



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/ConstantColumnVectors.java
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import org.apache.flink.table.data.vector.BooleanColumnVector;
+import org.apache.flink.table.data.vector.BytesColumnVector;
+import org.apache.flink.table.data.vector.DecimalColumnVector;
+import org.apache.flink.table.data.vector.DoubleColumnVector;
+import org.apache.flink.table.data.vector.FloatColumnVector;
+import org.apache.flink.table.data.vector.IntColumnVector;
+import org.apache.flink.table.data.vector.LongColumnVector;
+import org.apache.flink.table.data.vector.TimestampColumnVector;
+
+class ConstantColumnVectors {
+  private ConstantColumnVectors() {
+  }
+
+  static IntOrcColumnVector ints(Object constant) {
+    return new IntOrcColumnVector(constant);
+  }
+
+  static LongOrcColumnVector longs(Object constant) {
+    return new LongOrcColumnVector(constant);
+  }
+
+  static BooleanOrcColumnVector booleans(Object constant) {
+    return new BooleanOrcColumnVector(constant);
+  }
+
+  static DoubleOrcColumnVector doubles(Object constant) {
+    return new DoubleOrcColumnVector(constant);
+  }
+
+  static FloatOrcColumnVector floats(Object constant) {
+    return new FloatOrcColumnVector(constant);
+  }
+
+  static DecimalOrcColumnVector decimals(Object constant) {
+    return new DecimalOrcColumnVector(constant);
+  }
+
+  static TimestampOrcColumnVector timestamps(Object constant) {
+    return new TimestampOrcColumnVector(constant);
+  }
+
+  static BytesOrcColumnVector bytes(Object constant) {
+    return new BytesOrcColumnVector(constant);
+  }
+
+  private static class IntOrcColumnVector implements IntColumnVector {
+
+    private Object constant;
+
+    private IntOrcColumnVector(Object constant) {
+      this.constant = constant;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return false;

Review comment:
       ok,I updated it




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r630679099



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        FlinkRowColumnVector cv = (FlinkRowColumnVector) converter.convert(
+            new StructColumnVector(batch.size, batch.cols), batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) {
+      return (vector, batchSize, batchOffsetInFile) -> {
+        if (vector instanceof LongColumnVector) {
+          return new OrcNoHiveLongVector((LongColumnVector) vector);
+        } else if (vector instanceof DoubleColumnVector) {
+          return new OrcNoHiveDoubleVector((DoubleColumnVector) vector);
+        } else if (vector instanceof BytesColumnVector) {
+          return new OrcNoHiveBytesVector((BytesColumnVector) vector);
+        } else if (vector instanceof DecimalColumnVector) {
+          return new OrcNoHiveDecimalVector((DecimalColumnVector) vector);
+        } else if (vector instanceof TimestampColumnVector) {
+          return new OrcNoHiveTimestampVector((TimestampColumnVector) vector);
+        } else {
+          throw new UnsupportedOperationException(
+              "Unsupported vector: " + vector.getClass().getName());
+        }
+      };
+    }
+  }
+
+  private static class LongOrcColumnVector implements org.apache.flink.table.data.vector.LongColumnVector {
+    private long batchOffsetInFile;
+
+    LongOrcColumnVector(long batchOffsetInFile) {
+      this.batchOffsetInFile = batchOffsetInFile;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return false;
+    }
+
+    @Override
+    public long getLong(int i) {
+      return batchOffsetInFile + i;
+    }
+  }
+
+  private static class StructConverter implements Converter {
+    private final Types.StructType structType;
+    private final List<Converter> fieldConverters;
+    private final Map<Integer, ?> idToConstant;
+
+    private StructConverter(Types.StructType structType, List<Converter> fieldConverters,
+                            Map<Integer, ?> idToConstant) {
+      this.structType = structType;
+      this.fieldConverters = fieldConverters;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector vector, int batchSize,
+                                long batchOffsetInFile) {
+      StructColumnVector structVector = (StructColumnVector) vector;
+      List<Types.NestedField> fields = structType.fields();
+      ColumnVector[] fieldVectors = new ColumnVector[fields.size()];
+      for (int pos = 0; pos < fields.size(); pos++) {
+        Types.NestedField field = fields.get(pos);
+        if (idToConstant.containsKey(field.fieldId())) {
+          fieldVectors[pos] = getConstantVectors(field.type(), idToConstant.get(field.fieldId()));
+        } else if (field.equals(MetadataColumns.ROW_POSITION)) {
+          fieldVectors[pos] = new LongOrcColumnVector(batchOffsetInFile);
+        } else {
+          fieldVectors[pos] = fieldConverters.get(pos)
+              .convert(structVector.fields[pos], batchSize, batchOffsetInFile);
+        }
+      }
+
+      return new FlinkRowColumnVector(fieldVectors, structVector);
+    }
+
+    private ColumnVector getConstantVectors(Type type, Object constant) {

Review comment:
       This is generating the flink's `ConstantColumnVectors` based on the iceberg data type,  I'd prefer to name it as `toConstantColumnVector`.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r636084929



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        FlinkRowColumnVector cv = (FlinkRowColumnVector) converter.convert(
+            new StructColumnVector(batch.size, batch.cols), batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException("Unsupported vectorized read for map type.");
+    }
+
+    @Override
+    public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) {
+      return (vector, batchSize, batchOffsetInFile) -> {
+        if (vector instanceof LongColumnVector) {
+          return new OrcNoHiveLongVector((LongColumnVector) vector);
+        } else if (vector instanceof DoubleColumnVector) {
+          return new OrcNoHiveDoubleVector((DoubleColumnVector) vector);
+        } else if (vector instanceof BytesColumnVector) {
+          return new OrcNoHiveBytesVector((BytesColumnVector) vector);
+        } else if (vector instanceof DecimalColumnVector) {
+          return new OrcNoHiveDecimalVector((DecimalColumnVector) vector);
+        } else if (vector instanceof TimestampColumnVector) {
+          return new OrcNoHiveTimestampVector((TimestampColumnVector) vector);
+        } else {
+          throw new UnsupportedOperationException(
+              "Unsupported vector: " + vector.getClass().getName());
+        }
+      };
+    }
+  }
+
+  private static class RowPositionColumnVector implements org.apache.flink.table.data.vector.LongColumnVector {
+    private final long batchOffsetInFile;
+
+    RowPositionColumnVector(long batchOffsetInFile) {
+      this.batchOffsetInFile = batchOffsetInFile;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return false;
+    }
+
+    @Override
+    public long getLong(int i) {
+      return batchOffsetInFile + i;
+    }
+  }
+
+  private static class StructConverter implements Converter {
+    private final Types.StructType structType;
+    private final List<Converter> fieldConverters;
+    private final Map<Integer, ?> idToConstant;
+
+    private StructConverter(Types.StructType structType, List<Converter> fieldConverters,
+                            Map<Integer, ?> idToConstant) {
+      this.structType = structType;
+      this.fieldConverters = fieldConverters;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector vector, int batchSize,
+                                long batchOffsetInFile) {
+      StructColumnVector structVector = (StructColumnVector) vector;
+      List<Types.NestedField> fields = structType.fields();
+      ColumnVector[] fieldVectors = new ColumnVector[fields.size()];
+      for (int pos = 0; pos < fields.size(); pos++) {
+        Types.NestedField field = fields.get(pos);
+        if (idToConstant.containsKey(field.fieldId())) {
+          fieldVectors[pos] = toConstantColumnVector(field.type(), idToConstant.get(field.fieldId()));
+        } else if (field.equals(MetadataColumns.ROW_POSITION)) {
+          fieldVectors[pos] = new RowPositionColumnVector(batchOffsetInFile);
+        } else {
+          fieldVectors[pos] = fieldConverters.get(pos)
+              .convert(structVector.fields[pos], batchSize, batchOffsetInFile);
+        }
+      }
+
+      return new FlinkRowColumnVector(fieldVectors, structVector);
+    }
+
+    private ColumnVector toConstantColumnVector(Type type, Object constant) {
+      Type.TypeID typeID = type.typeId();
+      switch (typeID) {
+        case INTEGER:
+        case DATE:
+        case TIME:
+          return ConstantColumnVectors.ints(constant);
+
+        case LONG:
+          return ConstantColumnVectors.longs(constant);
+
+        case BOOLEAN:
+          return ConstantColumnVectors.booleans(constant);
+
+        case DOUBLE:
+          return ConstantColumnVectors.doubles(constant);
+
+        case FLOAT:
+          return ConstantColumnVectors.floats(constant);
+
+        case DECIMAL:
+          return ConstantColumnVectors.decimals(constant);
+
+        case TIMESTAMP:
+          return ConstantColumnVectors.timestamps(constant);
+
+        case FIXED:
+        case UUID:
+        case BINARY:

Review comment:
       > I still don't think we could use the `ConstantBytesColumnVector` to read the `BINARY` & `FIXED` data type because its constant object is a [StringData](https://github.com/apache/iceberg/blob/90225d6c9413016d611e2ce5eff37db1bc1b4fc5/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java#L66), we will definitely encounter a class cast exception when cast the `byte[]` to `BinaryStringData` , do we have an unit test to cover this ?
   
   from the [code](https://github.com/apache/iceberg/blob/90225d6c9413016d611e2ce5eff37db1bc1b4fc5/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java#L66) ,for `FIXED ` data type,it will be convert to byte[],so I I added the judgment that constant is byte[] instead of throw a cast exception. and add the UT for byte[] data type




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r630680833



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        FlinkRowColumnVector cv = (FlinkRowColumnVector) converter.convert(
+            new StructColumnVector(batch.size, batch.cols), batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) {
+      return (vector, batchSize, batchOffsetInFile) -> {
+        if (vector instanceof LongColumnVector) {
+          return new OrcNoHiveLongVector((LongColumnVector) vector);
+        } else if (vector instanceof DoubleColumnVector) {
+          return new OrcNoHiveDoubleVector((DoubleColumnVector) vector);
+        } else if (vector instanceof BytesColumnVector) {
+          return new OrcNoHiveBytesVector((BytesColumnVector) vector);
+        } else if (vector instanceof DecimalColumnVector) {
+          return new OrcNoHiveDecimalVector((DecimalColumnVector) vector);
+        } else if (vector instanceof TimestampColumnVector) {
+          return new OrcNoHiveTimestampVector((TimestampColumnVector) vector);
+        } else {
+          throw new UnsupportedOperationException(
+              "Unsupported vector: " + vector.getClass().getName());
+        }
+      };
+    }
+  }
+
+  private static class LongOrcColumnVector implements org.apache.flink.table.data.vector.LongColumnVector {
+    private long batchOffsetInFile;
+
+    LongOrcColumnVector(long batchOffsetInFile) {
+      this.batchOffsetInFile = batchOffsetInFile;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return false;
+    }
+
+    @Override
+    public long getLong(int i) {
+      return batchOffsetInFile + i;
+    }
+  }
+
+  private static class StructConverter implements Converter {
+    private final Types.StructType structType;
+    private final List<Converter> fieldConverters;
+    private final Map<Integer, ?> idToConstant;
+
+    private StructConverter(Types.StructType structType, List<Converter> fieldConverters,
+                            Map<Integer, ?> idToConstant) {
+      this.structType = structType;
+      this.fieldConverters = fieldConverters;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector vector, int batchSize,
+                                long batchOffsetInFile) {
+      StructColumnVector structVector = (StructColumnVector) vector;
+      List<Types.NestedField> fields = structType.fields();
+      ColumnVector[] fieldVectors = new ColumnVector[fields.size()];
+      for (int pos = 0; pos < fields.size(); pos++) {
+        Types.NestedField field = fields.get(pos);
+        if (idToConstant.containsKey(field.fieldId())) {
+          fieldVectors[pos] = getConstantVectors(field.type(), idToConstant.get(field.fieldId()));
+        } else if (field.equals(MetadataColumns.ROW_POSITION)) {
+          fieldVectors[pos] = new LongOrcColumnVector(batchOffsetInFile);
+        } else {
+          fieldVectors[pos] = fieldConverters.get(pos)
+              .convert(structVector.fields[pos], batchSize, batchOffsetInFile);
+        }
+      }
+
+      return new FlinkRowColumnVector(fieldVectors, structVector);
+    }
+
+    private ColumnVector getConstantVectors(Type type, Object constant) {
+      if (type.isPrimitiveType()) {

Review comment:
       We don't need this `type.isPrimitiveType()` any more because we've checked each data type in the following switch-case sentence.  We can throw an exception when we encounter primitive types that cannot be handled.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r631587964



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java
##########
@@ -49,19 +49,19 @@
 import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
 import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
 
-class FlinkOrcReaders {
+public class FlinkOrcReaders {
   private FlinkOrcReaders() {
   }
 
-  static OrcValueReader<StringData> strings() {
+  public static OrcValueReader<StringData> strings() {

Review comment:
       it is my mistake, I rolled back it




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r630684260



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        FlinkRowColumnVector cv = (FlinkRowColumnVector) converter.convert(
+            new StructColumnVector(batch.size, batch.cols), batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) {
+      return (vector, batchSize, batchOffsetInFile) -> {
+        if (vector instanceof LongColumnVector) {
+          return new OrcNoHiveLongVector((LongColumnVector) vector);
+        } else if (vector instanceof DoubleColumnVector) {
+          return new OrcNoHiveDoubleVector((DoubleColumnVector) vector);
+        } else if (vector instanceof BytesColumnVector) {
+          return new OrcNoHiveBytesVector((BytesColumnVector) vector);
+        } else if (vector instanceof DecimalColumnVector) {
+          return new OrcNoHiveDecimalVector((DecimalColumnVector) vector);
+        } else if (vector instanceof TimestampColumnVector) {
+          return new OrcNoHiveTimestampVector((TimestampColumnVector) vector);
+        } else {
+          throw new UnsupportedOperationException(
+              "Unsupported vector: " + vector.getClass().getName());
+        }
+      };
+    }
+  }
+
+  private static class LongOrcColumnVector implements org.apache.flink.table.data.vector.LongColumnVector {
+    private long batchOffsetInFile;
+
+    LongOrcColumnVector(long batchOffsetInFile) {
+      this.batchOffsetInFile = batchOffsetInFile;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return false;
+    }
+
+    @Override
+    public long getLong(int i) {
+      return batchOffsetInFile + i;
+    }
+  }
+
+  private static class StructConverter implements Converter {
+    private final Types.StructType structType;
+    private final List<Converter> fieldConverters;
+    private final Map<Integer, ?> idToConstant;
+
+    private StructConverter(Types.StructType structType, List<Converter> fieldConverters,
+                            Map<Integer, ?> idToConstant) {
+      this.structType = structType;
+      this.fieldConverters = fieldConverters;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector vector, int batchSize,
+                                long batchOffsetInFile) {
+      StructColumnVector structVector = (StructColumnVector) vector;
+      List<Types.NestedField> fields = structType.fields();
+      ColumnVector[] fieldVectors = new ColumnVector[fields.size()];
+      for (int pos = 0; pos < fields.size(); pos++) {
+        Types.NestedField field = fields.get(pos);
+        if (idToConstant.containsKey(field.fieldId())) {
+          fieldVectors[pos] = getConstantVectors(field.type(), idToConstant.get(field.fieldId()));
+        } else if (field.equals(MetadataColumns.ROW_POSITION)) {
+          fieldVectors[pos] = new LongOrcColumnVector(batchOffsetInFile);
+        } else {
+          fieldVectors[pos] = fieldConverters.get(pos)
+              .convert(structVector.fields[pos], batchSize, batchOffsetInFile);
+        }
+      }
+
+      return new FlinkRowColumnVector(fieldVectors, structVector);
+    }
+
+    private ColumnVector getConstantVectors(Type type, Object constant) {
+      if (type.isPrimitiveType()) {
+        Type.TypeID typeID = type.typeId();
+        switch (typeID) {
+          case INTEGER:
+          case DATE:
+          case TIME:
+            return ConstantColumnVectors.ints(constant);
+
+          case LONG:
+            return ConstantColumnVectors.longs(constant);
+
+          case BOOLEAN:
+            return ConstantColumnVectors.booleans(constant);
+
+          case DOUBLE:
+            return ConstantColumnVectors.doubles(constant);
+
+          case FLOAT:
+            return ConstantColumnVectors.floats(constant);
+
+          case DECIMAL:
+            return ConstantColumnVectors.decimals(constant);
+
+          case TIMESTAMP:
+            return ConstantColumnVectors.timestamps(constant);
+
+          case STRING:
+            return ConstantColumnVectors.bytes(constant);
+
+          default:
+            throw new UnsupportedOperationException("Unsupported data type for constant.");
+        }
+      } else {
+        throw new UnsupportedOperationException("ConstantColumnVector only supports primitives.");
+      }
+    }
+
+    private static class ArrayConverter implements Converter {
+      private final Converter elementConverter;
+
+      private ArrayConverter(Converter elementConverter) {
+        this.elementConverter = elementConverter;
+      }
+
+      @Override
+      public ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector vector, int batchSize,
+                                  long batchOffsetInFile) {
+        ListColumnVector listVector = (ListColumnVector) vector;
+        ColumnVector elementVector = elementConverter.convert(listVector.child, batchSize, batchOffsetInFile);
+
+        return new ArrayColumnVector() {
+          @Override
+          public ArrayData getArray(int rowId) {
+            int index = getRowIndex(rowId);
+            return new ColumnarArrayData(elementVector, (int) listVector.offsets[index],
+                (int) listVector.lengths[index]);
+          }
+
+          @Override
+          public boolean isNullAt(int rowId) {
+            return vector.isNull[getRowIndex(rowId)];
+          }
+
+          private int getRowIndex(int rowId) {
+            return vector.isRepeating ? 0 : rowId;
+          }
+        };
+      }
+    }
+  }
+
+  private static class FlinkRowColumnVector implements RowColumnVector {
+
+    private ColumnVector[] fieldVectors;
+    private StructColumnVector structVector;
+
+    FlinkRowColumnVector(ColumnVector[] fieldVectors,
+                         StructColumnVector structVector) {
+      this.fieldVectors = fieldVectors;
+      this.structVector = structVector;
+    }
+
+    @Override
+    public ColumnarRowData getRow(int i) {
+      VectorizedColumnBatch vectorizedColumnBatch = new VectorizedColumnBatch(fieldVectors);

Review comment:
       I don't think we need to create a separate `VectorizedColumnBatch` for every row, that's too wasting. How about moving this line into the `FlinkRowColumnVector` constructor ? 




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r636641845



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkTableOptions.java
##########
@@ -40,4 +40,10 @@ private FlinkTableOptions() {
           .intType()
           .defaultValue(100)
           .withDescription("Sets max infer parallelism for source operator.");
+
+  public static final ConfigOption<Boolean> ENABLE_VECTORIZED_READ =
+      ConfigOptions.key("enable.vectorized.read")

Review comment:
       it seems that Iceberg uses hyphen as config separator. maybe `enable-vectorized-read`?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r637715387



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/BatchRowDataIterator.java
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.iceberg.flink.source;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileContent;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.flink.data.RowDataUtil;
+import org.apache.iceberg.flink.data.vectorized.VectorizedFlinkOrcReaders;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PartitionUtil;
+import org.jetbrains.annotations.NotNull;
+
+class BatchRowDataIterator extends DataIterator<RowData> {
+
+  private final Schema tableSchema;
+  private final Schema projectedSchema;
+  private final String nameMapping;
+  private final boolean caseSensitive;
+  private final DataType[] dataTypes;
+
+  BatchRowDataIterator(CombinedScanTask task, FileIO io, EncryptionManager encryption, Schema tableSchema,
+                       Schema projectedSchema, String nameMapping, boolean caseSensitive, DataType[] dataTypes) {
+    super(task, io, encryption);
+    this.tableSchema = tableSchema;
+    this.projectedSchema = projectedSchema;
+    this.nameMapping = nameMapping;
+    this.caseSensitive = caseSensitive;
+    this.dataTypes = dataTypes;

Review comment:
       I think if we verify all nested types, including multi-level nested, it might be too complicated. I read the code of `VectorizedSparkOrcReaders`. It also does not do type verification, so I wonder if it is necessary for us to do type verification in flink?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r630677403



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        FlinkRowColumnVector cv = (FlinkRowColumnVector) converter.convert(
+            new StructColumnVector(batch.size, batch.cols), batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) {
+      return (vector, batchSize, batchOffsetInFile) -> {
+        if (vector instanceof LongColumnVector) {
+          return new OrcNoHiveLongVector((LongColumnVector) vector);
+        } else if (vector instanceof DoubleColumnVector) {
+          return new OrcNoHiveDoubleVector((DoubleColumnVector) vector);
+        } else if (vector instanceof BytesColumnVector) {
+          return new OrcNoHiveBytesVector((BytesColumnVector) vector);
+        } else if (vector instanceof DecimalColumnVector) {
+          return new OrcNoHiveDecimalVector((DecimalColumnVector) vector);
+        } else if (vector instanceof TimestampColumnVector) {
+          return new OrcNoHiveTimestampVector((TimestampColumnVector) vector);
+        } else {
+          throw new UnsupportedOperationException(
+              "Unsupported vector: " + vector.getClass().getName());
+        }
+      };
+    }
+  }
+
+  private static class LongOrcColumnVector implements org.apache.flink.table.data.vector.LongColumnVector {
+    private long batchOffsetInFile;
+
+    LongOrcColumnVector(long batchOffsetInFile) {
+      this.batchOffsetInFile = batchOffsetInFile;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return false;
+    }
+
+    @Override
+    public long getLong(int i) {
+      return batchOffsetInFile + i;
+    }
+  }
+
+  private static class StructConverter implements Converter {
+    private final Types.StructType structType;
+    private final List<Converter> fieldConverters;
+    private final Map<Integer, ?> idToConstant;
+
+    private StructConverter(Types.StructType structType, List<Converter> fieldConverters,
+                            Map<Integer, ?> idToConstant) {
+      this.structType = structType;
+      this.fieldConverters = fieldConverters;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector vector, int batchSize,
+                                long batchOffsetInFile) {
+      StructColumnVector structVector = (StructColumnVector) vector;
+      List<Types.NestedField> fields = structType.fields();
+      ColumnVector[] fieldVectors = new ColumnVector[fields.size()];
+      for (int pos = 0; pos < fields.size(); pos++) {
+        Types.NestedField field = fields.get(pos);
+        if (idToConstant.containsKey(field.fieldId())) {
+          fieldVectors[pos] = getConstantVectors(field.type(), idToConstant.get(field.fieldId()));
+        } else if (field.equals(MetadataColumns.ROW_POSITION)) {
+          fieldVectors[pos] = new LongOrcColumnVector(batchOffsetInFile);

Review comment:
       I think it's better to name it as `RowPositionColumnVector`  because it's not a normal `LongOrcColumnVector`, instead it is designed specially for reading the row offset for iceberg format v2.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r631587763



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/BaseDataIterator.java
##########
@@ -0,0 +1,32 @@
+/*
+ * 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.iceberg.flink.source;
+
+import org.apache.flink.table.data.RowData;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.FileIO;
+
+abstract class BaseDataIterator extends DataIterator<RowData> {

Review comment:
       At first I wanted to introduce a base iterator for `RowData`, indeed `DataIterator<RowData>` is enough, I rolled back the code




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r636681034



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/BatchRowDataIterator.java
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.iceberg.flink.source;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileContent;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.flink.data.RowDataUtil;
+import org.apache.iceberg.flink.data.vectorized.VectorizedFlinkOrcReaders;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PartitionUtil;
+import org.jetbrains.annotations.NotNull;
+
+class BatchRowDataIterator extends DataIterator<RowData> {
+
+  private final Schema tableSchema;
+  private final Schema projectedSchema;
+  private final String nameMapping;
+  private final boolean caseSensitive;
+  private final DataType[] dataTypes;
+
+  BatchRowDataIterator(CombinedScanTask task, FileIO io, EncryptionManager encryption, Schema tableSchema,
+                       Schema projectedSchema, String nameMapping, boolean caseSensitive, DataType[] dataTypes) {
+    super(task, io, encryption);
+    this.tableSchema = tableSchema;
+    this.projectedSchema = projectedSchema;
+    this.nameMapping = nameMapping;
+    this.caseSensitive = caseSensitive;
+    this.dataTypes = dataTypes;

Review comment:
       > 2\. Since the `DataType[]` is only extracted from table or projected schema in `FlinkSource`,  should such validation be done once in the  constructor of `FlinkInputFormat`?
   
   I am a little confused and don't know what verification you are referring to




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r630646966



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/ConstantColumnVectors.java
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import org.apache.flink.table.data.vector.BooleanColumnVector;
+import org.apache.flink.table.data.vector.BytesColumnVector;
+import org.apache.flink.table.data.vector.DecimalColumnVector;
+import org.apache.flink.table.data.vector.DoubleColumnVector;
+import org.apache.flink.table.data.vector.FloatColumnVector;
+import org.apache.flink.table.data.vector.IntColumnVector;
+import org.apache.flink.table.data.vector.LongColumnVector;
+import org.apache.flink.table.data.vector.TimestampColumnVector;
+
+class ConstantColumnVectors {
+  private ConstantColumnVectors() {
+  }
+
+  static IntOrcColumnVector ints(Object constant) {
+    return new IntOrcColumnVector(constant);
+  }
+
+  static LongOrcColumnVector longs(Object constant) {
+    return new LongOrcColumnVector(constant);
+  }
+
+  static BooleanOrcColumnVector booleans(Object constant) {
+    return new BooleanOrcColumnVector(constant);
+  }
+
+  static DoubleOrcColumnVector doubles(Object constant) {
+    return new DoubleOrcColumnVector(constant);
+  }
+
+  static FloatOrcColumnVector floats(Object constant) {
+    return new FloatOrcColumnVector(constant);
+  }
+
+  static DecimalOrcColumnVector decimals(Object constant) {
+    return new DecimalOrcColumnVector(constant);
+  }
+
+  static TimestampOrcColumnVector timestamps(Object constant) {
+    return new TimestampOrcColumnVector(constant);
+  }
+
+  static BytesOrcColumnVector bytes(Object constant) {
+    return new BytesOrcColumnVector(constant);
+  }
+
+  private static class IntOrcColumnVector implements IntColumnVector {
+
+    private Object constant;
+
+    private IntOrcColumnVector(Object constant) {
+      this.constant = constant;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return false;

Review comment:
       If the `constant` instance is null,  then this method should return `true`, right ?   So the correct way is: `return constant == null` here ? 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/ConstantColumnVectors.java
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import org.apache.flink.table.data.vector.BooleanColumnVector;
+import org.apache.flink.table.data.vector.BytesColumnVector;
+import org.apache.flink.table.data.vector.DecimalColumnVector;
+import org.apache.flink.table.data.vector.DoubleColumnVector;
+import org.apache.flink.table.data.vector.FloatColumnVector;
+import org.apache.flink.table.data.vector.IntColumnVector;
+import org.apache.flink.table.data.vector.LongColumnVector;
+import org.apache.flink.table.data.vector.TimestampColumnVector;
+
+class ConstantColumnVectors {
+  private ConstantColumnVectors() {
+  }
+
+  static IntOrcColumnVector ints(Object constant) {

Review comment:
       The `IntOrcColumnVector` could also be used for parquet vectorized reader , right ?  So maybe it's better to name it as `ConstantIntColumnVector`.   Besides,  we don't need to expose the `IntOrcColumnVector` to the package classes,  using the `ColumnVector` interface  is enough I think.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        FlinkRowColumnVector cv = (FlinkRowColumnVector) converter.convert(
+            new StructColumnVector(batch.size, batch.cols), batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) {
+      return (vector, batchSize, batchOffsetInFile) -> {
+        if (vector instanceof LongColumnVector) {
+          return new OrcNoHiveLongVector((LongColumnVector) vector);
+        } else if (vector instanceof DoubleColumnVector) {
+          return new OrcNoHiveDoubleVector((DoubleColumnVector) vector);
+        } else if (vector instanceof BytesColumnVector) {
+          return new OrcNoHiveBytesVector((BytesColumnVector) vector);
+        } else if (vector instanceof DecimalColumnVector) {
+          return new OrcNoHiveDecimalVector((DecimalColumnVector) vector);
+        } else if (vector instanceof TimestampColumnVector) {
+          return new OrcNoHiveTimestampVector((TimestampColumnVector) vector);
+        } else {
+          throw new UnsupportedOperationException(
+              "Unsupported vector: " + vector.getClass().getName());
+        }
+      };
+    }
+  }
+
+  private static class LongOrcColumnVector implements org.apache.flink.table.data.vector.LongColumnVector {
+    private long batchOffsetInFile;
+
+    LongOrcColumnVector(long batchOffsetInFile) {
+      this.batchOffsetInFile = batchOffsetInFile;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return false;
+    }
+
+    @Override
+    public long getLong(int i) {
+      return batchOffsetInFile + i;
+    }
+  }
+
+  private static class StructConverter implements Converter {
+    private final Types.StructType structType;
+    private final List<Converter> fieldConverters;
+    private final Map<Integer, ?> idToConstant;
+
+    private StructConverter(Types.StructType structType, List<Converter> fieldConverters,
+                            Map<Integer, ?> idToConstant) {
+      this.structType = structType;
+      this.fieldConverters = fieldConverters;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector vector, int batchSize,
+                                long batchOffsetInFile) {
+      StructColumnVector structVector = (StructColumnVector) vector;
+      List<Types.NestedField> fields = structType.fields();
+      ColumnVector[] fieldVectors = new ColumnVector[fields.size()];
+      for (int pos = 0; pos < fields.size(); pos++) {
+        Types.NestedField field = fields.get(pos);
+        if (idToConstant.containsKey(field.fieldId())) {
+          fieldVectors[pos] = getConstantVectors(field.type(), idToConstant.get(field.fieldId()));
+        } else if (field.equals(MetadataColumns.ROW_POSITION)) {
+          fieldVectors[pos] = new LongOrcColumnVector(batchOffsetInFile);
+        } else {
+          fieldVectors[pos] = fieldConverters.get(pos)
+              .convert(structVector.fields[pos], batchSize, batchOffsetInFile);
+        }
+      }
+
+      return new FlinkRowColumnVector(fieldVectors, structVector);
+    }
+
+    private ColumnVector getConstantVectors(Type type, Object constant) {
+      if (type.isPrimitiveType()) {
+        Type.TypeID typeID = type.typeId();
+        switch (typeID) {
+          case INTEGER:
+          case DATE:
+          case TIME:
+            return ConstantColumnVectors.ints(constant);
+
+          case LONG:
+            return ConstantColumnVectors.longs(constant);
+
+          case BOOLEAN:
+            return ConstantColumnVectors.booleans(constant);
+
+          case DOUBLE:
+            return ConstantColumnVectors.doubles(constant);
+
+          case FLOAT:
+            return ConstantColumnVectors.floats(constant);
+
+          case DECIMAL:
+            return ConstantColumnVectors.decimals(constant);
+
+          case TIMESTAMP:
+            return ConstantColumnVectors.timestamps(constant);
+
+          case STRING:
+            return ConstantColumnVectors.bytes(constant);
+
+          default:
+            throw new UnsupportedOperationException("Unsupported data type for constant.");

Review comment:
       For the iceberg data type `UUID`, `FIXED`, `BINARY` cases,  we should return `ConstantColumnVectors.bytes(constant)`.    The correct mapping relationships (iceberg data types <=> flink data types <=> orc data types)  are [here](https://github.com/apache/iceberg/blob/90225d6c9413016d611e2ce5eff37db1bc1b4fc5/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java#L87).

##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        FlinkRowColumnVector cv = (FlinkRowColumnVector) converter.convert(
+            new StructColumnVector(batch.size, batch.cols), batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) {
+      return (vector, batchSize, batchOffsetInFile) -> {
+        if (vector instanceof LongColumnVector) {
+          return new OrcNoHiveLongVector((LongColumnVector) vector);
+        } else if (vector instanceof DoubleColumnVector) {
+          return new OrcNoHiveDoubleVector((DoubleColumnVector) vector);
+        } else if (vector instanceof BytesColumnVector) {
+          return new OrcNoHiveBytesVector((BytesColumnVector) vector);
+        } else if (vector instanceof DecimalColumnVector) {
+          return new OrcNoHiveDecimalVector((DecimalColumnVector) vector);
+        } else if (vector instanceof TimestampColumnVector) {
+          return new OrcNoHiveTimestampVector((TimestampColumnVector) vector);
+        } else {
+          throw new UnsupportedOperationException(
+              "Unsupported vector: " + vector.getClass().getName());
+        }
+      };
+    }
+  }
+
+  private static class LongOrcColumnVector implements org.apache.flink.table.data.vector.LongColumnVector {
+    private long batchOffsetInFile;
+
+    LongOrcColumnVector(long batchOffsetInFile) {
+      this.batchOffsetInFile = batchOffsetInFile;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return false;
+    }
+
+    @Override
+    public long getLong(int i) {
+      return batchOffsetInFile + i;
+    }
+  }
+
+  private static class StructConverter implements Converter {
+    private final Types.StructType structType;
+    private final List<Converter> fieldConverters;
+    private final Map<Integer, ?> idToConstant;
+
+    private StructConverter(Types.StructType structType, List<Converter> fieldConverters,
+                            Map<Integer, ?> idToConstant) {
+      this.structType = structType;
+      this.fieldConverters = fieldConverters;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector vector, int batchSize,
+                                long batchOffsetInFile) {
+      StructColumnVector structVector = (StructColumnVector) vector;
+      List<Types.NestedField> fields = structType.fields();
+      ColumnVector[] fieldVectors = new ColumnVector[fields.size()];
+      for (int pos = 0; pos < fields.size(); pos++) {
+        Types.NestedField field = fields.get(pos);
+        if (idToConstant.containsKey(field.fieldId())) {
+          fieldVectors[pos] = getConstantVectors(field.type(), idToConstant.get(field.fieldId()));
+        } else if (field.equals(MetadataColumns.ROW_POSITION)) {
+          fieldVectors[pos] = new LongOrcColumnVector(batchOffsetInFile);
+        } else {
+          fieldVectors[pos] = fieldConverters.get(pos)
+              .convert(structVector.fields[pos], batchSize, batchOffsetInFile);
+        }
+      }
+
+      return new FlinkRowColumnVector(fieldVectors, structVector);
+    }
+
+    private ColumnVector getConstantVectors(Type type, Object constant) {
+      if (type.isPrimitiveType()) {
+        Type.TypeID typeID = type.typeId();
+        switch (typeID) {
+          case INTEGER:
+          case DATE:
+          case TIME:
+            return ConstantColumnVectors.ints(constant);
+
+          case LONG:
+            return ConstantColumnVectors.longs(constant);
+
+          case BOOLEAN:
+            return ConstantColumnVectors.booleans(constant);
+
+          case DOUBLE:
+            return ConstantColumnVectors.doubles(constant);
+
+          case FLOAT:
+            return ConstantColumnVectors.floats(constant);
+
+          case DECIMAL:
+            return ConstantColumnVectors.decimals(constant);
+
+          case TIMESTAMP:
+            return ConstantColumnVectors.timestamps(constant);
+
+          case STRING:
+            return ConstantColumnVectors.bytes(constant);

Review comment:
       In this case, we should return a `ColumnVector` with the `StringData` data type,  because we will need to convert the ORC backend `bytes` to flink's `StringData` so that apache flink could recognize it as flink's `String`.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r629848242



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        StructConverter.FlinkRowColumnVector cv =
+            (StructConverter.FlinkRowColumnVector) converter.convert(new StructColumnVector(batch.size, batch.cols),
+                batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException();

Review comment:
       add a proper error msg like `Map vector type not supported yet`




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r636677020



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/BatchRowDataIterator.java
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.iceberg.flink.source;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileContent;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.flink.data.RowDataUtil;
+import org.apache.iceberg.flink.data.vectorized.VectorizedFlinkOrcReaders;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PartitionUtil;
+import org.jetbrains.annotations.NotNull;
+
+class BatchRowDataIterator extends DataIterator<RowData> {
+
+  private final Schema tableSchema;
+  private final Schema projectedSchema;
+  private final String nameMapping;
+  private final boolean caseSensitive;
+  private final DataType[] dataTypes;
+
+  BatchRowDataIterator(CombinedScanTask task, FileIO io, EncryptionManager encryption, Schema tableSchema,
+                       Schema projectedSchema, String nameMapping, boolean caseSensitive, DataType[] dataTypes) {
+    super(task, io, encryption);
+    this.tableSchema = tableSchema;
+    this.projectedSchema = projectedSchema;
+    this.nameMapping = nameMapping;
+    this.caseSensitive = caseSensitive;
+    this.dataTypes = dataTypes;

Review comment:
       > 1. I believe the array is only for top level columns. what if a nested field is unsupported type
   
   Some types are supported, some are not,I add the test case : 
   https://github.com/zhangjun0x01/iceberg/blob/flink-vectorized-read/flink/src/test/java/org/apache/iceberg/flink/data/vectorized/TestVectorizedReads.java#L224




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r636084929



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        FlinkRowColumnVector cv = (FlinkRowColumnVector) converter.convert(
+            new StructColumnVector(batch.size, batch.cols), batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException("Unsupported vectorized read for map type.");
+    }
+
+    @Override
+    public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) {
+      return (vector, batchSize, batchOffsetInFile) -> {
+        if (vector instanceof LongColumnVector) {
+          return new OrcNoHiveLongVector((LongColumnVector) vector);
+        } else if (vector instanceof DoubleColumnVector) {
+          return new OrcNoHiveDoubleVector((DoubleColumnVector) vector);
+        } else if (vector instanceof BytesColumnVector) {
+          return new OrcNoHiveBytesVector((BytesColumnVector) vector);
+        } else if (vector instanceof DecimalColumnVector) {
+          return new OrcNoHiveDecimalVector((DecimalColumnVector) vector);
+        } else if (vector instanceof TimestampColumnVector) {
+          return new OrcNoHiveTimestampVector((TimestampColumnVector) vector);
+        } else {
+          throw new UnsupportedOperationException(
+              "Unsupported vector: " + vector.getClass().getName());
+        }
+      };
+    }
+  }
+
+  private static class RowPositionColumnVector implements org.apache.flink.table.data.vector.LongColumnVector {
+    private final long batchOffsetInFile;
+
+    RowPositionColumnVector(long batchOffsetInFile) {
+      this.batchOffsetInFile = batchOffsetInFile;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return false;
+    }
+
+    @Override
+    public long getLong(int i) {
+      return batchOffsetInFile + i;
+    }
+  }
+
+  private static class StructConverter implements Converter {
+    private final Types.StructType structType;
+    private final List<Converter> fieldConverters;
+    private final Map<Integer, ?> idToConstant;
+
+    private StructConverter(Types.StructType structType, List<Converter> fieldConverters,
+                            Map<Integer, ?> idToConstant) {
+      this.structType = structType;
+      this.fieldConverters = fieldConverters;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector vector, int batchSize,
+                                long batchOffsetInFile) {
+      StructColumnVector structVector = (StructColumnVector) vector;
+      List<Types.NestedField> fields = structType.fields();
+      ColumnVector[] fieldVectors = new ColumnVector[fields.size()];
+      for (int pos = 0; pos < fields.size(); pos++) {
+        Types.NestedField field = fields.get(pos);
+        if (idToConstant.containsKey(field.fieldId())) {
+          fieldVectors[pos] = toConstantColumnVector(field.type(), idToConstant.get(field.fieldId()));
+        } else if (field.equals(MetadataColumns.ROW_POSITION)) {
+          fieldVectors[pos] = new RowPositionColumnVector(batchOffsetInFile);
+        } else {
+          fieldVectors[pos] = fieldConverters.get(pos)
+              .convert(structVector.fields[pos], batchSize, batchOffsetInFile);
+        }
+      }
+
+      return new FlinkRowColumnVector(fieldVectors, structVector);
+    }
+
+    private ColumnVector toConstantColumnVector(Type type, Object constant) {
+      Type.TypeID typeID = type.typeId();
+      switch (typeID) {
+        case INTEGER:
+        case DATE:
+        case TIME:
+          return ConstantColumnVectors.ints(constant);
+
+        case LONG:
+          return ConstantColumnVectors.longs(constant);
+
+        case BOOLEAN:
+          return ConstantColumnVectors.booleans(constant);
+
+        case DOUBLE:
+          return ConstantColumnVectors.doubles(constant);
+
+        case FLOAT:
+          return ConstantColumnVectors.floats(constant);
+
+        case DECIMAL:
+          return ConstantColumnVectors.decimals(constant);
+
+        case TIMESTAMP:
+          return ConstantColumnVectors.timestamps(constant);
+
+        case FIXED:
+        case UUID:
+        case BINARY:

Review comment:
       > I still don't think we could use the `ConstantBytesColumnVector` to read the `BINARY` & `FIXED` data type because its constant object is a [StringData](https://github.com/apache/iceberg/blob/90225d6c9413016d611e2ce5eff37db1bc1b4fc5/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java#L66), we will definitely encounter a class cast exception when cast the `byte[]` to `BinaryStringData` , do we have an unit test to cover this ?
   
   from the [code](https://github.com/apache/iceberg/blob/90225d6c9413016d611e2ce5eff37db1bc1b4fc5/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java#L66) ,for `FIXED ` data type,it will be convert to byte[],so I added the judgment that constant is byte[] instead of throw a cast exception. and add the UT for byte[] data type




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r636084929



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        FlinkRowColumnVector cv = (FlinkRowColumnVector) converter.convert(
+            new StructColumnVector(batch.size, batch.cols), batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException("Unsupported vectorized read for map type.");
+    }
+
+    @Override
+    public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) {
+      return (vector, batchSize, batchOffsetInFile) -> {
+        if (vector instanceof LongColumnVector) {
+          return new OrcNoHiveLongVector((LongColumnVector) vector);
+        } else if (vector instanceof DoubleColumnVector) {
+          return new OrcNoHiveDoubleVector((DoubleColumnVector) vector);
+        } else if (vector instanceof BytesColumnVector) {
+          return new OrcNoHiveBytesVector((BytesColumnVector) vector);
+        } else if (vector instanceof DecimalColumnVector) {
+          return new OrcNoHiveDecimalVector((DecimalColumnVector) vector);
+        } else if (vector instanceof TimestampColumnVector) {
+          return new OrcNoHiveTimestampVector((TimestampColumnVector) vector);
+        } else {
+          throw new UnsupportedOperationException(
+              "Unsupported vector: " + vector.getClass().getName());
+        }
+      };
+    }
+  }
+
+  private static class RowPositionColumnVector implements org.apache.flink.table.data.vector.LongColumnVector {
+    private final long batchOffsetInFile;
+
+    RowPositionColumnVector(long batchOffsetInFile) {
+      this.batchOffsetInFile = batchOffsetInFile;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return false;
+    }
+
+    @Override
+    public long getLong(int i) {
+      return batchOffsetInFile + i;
+    }
+  }
+
+  private static class StructConverter implements Converter {
+    private final Types.StructType structType;
+    private final List<Converter> fieldConverters;
+    private final Map<Integer, ?> idToConstant;
+
+    private StructConverter(Types.StructType structType, List<Converter> fieldConverters,
+                            Map<Integer, ?> idToConstant) {
+      this.structType = structType;
+      this.fieldConverters = fieldConverters;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector vector, int batchSize,
+                                long batchOffsetInFile) {
+      StructColumnVector structVector = (StructColumnVector) vector;
+      List<Types.NestedField> fields = structType.fields();
+      ColumnVector[] fieldVectors = new ColumnVector[fields.size()];
+      for (int pos = 0; pos < fields.size(); pos++) {
+        Types.NestedField field = fields.get(pos);
+        if (idToConstant.containsKey(field.fieldId())) {
+          fieldVectors[pos] = toConstantColumnVector(field.type(), idToConstant.get(field.fieldId()));
+        } else if (field.equals(MetadataColumns.ROW_POSITION)) {
+          fieldVectors[pos] = new RowPositionColumnVector(batchOffsetInFile);
+        } else {
+          fieldVectors[pos] = fieldConverters.get(pos)
+              .convert(structVector.fields[pos], batchSize, batchOffsetInFile);
+        }
+      }
+
+      return new FlinkRowColumnVector(fieldVectors, structVector);
+    }
+
+    private ColumnVector toConstantColumnVector(Type type, Object constant) {
+      Type.TypeID typeID = type.typeId();
+      switch (typeID) {
+        case INTEGER:
+        case DATE:
+        case TIME:
+          return ConstantColumnVectors.ints(constant);
+
+        case LONG:
+          return ConstantColumnVectors.longs(constant);
+
+        case BOOLEAN:
+          return ConstantColumnVectors.booleans(constant);
+
+        case DOUBLE:
+          return ConstantColumnVectors.doubles(constant);
+
+        case FLOAT:
+          return ConstantColumnVectors.floats(constant);
+
+        case DECIMAL:
+          return ConstantColumnVectors.decimals(constant);
+
+        case TIMESTAMP:
+          return ConstantColumnVectors.timestamps(constant);
+
+        case FIXED:
+        case UUID:
+        case BINARY:

Review comment:
       from the [code](https://github.com/apache/iceberg/blob/90225d6c9413016d611e2ce5eff37db1bc1b4fc5/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java#L66) 
   
   > I still don't think we could use the `ConstantBytesColumnVector` to read the `BINARY` & `FIXED` data type because its constant object is a [StringData](https://github.com/apache/iceberg/blob/90225d6c9413016d611e2ce5eff37db1bc1b4fc5/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java#L66), we will definitely encounter a class cast exception when cast the `byte[]` to `BinaryStringData` , do we have an unit test to cover this ?
   
   from the [code](https://github.com/apache/iceberg/blob/90225d6c9413016d611e2ce5eff37db1bc1b4fc5/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java#L66) ,for `FIXED ` data type,it will be convert to byte[],so I I added the judgment that constant is byte[] instead of throw a cast exception.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r630684651



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        FlinkRowColumnVector cv = (FlinkRowColumnVector) converter.convert(
+            new StructColumnVector(batch.size, batch.cols), batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) {
+      return (vector, batchSize, batchOffsetInFile) -> {
+        if (vector instanceof LongColumnVector) {
+          return new OrcNoHiveLongVector((LongColumnVector) vector);
+        } else if (vector instanceof DoubleColumnVector) {
+          return new OrcNoHiveDoubleVector((DoubleColumnVector) vector);
+        } else if (vector instanceof BytesColumnVector) {
+          return new OrcNoHiveBytesVector((BytesColumnVector) vector);
+        } else if (vector instanceof DecimalColumnVector) {
+          return new OrcNoHiveDecimalVector((DecimalColumnVector) vector);
+        } else if (vector instanceof TimestampColumnVector) {
+          return new OrcNoHiveTimestampVector((TimestampColumnVector) vector);
+        } else {
+          throw new UnsupportedOperationException(
+              "Unsupported vector: " + vector.getClass().getName());
+        }
+      };
+    }
+  }
+
+  private static class LongOrcColumnVector implements org.apache.flink.table.data.vector.LongColumnVector {
+    private long batchOffsetInFile;
+
+    LongOrcColumnVector(long batchOffsetInFile) {
+      this.batchOffsetInFile = batchOffsetInFile;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return false;
+    }
+
+    @Override
+    public long getLong(int i) {
+      return batchOffsetInFile + i;
+    }
+  }
+
+  private static class StructConverter implements Converter {
+    private final Types.StructType structType;
+    private final List<Converter> fieldConverters;
+    private final Map<Integer, ?> idToConstant;
+
+    private StructConverter(Types.StructType structType, List<Converter> fieldConverters,
+                            Map<Integer, ?> idToConstant) {
+      this.structType = structType;
+      this.fieldConverters = fieldConverters;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector vector, int batchSize,
+                                long batchOffsetInFile) {
+      StructColumnVector structVector = (StructColumnVector) vector;
+      List<Types.NestedField> fields = structType.fields();
+      ColumnVector[] fieldVectors = new ColumnVector[fields.size()];
+      for (int pos = 0; pos < fields.size(); pos++) {
+        Types.NestedField field = fields.get(pos);
+        if (idToConstant.containsKey(field.fieldId())) {
+          fieldVectors[pos] = getConstantVectors(field.type(), idToConstant.get(field.fieldId()));
+        } else if (field.equals(MetadataColumns.ROW_POSITION)) {
+          fieldVectors[pos] = new LongOrcColumnVector(batchOffsetInFile);
+        } else {
+          fieldVectors[pos] = fieldConverters.get(pos)
+              .convert(structVector.fields[pos], batchSize, batchOffsetInFile);
+        }
+      }
+
+      return new FlinkRowColumnVector(fieldVectors, structVector);
+    }
+
+    private ColumnVector getConstantVectors(Type type, Object constant) {
+      if (type.isPrimitiveType()) {
+        Type.TypeID typeID = type.typeId();
+        switch (typeID) {
+          case INTEGER:
+          case DATE:
+          case TIME:
+            return ConstantColumnVectors.ints(constant);
+
+          case LONG:
+            return ConstantColumnVectors.longs(constant);
+
+          case BOOLEAN:
+            return ConstantColumnVectors.booleans(constant);
+
+          case DOUBLE:
+            return ConstantColumnVectors.doubles(constant);
+
+          case FLOAT:
+            return ConstantColumnVectors.floats(constant);
+
+          case DECIMAL:
+            return ConstantColumnVectors.decimals(constant);
+
+          case TIMESTAMP:
+            return ConstantColumnVectors.timestamps(constant);
+
+          case STRING:
+            return ConstantColumnVectors.bytes(constant);
+
+          default:
+            throw new UnsupportedOperationException("Unsupported data type for constant.");
+        }
+      } else {
+        throw new UnsupportedOperationException("ConstantColumnVector only supports primitives.");
+      }
+    }
+
+    private static class ArrayConverter implements Converter {
+      private final Converter elementConverter;
+
+      private ArrayConverter(Converter elementConverter) {
+        this.elementConverter = elementConverter;
+      }
+
+      @Override
+      public ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector vector, int batchSize,
+                                  long batchOffsetInFile) {
+        ListColumnVector listVector = (ListColumnVector) vector;
+        ColumnVector elementVector = elementConverter.convert(listVector.child, batchSize, batchOffsetInFile);
+
+        return new ArrayColumnVector() {
+          @Override
+          public ArrayData getArray(int rowId) {
+            int index = getRowIndex(rowId);
+            return new ColumnarArrayData(elementVector, (int) listVector.offsets[index],
+                (int) listVector.lengths[index]);
+          }
+
+          @Override
+          public boolean isNullAt(int rowId) {
+            return vector.isNull[getRowIndex(rowId)];
+          }
+
+          private int getRowIndex(int rowId) {
+            return vector.isRepeating ? 0 : rowId;
+          }
+        };
+      }
+    }
+  }
+
+  private static class FlinkRowColumnVector implements RowColumnVector {
+
+    private ColumnVector[] fieldVectors;
+    private StructColumnVector structVector;
+
+    FlinkRowColumnVector(ColumnVector[] fieldVectors,
+                         StructColumnVector structVector) {
+      this.fieldVectors = fieldVectors;
+      this.structVector = structVector;
+    }
+
+    @Override
+    public ColumnarRowData getRow(int i) {
+      VectorizedColumnBatch vectorizedColumnBatch = new VectorizedColumnBatch(fieldVectors);
+      return new ColumnarRowData(vectorizedColumnBatch, i);
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return structVector.isNull[i];
+    }
+
+    public ColumnVector[] getFieldVectors() {

Review comment:
       Nit:  this method don't have to be public.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r631505534



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/ConstantColumnVectors.java
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import org.apache.flink.table.data.vector.BooleanColumnVector;
+import org.apache.flink.table.data.vector.BytesColumnVector;
+import org.apache.flink.table.data.vector.DecimalColumnVector;
+import org.apache.flink.table.data.vector.DoubleColumnVector;
+import org.apache.flink.table.data.vector.FloatColumnVector;
+import org.apache.flink.table.data.vector.IntColumnVector;
+import org.apache.flink.table.data.vector.LongColumnVector;
+import org.apache.flink.table.data.vector.TimestampColumnVector;
+
+class ConstantColumnVectors {
+  private ConstantColumnVectors() {
+  }
+
+  static IntOrcColumnVector ints(Object constant) {
+    return new IntOrcColumnVector(constant);
+  }
+
+  static LongOrcColumnVector longs(Object constant) {
+    return new LongOrcColumnVector(constant);
+  }
+
+  static BooleanOrcColumnVector booleans(Object constant) {
+    return new BooleanOrcColumnVector(constant);
+  }
+
+  static DoubleOrcColumnVector doubles(Object constant) {
+    return new DoubleOrcColumnVector(constant);
+  }
+
+  static FloatOrcColumnVector floats(Object constant) {
+    return new FloatOrcColumnVector(constant);
+  }
+
+  static DecimalOrcColumnVector decimals(Object constant) {
+    return new DecimalOrcColumnVector(constant);
+  }
+
+  static TimestampOrcColumnVector timestamps(Object constant) {
+    return new TimestampOrcColumnVector(constant);
+  }
+
+  static BytesOrcColumnVector bytes(Object constant) {
+    return new BytesOrcColumnVector(constant);
+  }
+
+  private static class IntOrcColumnVector implements IntColumnVector {
+
+    private Object constant;
+
+    private IntOrcColumnVector(Object constant) {
+      this.constant = constant;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return false;

Review comment:
       It's possible that the [constant value to be null ](https://github.com/apache/iceberg/blob/90225d6c9413016d611e2ce5eff37db1bc1b4fc5/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java#L47).   Consider that there's an optional field in the partition fields, then if its value is null,  then the constantMap will be null.   The [PartitionUtil#constantsMap](https://github.com/apache/iceberg/blob/90225d6c9413016d611e2ce5eff37db1bc1b4fc5/core/src/main/java/org/apache/iceberg/util/PartitionUtil.java#L47) also clearly stated that there will be null values. 




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r629106172



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java
##########
@@ -91,11 +102,77 @@ public void configure(Configuration parameters) {
 
   @Override
   public void open(FlinkInputSplit split) {
+    boolean enableVectorizedRead = readableConfig.get(FlinkTableOptions.ENABLE_VECTORIZED_READ);
+
+    if (enableVectorizedRead) {
+      if (useOrcVectorizedRead()) {

Review comment:
       > 3\. All the columns to read should all be primitives,  that means all the byte width should be the same size
   
   We don't support `Map`, `Row`, `Array` data type?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r630675446



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        FlinkRowColumnVector cv = (FlinkRowColumnVector) converter.convert(
+            new StructColumnVector(batch.size, batch.cols), batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) {
+      return (vector, batchSize, batchOffsetInFile) -> {
+        if (vector instanceof LongColumnVector) {
+          return new OrcNoHiveLongVector((LongColumnVector) vector);
+        } else if (vector instanceof DoubleColumnVector) {
+          return new OrcNoHiveDoubleVector((DoubleColumnVector) vector);
+        } else if (vector instanceof BytesColumnVector) {
+          return new OrcNoHiveBytesVector((BytesColumnVector) vector);
+        } else if (vector instanceof DecimalColumnVector) {
+          return new OrcNoHiveDecimalVector((DecimalColumnVector) vector);
+        } else if (vector instanceof TimestampColumnVector) {
+          return new OrcNoHiveTimestampVector((TimestampColumnVector) vector);
+        } else {
+          throw new UnsupportedOperationException(
+              "Unsupported vector: " + vector.getClass().getName());
+        }
+      };
+    }
+  }
+
+  private static class LongOrcColumnVector implements org.apache.flink.table.data.vector.LongColumnVector {

Review comment:
       We cannot just import the `org.apache.flink.table.data.vector.LongColumnVector` because it has been conflicted with the imported one `org.apache.orc.storage.ql.exec.vector.LongColumnVector`.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r636084929



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        FlinkRowColumnVector cv = (FlinkRowColumnVector) converter.convert(
+            new StructColumnVector(batch.size, batch.cols), batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException("Unsupported vectorized read for map type.");
+    }
+
+    @Override
+    public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) {
+      return (vector, batchSize, batchOffsetInFile) -> {
+        if (vector instanceof LongColumnVector) {
+          return new OrcNoHiveLongVector((LongColumnVector) vector);
+        } else if (vector instanceof DoubleColumnVector) {
+          return new OrcNoHiveDoubleVector((DoubleColumnVector) vector);
+        } else if (vector instanceof BytesColumnVector) {
+          return new OrcNoHiveBytesVector((BytesColumnVector) vector);
+        } else if (vector instanceof DecimalColumnVector) {
+          return new OrcNoHiveDecimalVector((DecimalColumnVector) vector);
+        } else if (vector instanceof TimestampColumnVector) {
+          return new OrcNoHiveTimestampVector((TimestampColumnVector) vector);
+        } else {
+          throw new UnsupportedOperationException(
+              "Unsupported vector: " + vector.getClass().getName());
+        }
+      };
+    }
+  }
+
+  private static class RowPositionColumnVector implements org.apache.flink.table.data.vector.LongColumnVector {
+    private final long batchOffsetInFile;
+
+    RowPositionColumnVector(long batchOffsetInFile) {
+      this.batchOffsetInFile = batchOffsetInFile;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return false;
+    }
+
+    @Override
+    public long getLong(int i) {
+      return batchOffsetInFile + i;
+    }
+  }
+
+  private static class StructConverter implements Converter {
+    private final Types.StructType structType;
+    private final List<Converter> fieldConverters;
+    private final Map<Integer, ?> idToConstant;
+
+    private StructConverter(Types.StructType structType, List<Converter> fieldConverters,
+                            Map<Integer, ?> idToConstant) {
+      this.structType = structType;
+      this.fieldConverters = fieldConverters;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector vector, int batchSize,
+                                long batchOffsetInFile) {
+      StructColumnVector structVector = (StructColumnVector) vector;
+      List<Types.NestedField> fields = structType.fields();
+      ColumnVector[] fieldVectors = new ColumnVector[fields.size()];
+      for (int pos = 0; pos < fields.size(); pos++) {
+        Types.NestedField field = fields.get(pos);
+        if (idToConstant.containsKey(field.fieldId())) {
+          fieldVectors[pos] = toConstantColumnVector(field.type(), idToConstant.get(field.fieldId()));
+        } else if (field.equals(MetadataColumns.ROW_POSITION)) {
+          fieldVectors[pos] = new RowPositionColumnVector(batchOffsetInFile);
+        } else {
+          fieldVectors[pos] = fieldConverters.get(pos)
+              .convert(structVector.fields[pos], batchSize, batchOffsetInFile);
+        }
+      }
+
+      return new FlinkRowColumnVector(fieldVectors, structVector);
+    }
+
+    private ColumnVector toConstantColumnVector(Type type, Object constant) {
+      Type.TypeID typeID = type.typeId();
+      switch (typeID) {
+        case INTEGER:
+        case DATE:
+        case TIME:
+          return ConstantColumnVectors.ints(constant);
+
+        case LONG:
+          return ConstantColumnVectors.longs(constant);
+
+        case BOOLEAN:
+          return ConstantColumnVectors.booleans(constant);
+
+        case DOUBLE:
+          return ConstantColumnVectors.doubles(constant);
+
+        case FLOAT:
+          return ConstantColumnVectors.floats(constant);
+
+        case DECIMAL:
+          return ConstantColumnVectors.decimals(constant);
+
+        case TIMESTAMP:
+          return ConstantColumnVectors.timestamps(constant);
+
+        case FIXED:
+        case UUID:
+        case BINARY:

Review comment:
       
   > I still don't think we could use the `ConstantBytesColumnVector` to read the `BINARY` & `FIXED` data type because its constant object is a [StringData](https://github.com/apache/iceberg/blob/90225d6c9413016d611e2ce5eff37db1bc1b4fc5/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java#L66), we will definitely encounter a class cast exception when cast the `byte[]` to `BinaryStringData` , do we have an unit test to cover this ?
   
   from the [code](https://github.com/apache/iceberg/blob/90225d6c9413016d611e2ce5eff37db1bc1b4fc5/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java#L66) ,for `FIXED ` data type,it will be convert to byte[],so I I added the judgment that constant is byte[] instead of throw a cast exception.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r636089466



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        FlinkRowColumnVector cv = (FlinkRowColumnVector) converter.convert(
+            new StructColumnVector(batch.size, batch.cols), batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException("Unsupported vectorized read for map type.");
+    }
+
+    @Override
+    public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) {
+      return (vector, batchSize, batchOffsetInFile) -> {
+        if (vector instanceof LongColumnVector) {
+          return new OrcNoHiveLongVector((LongColumnVector) vector);
+        } else if (vector instanceof DoubleColumnVector) {

Review comment:
       yes,the `OrcNoHiveLongVector` can handle long , int, short etc.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r631507927



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        FlinkRowColumnVector cv = (FlinkRowColumnVector) converter.convert(
+            new StructColumnVector(batch.size, batch.cols), batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) {
+      return (vector, batchSize, batchOffsetInFile) -> {
+        if (vector instanceof LongColumnVector) {
+          return new OrcNoHiveLongVector((LongColumnVector) vector);
+        } else if (vector instanceof DoubleColumnVector) {
+          return new OrcNoHiveDoubleVector((DoubleColumnVector) vector);
+        } else if (vector instanceof BytesColumnVector) {
+          return new OrcNoHiveBytesVector((BytesColumnVector) vector);
+        } else if (vector instanceof DecimalColumnVector) {
+          return new OrcNoHiveDecimalVector((DecimalColumnVector) vector);
+        } else if (vector instanceof TimestampColumnVector) {
+          return new OrcNoHiveTimestampVector((TimestampColumnVector) vector);
+        } else {
+          throw new UnsupportedOperationException(
+              "Unsupported vector: " + vector.getClass().getName());
+        }
+      };
+    }
+  }
+
+  private static class LongOrcColumnVector implements org.apache.flink.table.data.vector.LongColumnVector {
+    private long batchOffsetInFile;
+
+    LongOrcColumnVector(long batchOffsetInFile) {
+      this.batchOffsetInFile = batchOffsetInFile;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return false;
+    }
+
+    @Override
+    public long getLong(int i) {
+      return batchOffsetInFile + i;
+    }
+  }
+
+  private static class StructConverter implements Converter {
+    private final Types.StructType structType;
+    private final List<Converter> fieldConverters;
+    private final Map<Integer, ?> idToConstant;
+
+    private StructConverter(Types.StructType structType, List<Converter> fieldConverters,
+                            Map<Integer, ?> idToConstant) {
+      this.structType = structType;
+      this.fieldConverters = fieldConverters;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector vector, int batchSize,
+                                long batchOffsetInFile) {
+      StructColumnVector structVector = (StructColumnVector) vector;
+      List<Types.NestedField> fields = structType.fields();
+      ColumnVector[] fieldVectors = new ColumnVector[fields.size()];
+      for (int pos = 0; pos < fields.size(); pos++) {
+        Types.NestedField field = fields.get(pos);
+        if (idToConstant.containsKey(field.fieldId())) {
+          fieldVectors[pos] = getConstantVectors(field.type(), idToConstant.get(field.fieldId()));
+        } else if (field.equals(MetadataColumns.ROW_POSITION)) {
+          fieldVectors[pos] = new LongOrcColumnVector(batchOffsetInFile);
+        } else {
+          fieldVectors[pos] = fieldConverters.get(pos)
+              .convert(structVector.fields[pos], batchSize, batchOffsetInFile);
+        }
+      }
+
+      return new FlinkRowColumnVector(fieldVectors, structVector);
+    }
+
+    private ColumnVector getConstantVectors(Type type, Object constant) {
+      if (type.isPrimitiveType()) {
+        Type.TypeID typeID = type.typeId();
+        switch (typeID) {
+          case INTEGER:
+          case DATE:
+          case TIME:
+            return ConstantColumnVectors.ints(constant);
+
+          case LONG:
+            return ConstantColumnVectors.longs(constant);
+
+          case BOOLEAN:
+            return ConstantColumnVectors.booleans(constant);
+
+          case DOUBLE:
+            return ConstantColumnVectors.doubles(constant);
+
+          case FLOAT:
+            return ConstantColumnVectors.floats(constant);
+
+          case DECIMAL:
+            return ConstantColumnVectors.decimals(constant);
+
+          case TIMESTAMP:
+            return ConstantColumnVectors.timestamps(constant);
+
+          case STRING:
+            return ConstantColumnVectors.bytes(constant);

Review comment:
       now the interface `org.apache.flink.table.data.vector.ColumnVector` in flink do not has a  sub 
    interface `StringDataColumnVector` to return `StringData`, so I use `BytesColumnVector` to parse `String`




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r632365052



##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/vectorized/TestVectorizedReads.java
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.math.BigDecimal;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.flink.FlinkCatalogTestBase;
+import org.apache.iceberg.flink.FlinkTableOptions;
+import org.apache.iceberg.flink.TestHelpers;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestVectorizedReads extends FlinkCatalogTestBase {

Review comment:
       We may need to add our own flink `MapColumnVector` in apache iceberg repo now so that we could pass all test cases after introducing the `vectorized` parameterized variable ( Because we've already had many test suites that introduced MAP data types, if the flink vectorized reader don't support MAP data type, then we have to pick those cases one by one,  It will be very trivial).




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r632363462



##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/vectorized/TestVectorizedReads.java
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.math.BigDecimal;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.flink.FlinkCatalogTestBase;
+import org.apache.iceberg.flink.FlinkTableOptions;
+import org.apache.iceberg.flink.TestHelpers;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestVectorizedReads extends FlinkCatalogTestBase {

Review comment:
       Is possible to add a parameterized variable named `vectorized` in the previous  flink+ORC test cases , so that we could cover all the existing cases in the test suites ?   Let's follow the spark's test approach [in this PR](https://github.com/apache/iceberg/commit/6fab8f57bdb7e5fe7eadc3ff41558581338e1b69) if possible.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r629849671



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        FlinkRowColumnVector cv = (FlinkRowColumnVector) converter.convert(
+            new StructColumnVector(batch.size, batch.cols), batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) {
+      return (vector, batchSize, batchOffsetInFile) -> {
+        if (vector instanceof LongColumnVector) {
+          return new OrcNoHiveLongVector((LongColumnVector) vector);
+        } else if (vector instanceof DoubleColumnVector) {
+          return new OrcNoHiveDoubleVector((DoubleColumnVector) vector);
+        } else if (vector instanceof BytesColumnVector) {
+          return new OrcNoHiveBytesVector((BytesColumnVector) vector);
+        } else if (vector instanceof DecimalColumnVector) {
+          return new OrcNoHiveDecimalVector((DecimalColumnVector) vector);
+        } else if (vector instanceof TimestampColumnVector) {
+          return new OrcNoHiveTimestampVector((TimestampColumnVector) vector);
+        } else {
+          throw new UnsupportedOperationException(
+              "Unsupported vector: " + vector.getClass().getName());
+        }
+      };
+    }
+  }
+
+  private static class LongOrcColumnVector implements org.apache.flink.table.data.vector.LongColumnVector {

Review comment:
       import?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r631585440



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java
##########
@@ -91,11 +105,85 @@ public void configure(Configuration parameters) {
 
   @Override
   public void open(FlinkInputSplit split) {
+    boolean enableVectorizedRead = readableConfig.get(FlinkTableOptions.ENABLE_VECTORIZED_READ);
+
+    if (enableVectorizedRead) {
+      if (useOrcVectorizedRead(split.getTask())) {

Review comment:
       > I think this check should be moved inside `BatchRowDateIterator`. Here the logic should be simple
   > 
   > ```
   > if (enableVectorizedRead) {
   >   ... new BatchRowDataIterator
   > } else {
   >    ... new RowDataIterator
   > }
   > ```
   
   done




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r629025442



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,381 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        StructConverter.FlinkRowColumnVector cv =
+            (StructConverter.FlinkRowColumnVector) converter.convert(new StructColumnVector(batch.size, batch.cols),
+                batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) {
+      return (vector, batchSize, batchOffsetInFile) -> {
+        if (vector instanceof LongColumnVector) {
+          return new OrcNoHiveLongVector((LongColumnVector) vector);
+        } else if (vector instanceof DoubleColumnVector) {
+          return new OrcNoHiveDoubleVector((DoubleColumnVector) vector);
+        } else if (vector instanceof BytesColumnVector) {
+          return new OrcNoHiveBytesVector((BytesColumnVector) vector);
+        } else if (vector instanceof DecimalColumnVector) {
+          return new OrcNoHiveDecimalVector((DecimalColumnVector) vector);
+        } else if (vector instanceof TimestampColumnVector) {
+          return new OrcNoHiveTimestampVector((TimestampColumnVector) vector);
+        } else {
+          throw new UnsupportedOperationException(
+              "Unsupported vector: " + vector.getClass().getName());
+        }
+      };
+    }
+  }
+
+  private static class StructConverter implements Converter {
+    private final Types.StructType structType;
+    private final List<Converter> fieldConverters;
+    private final Map<Integer, ?> idToConstant;
+
+    private StructConverter(Types.StructType structType, List<Converter> fieldConverters,
+                            Map<Integer, ?> idToConstant) {
+      this.structType = structType;
+      this.fieldConverters = fieldConverters;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector vector, int batchSize,
+                                long batchOffsetInFile) {
+      StructColumnVector structVector = (StructColumnVector) vector;
+      List<Types.NestedField> fields = structType.fields();
+      ColumnVector[] fieldVectors = new ColumnVector[fields.size()];
+      for (int pos = 0; pos < fields.size(); pos++) {
+        Types.NestedField field = fields.get(pos);
+        if (idToConstant.containsKey(field.fieldId())) {
+          fieldVectors[pos] = getConstantVectors(field.type(), idToConstant.get(field.fieldId()));
+        } else if (field.equals(MetadataColumns.ROW_POSITION)) {
+          fieldVectors[pos] = new org.apache.flink.table.data.vector.LongColumnVector() {
+
+            @Override
+            public boolean isNullAt(int i) {
+              return false;
+            }
+
+            @Override
+            public long getLong(int i) {
+              return batchOffsetInFile + i;
+            }
+          };
+        } else {
+          fieldVectors[pos] = fieldConverters.get(pos)
+              .convert(structVector.fields[pos], batchSize, batchOffsetInFile);
+        }
+      }
+
+      return new FlinkRowColumnVector(fieldVectors, structVector);
+    }
+
+    private ColumnVector getConstantVectors(Type type, Object constant) {
+      if (type.isPrimitiveType()) {
+        Type.TypeID typeID = type.typeId();
+        switch (typeID) {
+          case INTEGER:
+            return new org.apache.flink.table.data.vector.IntColumnVector() {
+              @Override
+              public boolean isNullAt(int i) {
+                return false;
+              }
+
+              @Override
+              public int getInt(int i) {
+                return (int) constant;
+              }
+            };
+
+          case LONG:
+            return new org.apache.flink.table.data.vector.LongColumnVector() {
+              @Override
+              public long getLong(int i) {
+                return (long) constant;
+              }
+
+              @Override
+              public boolean isNullAt(int i) {
+                return false;
+              }
+            };
+
+          case BOOLEAN:
+            return new org.apache.flink.table.data.vector.BooleanColumnVector() {
+              @Override
+              public boolean getBoolean(int i) {
+                return (boolean) constant;
+              }
+
+              @Override
+              public boolean isNullAt(int i) {
+                return false;
+              }
+            };
+
+          case DOUBLE:
+            return new org.apache.flink.table.data.vector.DoubleColumnVector() {
+              @Override
+              public double getDouble(int i) {
+                return (double) constant;
+              }
+
+              @Override
+              public boolean isNullAt(int i) {
+                return false;
+              }
+            };
+
+          case FLOAT:
+            return new org.apache.flink.table.data.vector.FloatColumnVector() {
+              @Override
+              public float getFloat(int i) {
+                return (float) constant;
+              }
+
+              @Override
+              public boolean isNullAt(int i) {
+                return false;
+              }
+            };
+
+          case DECIMAL:
+            return new org.apache.flink.table.data.vector.DecimalColumnVector() {
+              @Override
+              public DecimalData getDecimal(int i, int precision, int scale) {
+                return (DecimalData) constant;
+              }
+
+              @Override
+              public boolean isNullAt(int i) {
+                return false;
+              }
+            };
+
+          case DATE:
+            return new org.apache.flink.table.data.vector.IntColumnVector() {
+              @Override
+              public int getInt(int i) {
+                return (int) constant;
+              }
+
+              @Override
+              public boolean isNullAt(int i) {
+                return false;
+              }
+            };
+
+          case TIME:
+            return new org.apache.flink.table.data.vector.IntColumnVector() {
+              @Override
+              public int getInt(int i) {
+                return (int) constant;
+              }
+
+              @Override
+              public boolean isNullAt(int i) {
+                return false;
+              }
+            };
+
+          case TIMESTAMP:
+            return new org.apache.flink.table.data.vector.TimestampColumnVector() {
+              @Override
+              public TimestampData getTimestamp(int i, int precision) {
+                return (TimestampData) constant;
+              }
+
+              @Override
+              public boolean isNullAt(int i) {
+                return false;
+              }
+            };
+
+          case STRING:
+            return new org.apache.flink.table.data.vector.BytesColumnVector() {
+              @Override
+              public Bytes getBytes(int i) {
+                BinaryStringData str = (BinaryStringData) constant;
+                return new Bytes(str.toBytes(), 0, str.getSizeInBytes());
+              }
+
+              @Override
+              public boolean isNullAt(int i) {
+                return false;
+              }
+            };
+
+
+          default:
+            throw new UnsupportedOperationException("Unsupported data type for constant.");
+        }
+      } else {
+        throw new UnsupportedOperationException("ConstantColumnVector only supports primitives.");
+      }
+    }

Review comment:
       The [comment](https://github.com/apache/iceberg/pull/2566/files#r629023952) should be also applied to this section.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r629023952



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,381 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        StructConverter.FlinkRowColumnVector cv =
+            (StructConverter.FlinkRowColumnVector) converter.convert(new StructColumnVector(batch.size, batch.cols),
+                batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) {
+      return (vector, batchSize, batchOffsetInFile) -> {
+        if (vector instanceof LongColumnVector) {
+          return new OrcNoHiveLongVector((LongColumnVector) vector);
+        } else if (vector instanceof DoubleColumnVector) {
+          return new OrcNoHiveDoubleVector((DoubleColumnVector) vector);
+        } else if (vector instanceof BytesColumnVector) {
+          return new OrcNoHiveBytesVector((BytesColumnVector) vector);
+        } else if (vector instanceof DecimalColumnVector) {
+          return new OrcNoHiveDecimalVector((DecimalColumnVector) vector);
+        } else if (vector instanceof TimestampColumnVector) {
+          return new OrcNoHiveTimestampVector((TimestampColumnVector) vector);
+        } else {
+          throw new UnsupportedOperationException(
+              "Unsupported vector: " + vector.getClass().getName());
+        }
+      };
+    }
+  }
+
+  private static class StructConverter implements Converter {
+    private final Types.StructType structType;
+    private final List<Converter> fieldConverters;
+    private final Map<Integer, ?> idToConstant;
+
+    private StructConverter(Types.StructType structType, List<Converter> fieldConverters,
+                            Map<Integer, ?> idToConstant) {
+      this.structType = structType;
+      this.fieldConverters = fieldConverters;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector vector, int batchSize,
+                                long batchOffsetInFile) {
+      StructColumnVector structVector = (StructColumnVector) vector;
+      List<Types.NestedField> fields = structType.fields();
+      ColumnVector[] fieldVectors = new ColumnVector[fields.size()];
+      for (int pos = 0; pos < fields.size(); pos++) {
+        Types.NestedField field = fields.get(pos);
+        if (idToConstant.containsKey(field.fieldId())) {
+          fieldVectors[pos] = getConstantVectors(field.type(), idToConstant.get(field.fieldId()));
+        } else if (field.equals(MetadataColumns.ROW_POSITION)) {
+          fieldVectors[pos] = new org.apache.flink.table.data.vector.LongColumnVector() {
+
+            @Override
+            public boolean isNullAt(int i) {
+              return false;
+            }
+
+            @Override
+            public long getLong(int i) {
+              return batchOffsetInFile + i;
+            }
+          };

Review comment:
       I will suggest to introduce a separate `LongOrcColumnVector`  to read the correct value.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r630718096



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java
##########
@@ -91,11 +105,85 @@ public void configure(Configuration parameters) {
 
   @Override
   public void open(FlinkInputSplit split) {
+    boolean enableVectorizedRead = readableConfig.get(FlinkTableOptions.ENABLE_VECTORIZED_READ);
+
+    if (enableVectorizedRead) {
+      if (useOrcVectorizedRead(split.getTask())) {

Review comment:
       We also don't need the separate method of `setDefaultIterator`




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r629837699



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java
##########
@@ -49,19 +49,19 @@
 import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
 import org.apache.orc.storage.serde2.io.HiveDecimalWritable;
 
-class FlinkOrcReaders {
+public class FlinkOrcReaders {
   private FlinkOrcReaders() {
   }
 
-  static OrcValueReader<StringData> strings() {
+  public static OrcValueReader<StringData> strings() {

Review comment:
       curious why only making these 3 public. what about other package scope methods like `times`, `timestamps`, `array`?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r631586729



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        FlinkRowColumnVector cv = (FlinkRowColumnVector) converter.convert(
+            new StructColumnVector(batch.size, batch.cols), batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) {
+      return (vector, batchSize, batchOffsetInFile) -> {
+        if (vector instanceof LongColumnVector) {
+          return new OrcNoHiveLongVector((LongColumnVector) vector);
+        } else if (vector instanceof DoubleColumnVector) {
+          return new OrcNoHiveDoubleVector((DoubleColumnVector) vector);
+        } else if (vector instanceof BytesColumnVector) {
+          return new OrcNoHiveBytesVector((BytesColumnVector) vector);
+        } else if (vector instanceof DecimalColumnVector) {
+          return new OrcNoHiveDecimalVector((DecimalColumnVector) vector);
+        } else if (vector instanceof TimestampColumnVector) {
+          return new OrcNoHiveTimestampVector((TimestampColumnVector) vector);
+        } else {
+          throw new UnsupportedOperationException(
+              "Unsupported vector: " + vector.getClass().getName());
+        }
+      };
+    }
+  }
+
+  private static class LongOrcColumnVector implements org.apache.flink.table.data.vector.LongColumnVector {
+    private long batchOffsetInFile;
+
+    LongOrcColumnVector(long batchOffsetInFile) {
+      this.batchOffsetInFile = batchOffsetInFile;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return false;
+    }
+
+    @Override
+    public long getLong(int i) {
+      return batchOffsetInFile + i;
+    }
+  }
+
+  private static class StructConverter implements Converter {
+    private final Types.StructType structType;
+    private final List<Converter> fieldConverters;
+    private final Map<Integer, ?> idToConstant;
+
+    private StructConverter(Types.StructType structType, List<Converter> fieldConverters,
+                            Map<Integer, ?> idToConstant) {
+      this.structType = structType;
+      this.fieldConverters = fieldConverters;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector vector, int batchSize,
+                                long batchOffsetInFile) {
+      StructColumnVector structVector = (StructColumnVector) vector;
+      List<Types.NestedField> fields = structType.fields();
+      ColumnVector[] fieldVectors = new ColumnVector[fields.size()];
+      for (int pos = 0; pos < fields.size(); pos++) {
+        Types.NestedField field = fields.get(pos);
+        if (idToConstant.containsKey(field.fieldId())) {
+          fieldVectors[pos] = getConstantVectors(field.type(), idToConstant.get(field.fieldId()));
+        } else if (field.equals(MetadataColumns.ROW_POSITION)) {
+          fieldVectors[pos] = new LongOrcColumnVector(batchOffsetInFile);
+        } else {
+          fieldVectors[pos] = fieldConverters.get(pos)
+              .convert(structVector.fields[pos], batchSize, batchOffsetInFile);
+        }
+      }
+
+      return new FlinkRowColumnVector(fieldVectors, structVector);
+    }
+
+    private ColumnVector getConstantVectors(Type type, Object constant) {
+      if (type.isPrimitiveType()) {
+        Type.TypeID typeID = type.typeId();
+        switch (typeID) {
+          case INTEGER:
+          case DATE:
+          case TIME:
+            return ConstantColumnVectors.ints(constant);
+
+          case LONG:
+            return ConstantColumnVectors.longs(constant);
+
+          case BOOLEAN:
+            return ConstantColumnVectors.booleans(constant);
+
+          case DOUBLE:
+            return ConstantColumnVectors.doubles(constant);
+
+          case FLOAT:
+            return ConstantColumnVectors.floats(constant);
+
+          case DECIMAL:
+            return ConstantColumnVectors.decimals(constant);
+
+          case TIMESTAMP:
+            return ConstantColumnVectors.timestamps(constant);
+
+          case STRING:
+            return ConstantColumnVectors.bytes(constant);
+
+          default:
+            throw new UnsupportedOperationException("Unsupported data type for constant.");
+        }
+      } else {
+        throw new UnsupportedOperationException("ConstantColumnVector only supports primitives.");
+      }
+    }
+
+    private static class ArrayConverter implements Converter {
+      private final Converter elementConverter;
+
+      private ArrayConverter(Converter elementConverter) {
+        this.elementConverter = elementConverter;
+      }
+
+      @Override
+      public ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector vector, int batchSize,
+                                  long batchOffsetInFile) {
+        ListColumnVector listVector = (ListColumnVector) vector;
+        ColumnVector elementVector = elementConverter.convert(listVector.child, batchSize, batchOffsetInFile);
+
+        return new ArrayColumnVector() {
+          @Override
+          public ArrayData getArray(int rowId) {
+            int index = getRowIndex(rowId);
+            return new ColumnarArrayData(elementVector, (int) listVector.offsets[index],
+                (int) listVector.lengths[index]);
+          }
+
+          @Override
+          public boolean isNullAt(int rowId) {
+            return vector.isNull[getRowIndex(rowId)];
+          }
+
+          private int getRowIndex(int rowId) {
+            return vector.isRepeating ? 0 : rowId;
+          }
+        };
+      }
+    }
+  }
+
+  private static class FlinkRowColumnVector implements RowColumnVector {
+
+    private ColumnVector[] fieldVectors;
+    private StructColumnVector structVector;
+
+    FlinkRowColumnVector(ColumnVector[] fieldVectors,
+                         StructColumnVector structVector) {
+      this.fieldVectors = fieldVectors;
+      this.structVector = structVector;
+    }
+
+    @Override
+    public ColumnarRowData getRow(int i) {
+      VectorizedColumnBatch vectorizedColumnBatch = new VectorizedColumnBatch(fieldVectors);

Review comment:
       I updated it




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r632357231



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        FlinkRowColumnVector cv = (FlinkRowColumnVector) converter.convert(
+            new StructColumnVector(batch.size, batch.cols), batch.size, batchOffsetInFile);

Review comment:
       Should we add comment to explain this ? I think it will be easier for people to read this code.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r632330112



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/ConstantColumnVectors.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import org.apache.flink.table.data.vector.BooleanColumnVector;
+import org.apache.flink.table.data.vector.BytesColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.DecimalColumnVector;
+import org.apache.flink.table.data.vector.DoubleColumnVector;
+import org.apache.flink.table.data.vector.FloatColumnVector;
+import org.apache.flink.table.data.vector.IntColumnVector;
+import org.apache.flink.table.data.vector.LongColumnVector;
+import org.apache.flink.table.data.vector.TimestampColumnVector;
+
+class ConstantColumnVectors {
+  private ConstantColumnVectors() {
+  }
+
+  static ColumnVector ints(Object constant) {
+    return new ConstantIntColumnVector(constant);
+  }
+
+  static ColumnVector longs(Object constant) {
+    return new ConstantLongColumnVongector(constant);
+  }
+
+  static ColumnVector booleans(Object constant) {
+    return new ConstantBooleanColumnVector(constant);
+  }
+
+  static ColumnVector doubles(Object constant) {
+    return new ConstantDoubleColumnVector(constant);
+  }
+
+  static ColumnVector floats(Object constant) {
+    return new ConstantFloatColumnVector(constant);
+  }
+
+  static ColumnVector decimals(Object constant) {
+    return new ConstantDecimalColumnVector(constant);
+  }
+
+  static ColumnVector timestamps(Object constant) {
+    return new ConstantTimestampColumnVector(constant);
+  }
+
+  static ColumnVector bytes(Object constant) {
+    return new ConstantBytesColumnVector(constant);
+  }
+
+  private static class ConstantIntColumnVector implements IntColumnVector {
+
+    private final Object constant;
+
+    private ConstantIntColumnVector(Object constant) {
+      this.constant = constant;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return constant == null;
+    }
+
+    @Override
+    public int getInt(int i) {
+      return (int) constant;
+    }
+  }
+
+  private static class ConstantLongColumnVongector implements LongColumnVector {

Review comment:
       Typo ?  `ConstantLongColumnVongector` -> `ConstantLongColumnVector`

##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        FlinkRowColumnVector cv = (FlinkRowColumnVector) converter.convert(
+            new StructColumnVector(batch.size, batch.cols), batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException("Unsupported vectorized read for map type.");
+    }
+
+    @Override
+    public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) {
+      return (vector, batchSize, batchOffsetInFile) -> {
+        if (vector instanceof LongColumnVector) {
+          return new OrcNoHiveLongVector((LongColumnVector) vector);
+        } else if (vector instanceof DoubleColumnVector) {
+          return new OrcNoHiveDoubleVector((DoubleColumnVector) vector);
+        } else if (vector instanceof BytesColumnVector) {
+          return new OrcNoHiveBytesVector((BytesColumnVector) vector);
+        } else if (vector instanceof DecimalColumnVector) {
+          return new OrcNoHiveDecimalVector((DecimalColumnVector) vector);
+        } else if (vector instanceof TimestampColumnVector) {
+          return new OrcNoHiveTimestampVector((TimestampColumnVector) vector);
+        } else {
+          throw new UnsupportedOperationException(
+              "Unsupported vector: " + vector.getClass().getName());
+        }
+      };
+    }
+  }
+
+  private static class RowPositionColumnVector implements org.apache.flink.table.data.vector.LongColumnVector {
+    private final long batchOffsetInFile;
+
+    RowPositionColumnVector(long batchOffsetInFile) {
+      this.batchOffsetInFile = batchOffsetInFile;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return false;
+    }
+
+    @Override
+    public long getLong(int i) {
+      return batchOffsetInFile + i;
+    }
+  }
+
+  private static class StructConverter implements Converter {
+    private final Types.StructType structType;
+    private final List<Converter> fieldConverters;
+    private final Map<Integer, ?> idToConstant;
+
+    private StructConverter(Types.StructType structType, List<Converter> fieldConverters,
+                            Map<Integer, ?> idToConstant) {
+      this.structType = structType;
+      this.fieldConverters = fieldConverters;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector vector, int batchSize,
+                                long batchOffsetInFile) {
+      StructColumnVector structVector = (StructColumnVector) vector;
+      List<Types.NestedField> fields = structType.fields();
+      ColumnVector[] fieldVectors = new ColumnVector[fields.size()];
+      for (int pos = 0; pos < fields.size(); pos++) {
+        Types.NestedField field = fields.get(pos);
+        if (idToConstant.containsKey(field.fieldId())) {
+          fieldVectors[pos] = toConstantColumnVector(field.type(), idToConstant.get(field.fieldId()));
+        } else if (field.equals(MetadataColumns.ROW_POSITION)) {
+          fieldVectors[pos] = new RowPositionColumnVector(batchOffsetInFile);
+        } else {
+          fieldVectors[pos] = fieldConverters.get(pos)
+              .convert(structVector.fields[pos], batchSize, batchOffsetInFile);
+        }
+      }
+
+      return new FlinkRowColumnVector(fieldVectors, structVector);
+    }
+
+    private ColumnVector toConstantColumnVector(Type type, Object constant) {
+      Type.TypeID typeID = type.typeId();
+      switch (typeID) {
+        case INTEGER:
+        case DATE:
+        case TIME:
+          return ConstantColumnVectors.ints(constant);
+
+        case LONG:
+          return ConstantColumnVectors.longs(constant);
+
+        case BOOLEAN:
+          return ConstantColumnVectors.booleans(constant);
+
+        case DOUBLE:
+          return ConstantColumnVectors.doubles(constant);
+
+        case FLOAT:
+          return ConstantColumnVectors.floats(constant);
+
+        case DECIMAL:
+          return ConstantColumnVectors.decimals(constant);
+
+        case TIMESTAMP:
+          return ConstantColumnVectors.timestamps(constant);
+
+        case FIXED:
+        case UUID:
+        case BINARY:

Review comment:
       Be careful that the `FIXED` data type has three constant data type:
   * byte[]
   * GenerateData.Fixed
   * ByteBuffer
   
   We will need to make the `ConstantBytesColumnVector` handle all those types ! 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        FlinkRowColumnVector cv = (FlinkRowColumnVector) converter.convert(
+            new StructColumnVector(batch.size, batch.cols), batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException("Unsupported vectorized read for map type.");
+    }
+
+    @Override
+    public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) {
+      return (vector, batchSize, batchOffsetInFile) -> {
+        if (vector instanceof LongColumnVector) {
+          return new OrcNoHiveLongVector((LongColumnVector) vector);
+        } else if (vector instanceof DoubleColumnVector) {
+          return new OrcNoHiveDoubleVector((DoubleColumnVector) vector);
+        } else if (vector instanceof BytesColumnVector) {
+          return new OrcNoHiveBytesVector((BytesColumnVector) vector);
+        } else if (vector instanceof DecimalColumnVector) {
+          return new OrcNoHiveDecimalVector((DecimalColumnVector) vector);
+        } else if (vector instanceof TimestampColumnVector) {
+          return new OrcNoHiveTimestampVector((TimestampColumnVector) vector);
+        } else {
+          throw new UnsupportedOperationException(
+              "Unsupported vector: " + vector.getClass().getName());
+        }
+      };
+    }
+  }
+
+  private static class RowPositionColumnVector implements org.apache.flink.table.data.vector.LongColumnVector {
+    private final long batchOffsetInFile;
+
+    RowPositionColumnVector(long batchOffsetInFile) {
+      this.batchOffsetInFile = batchOffsetInFile;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return false;
+    }
+
+    @Override
+    public long getLong(int i) {
+      return batchOffsetInFile + i;
+    }
+  }
+
+  private static class StructConverter implements Converter {
+    private final Types.StructType structType;
+    private final List<Converter> fieldConverters;
+    private final Map<Integer, ?> idToConstant;
+
+    private StructConverter(Types.StructType structType, List<Converter> fieldConverters,
+                            Map<Integer, ?> idToConstant) {
+      this.structType = structType;
+      this.fieldConverters = fieldConverters;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector vector, int batchSize,
+                                long batchOffsetInFile) {
+      StructColumnVector structVector = (StructColumnVector) vector;
+      List<Types.NestedField> fields = structType.fields();
+      ColumnVector[] fieldVectors = new ColumnVector[fields.size()];
+      for (int pos = 0; pos < fields.size(); pos++) {
+        Types.NestedField field = fields.get(pos);
+        if (idToConstant.containsKey(field.fieldId())) {
+          fieldVectors[pos] = toConstantColumnVector(field.type(), idToConstant.get(field.fieldId()));
+        } else if (field.equals(MetadataColumns.ROW_POSITION)) {
+          fieldVectors[pos] = new RowPositionColumnVector(batchOffsetInFile);
+        } else {
+          fieldVectors[pos] = fieldConverters.get(pos)
+              .convert(structVector.fields[pos], batchSize, batchOffsetInFile);
+        }
+      }
+
+      return new FlinkRowColumnVector(fieldVectors, structVector);
+    }
+
+    private ColumnVector toConstantColumnVector(Type type, Object constant) {
+      Type.TypeID typeID = type.typeId();
+      switch (typeID) {
+        case INTEGER:
+        case DATE:
+        case TIME:
+          return ConstantColumnVectors.ints(constant);
+
+        case LONG:
+          return ConstantColumnVectors.longs(constant);
+
+        case BOOLEAN:
+          return ConstantColumnVectors.booleans(constant);
+
+        case DOUBLE:
+          return ConstantColumnVectors.doubles(constant);
+
+        case FLOAT:
+          return ConstantColumnVectors.floats(constant);
+
+        case DECIMAL:
+          return ConstantColumnVectors.decimals(constant);
+
+        case TIMESTAMP:
+          return ConstantColumnVectors.timestamps(constant);
+
+        case FIXED:
+        case UUID:
+        case BINARY:

Review comment:
       I still don't think we could use the `ConstantBytesColumnVector` to read the `BINARY` & `FIXED` data type because its constant object is a [StringData](https://github.com/apache/iceberg/blob/90225d6c9413016d611e2ce5eff37db1bc1b4fc5/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java#L66),  we will definitely encounter a class cast exception when cast the `byte[]` to `BinaryStringData` ,  do we have an unit test to cover this ? 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        FlinkRowColumnVector cv = (FlinkRowColumnVector) converter.convert(
+            new StructColumnVector(batch.size, batch.cols), batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException("Unsupported vectorized read for map type.");
+    }
+
+    @Override
+    public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) {
+      return (vector, batchSize, batchOffsetInFile) -> {
+        if (vector instanceof LongColumnVector) {
+          return new OrcNoHiveLongVector((LongColumnVector) vector);
+        } else if (vector instanceof DoubleColumnVector) {

Review comment:
       The flink's `OrcNoHiveDoubleVector` have a very strange design, sounds like it could read both `float` & `double` data type from it.  I was confused by this name a lot.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/ConstantColumnVectors.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import org.apache.flink.table.data.vector.BooleanColumnVector;
+import org.apache.flink.table.data.vector.BytesColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.DecimalColumnVector;
+import org.apache.flink.table.data.vector.DoubleColumnVector;
+import org.apache.flink.table.data.vector.FloatColumnVector;
+import org.apache.flink.table.data.vector.IntColumnVector;
+import org.apache.flink.table.data.vector.LongColumnVector;
+import org.apache.flink.table.data.vector.TimestampColumnVector;
+
+class ConstantColumnVectors {
+  private ConstantColumnVectors() {
+  }
+
+  static ColumnVector ints(Object constant) {
+    return new ConstantIntColumnVector(constant);
+  }
+
+  static ColumnVector longs(Object constant) {
+    return new ConstantLongColumnVongector(constant);
+  }
+
+  static ColumnVector booleans(Object constant) {
+    return new ConstantBooleanColumnVector(constant);
+  }
+
+  static ColumnVector doubles(Object constant) {
+    return new ConstantDoubleColumnVector(constant);
+  }
+
+  static ColumnVector floats(Object constant) {
+    return new ConstantFloatColumnVector(constant);
+  }
+
+  static ColumnVector decimals(Object constant) {
+    return new ConstantDecimalColumnVector(constant);
+  }
+
+  static ColumnVector timestamps(Object constant) {
+    return new ConstantTimestampColumnVector(constant);
+  }
+
+  static ColumnVector bytes(Object constant) {
+    return new ConstantBytesColumnVector(constant);
+  }
+
+  private static class ConstantIntColumnVector implements IntColumnVector {
+
+    private final Object constant;
+
+    private ConstantIntColumnVector(Object constant) {
+      this.constant = constant;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return constant == null;
+    }
+
+    @Override
+    public int getInt(int i) {
+      return (int) constant;
+    }
+  }
+
+  private static class ConstantLongColumnVongector implements LongColumnVector {
+
+    private final Object constant;
+
+    private ConstantLongColumnVongector(Object constant) {
+      this.constant = constant;
+    }
+
+    @Override
+    public long getLong(int i) {
+      return (long) constant;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return constant == null;
+    }
+  }
+
+  private static class ConstantBooleanColumnVector implements BooleanColumnVector {
+    private final Object constant;
+
+    private ConstantBooleanColumnVector(Object constant) {
+      this.constant = constant;
+    }
+
+    @Override
+    public boolean getBoolean(int i) {
+      return (boolean) constant;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return constant == null;
+    }
+  }
+
+  private static class ConstantDoubleColumnVector implements DoubleColumnVector {
+    private final Object constant;
+
+    private ConstantDoubleColumnVector(Object constant) {
+      this.constant = constant;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return constant == null;
+    }
+
+    @Override
+    public double getDouble(int i) {
+      return (double) constant;
+    }
+  }
+
+  private static class ConstantFloatColumnVector implements FloatColumnVector {
+    private final Object constant;
+
+    private ConstantFloatColumnVector(Object constant) {
+      this.constant = constant;
+    }
+
+    @Override
+    public float getFloat(int i) {
+      return (float) constant;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return constant == null;
+    }
+  }
+
+  private static class ConstantDecimalColumnVector implements DecimalColumnVector {
+    private final Object constant;
+
+    private ConstantDecimalColumnVector(Object constant) {
+      this.constant = constant;
+    }
+
+    @Override
+    public DecimalData getDecimal(int i, int precision, int scale) {
+      return (DecimalData) constant;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return constant == null;
+    }
+  }
+
+  private static class ConstantTimestampColumnVector implements TimestampColumnVector {
+    private final Object constant;
+
+    private ConstantTimestampColumnVector(Object constant) {
+      this.constant = constant;
+    }
+
+    @Override
+    public TimestampData getTimestamp(int i, int precision) {
+      return (TimestampData) constant;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return constant == null;
+    }
+  }
+
+  private static class ConstantBytesColumnVector implements BytesColumnVector {
+    private final Object constant;
+
+    private ConstantBytesColumnVector(Object constant) {
+      this.constant = constant;
+    }
+
+    @Override
+    public Bytes getBytes(int i) {
+      BinaryStringData str = (BinaryStringData) constant;
+      return new Bytes(str.toBytes(), 0, str.getSizeInBytes());

Review comment:
       The `str.toBytes()` will create a new object byte array each time, for a constant value  it's not worth to new byte[] every time.  I will suggest to allocate that data bytes in the constructor, and then always return the same instance. 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/ConstantColumnVectors.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import org.apache.flink.table.data.vector.BooleanColumnVector;
+import org.apache.flink.table.data.vector.BytesColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.DecimalColumnVector;
+import org.apache.flink.table.data.vector.DoubleColumnVector;
+import org.apache.flink.table.data.vector.FloatColumnVector;
+import org.apache.flink.table.data.vector.IntColumnVector;
+import org.apache.flink.table.data.vector.LongColumnVector;
+import org.apache.flink.table.data.vector.TimestampColumnVector;
+
+class ConstantColumnVectors {
+  private ConstantColumnVectors() {
+  }
+
+  static ColumnVector ints(Object constant) {
+    return new ConstantIntColumnVector(constant);
+  }
+
+  static ColumnVector longs(Object constant) {
+    return new ConstantLongColumnVongector(constant);
+  }
+
+  static ColumnVector booleans(Object constant) {
+    return new ConstantBooleanColumnVector(constant);
+  }
+
+  static ColumnVector doubles(Object constant) {
+    return new ConstantDoubleColumnVector(constant);
+  }
+
+  static ColumnVector floats(Object constant) {
+    return new ConstantFloatColumnVector(constant);
+  }
+
+  static ColumnVector decimals(Object constant) {
+    return new ConstantDecimalColumnVector(constant);
+  }
+
+  static ColumnVector timestamps(Object constant) {
+    return new ConstantTimestampColumnVector(constant);
+  }
+
+  static ColumnVector bytes(Object constant) {
+    return new ConstantBytesColumnVector(constant);
+  }
+
+  private static class ConstantIntColumnVector implements IntColumnVector {
+
+    private final Object constant;
+
+    private ConstantIntColumnVector(Object constant) {
+      this.constant = constant;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return constant == null;
+    }
+
+    @Override
+    public int getInt(int i) {
+      return (int) constant;
+    }
+  }
+
+  private static class ConstantLongColumnVongector implements LongColumnVector {
+
+    private final Object constant;
+
+    private ConstantLongColumnVongector(Object constant) {
+      this.constant = constant;
+    }
+
+    @Override
+    public long getLong(int i) {
+      return (long) constant;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return constant == null;
+    }

Review comment:
       Nit: Could we keep all those ConstantXXColumnVector has the same order for `getLong` method and `isNullAt` method ?  I see `ConstantIntColumnVector` put the `isNullAt` ahead of `getInt`, but `ConstantLongColumnVongector` is the reversed order.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        FlinkRowColumnVector cv = (FlinkRowColumnVector) converter.convert(
+            new StructColumnVector(batch.size, batch.cols), batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException("Unsupported vectorized read for map type.");
+    }
+
+    @Override
+    public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) {
+      return (vector, batchSize, batchOffsetInFile) -> {
+        if (vector instanceof LongColumnVector) {
+          return new OrcNoHiveLongVector((LongColumnVector) vector);
+        } else if (vector instanceof DoubleColumnVector) {
+          return new OrcNoHiveDoubleVector((DoubleColumnVector) vector);
+        } else if (vector instanceof BytesColumnVector) {
+          return new OrcNoHiveBytesVector((BytesColumnVector) vector);
+        } else if (vector instanceof DecimalColumnVector) {
+          return new OrcNoHiveDecimalVector((DecimalColumnVector) vector);
+        } else if (vector instanceof TimestampColumnVector) {
+          return new OrcNoHiveTimestampVector((TimestampColumnVector) vector);
+        } else {
+          throw new UnsupportedOperationException(
+              "Unsupported vector: " + vector.getClass().getName());

Review comment:
       Please add the iceberg `iPrimitive` data type & ORC `primitive` data type in this error message. 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        FlinkRowColumnVector cv = (FlinkRowColumnVector) converter.convert(
+            new StructColumnVector(batch.size, batch.cols), batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException("Unsupported vectorized read for map type.");
+    }
+
+    @Override
+    public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) {
+      return (vector, batchSize, batchOffsetInFile) -> {
+        if (vector instanceof LongColumnVector) {
+          return new OrcNoHiveLongVector((LongColumnVector) vector);
+        } else if (vector instanceof DoubleColumnVector) {
+          return new OrcNoHiveDoubleVector((DoubleColumnVector) vector);
+        } else if (vector instanceof BytesColumnVector) {
+          return new OrcNoHiveBytesVector((BytesColumnVector) vector);
+        } else if (vector instanceof DecimalColumnVector) {
+          return new OrcNoHiveDecimalVector((DecimalColumnVector) vector);
+        } else if (vector instanceof TimestampColumnVector) {
+          return new OrcNoHiveTimestampVector((TimestampColumnVector) vector);
+        } else {
+          throw new UnsupportedOperationException(
+              "Unsupported vector: " + vector.getClass().getName());
+        }
+      };
+    }
+  }
+
+  private static class RowPositionColumnVector implements org.apache.flink.table.data.vector.LongColumnVector {
+    private final long batchOffsetInFile;
+
+    RowPositionColumnVector(long batchOffsetInFile) {
+      this.batchOffsetInFile = batchOffsetInFile;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return false;
+    }
+
+    @Override
+    public long getLong(int i) {
+      return batchOffsetInFile + i;
+    }
+  }
+
+  private static class StructConverter implements Converter {
+    private final Types.StructType structType;
+    private final List<Converter> fieldConverters;
+    private final Map<Integer, ?> idToConstant;
+
+    private StructConverter(Types.StructType structType, List<Converter> fieldConverters,
+                            Map<Integer, ?> idToConstant) {
+      this.structType = structType;
+      this.fieldConverters = fieldConverters;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector vector, int batchSize,
+                                long batchOffsetInFile) {
+      StructColumnVector structVector = (StructColumnVector) vector;
+      List<Types.NestedField> fields = structType.fields();
+      ColumnVector[] fieldVectors = new ColumnVector[fields.size()];
+      for (int pos = 0; pos < fields.size(); pos++) {
+        Types.NestedField field = fields.get(pos);
+        if (idToConstant.containsKey(field.fieldId())) {
+          fieldVectors[pos] = toConstantColumnVector(field.type(), idToConstant.get(field.fieldId()));
+        } else if (field.equals(MetadataColumns.ROW_POSITION)) {
+          fieldVectors[pos] = new RowPositionColumnVector(batchOffsetInFile);
+        } else {
+          fieldVectors[pos] = fieldConverters.get(pos)
+              .convert(structVector.fields[pos], batchSize, batchOffsetInFile);
+        }
+      }
+
+      return new FlinkRowColumnVector(fieldVectors, structVector);
+    }
+
+    private ColumnVector toConstantColumnVector(Type type, Object constant) {
+      Type.TypeID typeID = type.typeId();
+      switch (typeID) {
+        case INTEGER:
+        case DATE:
+        case TIME:
+          return ConstantColumnVectors.ints(constant);
+
+        case LONG:
+          return ConstantColumnVectors.longs(constant);
+
+        case BOOLEAN:
+          return ConstantColumnVectors.booleans(constant);
+
+        case DOUBLE:
+          return ConstantColumnVectors.doubles(constant);
+
+        case FLOAT:
+          return ConstantColumnVectors.floats(constant);
+
+        case DECIMAL:
+          return ConstantColumnVectors.decimals(constant);
+
+        case TIMESTAMP:
+          return ConstantColumnVectors.timestamps(constant);
+
+        case FIXED:
+        case UUID:
+        case BINARY:
+        case STRING:
+          return ConstantColumnVectors.bytes(constant);
+
+        default:
+          throw new UnsupportedOperationException("Unsupported data type for constant.");
+      }
+    }
+
+    private static class ArrayConverter implements Converter {
+      private final Converter elementConverter;
+
+      private ArrayConverter(Converter elementConverter) {
+        this.elementConverter = elementConverter;
+      }
+
+      @Override
+      public ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector vector, int batchSize,
+                                  long batchOffsetInFile) {
+        ListColumnVector listVector = (ListColumnVector) vector;
+        ColumnVector elementVector = elementConverter.convert(listVector.child, batchSize, batchOffsetInFile);
+
+        return new ArrayColumnVector() {
+          @Override
+          public ArrayData getArray(int rowId) {
+            int index = getRowIndex(rowId);
+            return new ColumnarArrayData(elementVector, (int) listVector.offsets[index],
+                (int) listVector.lengths[index]);
+          }
+
+          @Override
+          public boolean isNullAt(int rowId) {
+            return vector.isNull[getRowIndex(rowId)];
+          }
+
+          private int getRowIndex(int rowId) {
+            return vector.isRepeating ? 0 : rowId;
+          }
+        };
+      }
+    }
+  }
+
+  private static class FlinkRowColumnVector implements RowColumnVector {
+
+    private final ColumnVector[] fieldVectors;
+    private final StructColumnVector structVector;
+    private final VectorizedColumnBatch vectorizedColumnBatch;
+
+    FlinkRowColumnVector(ColumnVector[] fieldVectors,
+                         StructColumnVector structVector) {
+      this.fieldVectors = fieldVectors;
+      this.structVector = structVector;
+      vectorizedColumnBatch = new VectorizedColumnBatch(fieldVectors);

Review comment:
       Nit:  It will be good if we could algin the code style with the above line :  `this.vectorizedColumnBatch=new VectorizedColumnBatch(fieldVectors)`.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r629032357



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,381 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        StructConverter.FlinkRowColumnVector cv =
+            (StructConverter.FlinkRowColumnVector) converter.convert(new StructColumnVector(batch.size, batch.cols),
+                batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) {
+      return (vector, batchSize, batchOffsetInFile) -> {
+        if (vector instanceof LongColumnVector) {
+          return new OrcNoHiveLongVector((LongColumnVector) vector);
+        } else if (vector instanceof DoubleColumnVector) {
+          return new OrcNoHiveDoubleVector((DoubleColumnVector) vector);
+        } else if (vector instanceof BytesColumnVector) {
+          return new OrcNoHiveBytesVector((BytesColumnVector) vector);
+        } else if (vector instanceof DecimalColumnVector) {
+          return new OrcNoHiveDecimalVector((DecimalColumnVector) vector);
+        } else if (vector instanceof TimestampColumnVector) {
+          return new OrcNoHiveTimestampVector((TimestampColumnVector) vector);
+        } else {
+          throw new UnsupportedOperationException(
+              "Unsupported vector: " + vector.getClass().getName());
+        }
+      };
+    }
+  }
+
+  private static class StructConverter implements Converter {
+    private final Types.StructType structType;
+    private final List<Converter> fieldConverters;
+    private final Map<Integer, ?> idToConstant;
+
+    private StructConverter(Types.StructType structType, List<Converter> fieldConverters,
+                            Map<Integer, ?> idToConstant) {
+      this.structType = structType;
+      this.fieldConverters = fieldConverters;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector vector, int batchSize,
+                                long batchOffsetInFile) {
+      StructColumnVector structVector = (StructColumnVector) vector;
+      List<Types.NestedField> fields = structType.fields();
+      ColumnVector[] fieldVectors = new ColumnVector[fields.size()];
+      for (int pos = 0; pos < fields.size(); pos++) {
+        Types.NestedField field = fields.get(pos);
+        if (idToConstant.containsKey(field.fieldId())) {
+          fieldVectors[pos] = getConstantVectors(field.type(), idToConstant.get(field.fieldId()));
+        } else if (field.equals(MetadataColumns.ROW_POSITION)) {
+          fieldVectors[pos] = new org.apache.flink.table.data.vector.LongColumnVector() {
+
+            @Override
+            public boolean isNullAt(int i) {
+              return false;
+            }
+
+            @Override
+            public long getLong(int i) {
+              return batchOffsetInFile + i;
+            }
+          };
+        } else {
+          fieldVectors[pos] = fieldConverters.get(pos)
+              .convert(structVector.fields[pos], batchSize, batchOffsetInFile);
+        }
+      }
+
+      return new FlinkRowColumnVector(fieldVectors, structVector);
+    }
+
+    private ColumnVector getConstantVectors(Type type, Object constant) {
+      if (type.isPrimitiveType()) {
+        Type.TypeID typeID = type.typeId();
+        switch (typeID) {
+          case INTEGER:
+            return new org.apache.flink.table.data.vector.IntColumnVector() {
+              @Override
+              public boolean isNullAt(int i) {
+                return false;
+              }
+
+              @Override
+              public int getInt(int i) {
+                return (int) constant;
+              }
+            };
+
+          case LONG:
+            return new org.apache.flink.table.data.vector.LongColumnVector() {
+              @Override
+              public long getLong(int i) {
+                return (long) constant;
+              }
+
+              @Override
+              public boolean isNullAt(int i) {
+                return false;
+              }
+            };
+
+          case BOOLEAN:
+            return new org.apache.flink.table.data.vector.BooleanColumnVector() {
+              @Override
+              public boolean getBoolean(int i) {
+                return (boolean) constant;
+              }
+
+              @Override
+              public boolean isNullAt(int i) {
+                return false;
+              }
+            };
+
+          case DOUBLE:
+            return new org.apache.flink.table.data.vector.DoubleColumnVector() {
+              @Override
+              public double getDouble(int i) {
+                return (double) constant;
+              }
+
+              @Override
+              public boolean isNullAt(int i) {
+                return false;
+              }
+            };
+
+          case FLOAT:
+            return new org.apache.flink.table.data.vector.FloatColumnVector() {
+              @Override
+              public float getFloat(int i) {
+                return (float) constant;
+              }
+
+              @Override
+              public boolean isNullAt(int i) {
+                return false;
+              }
+            };
+
+          case DECIMAL:
+            return new org.apache.flink.table.data.vector.DecimalColumnVector() {
+              @Override
+              public DecimalData getDecimal(int i, int precision, int scale) {
+                return (DecimalData) constant;
+              }
+
+              @Override
+              public boolean isNullAt(int i) {
+                return false;
+              }
+            };
+
+          case DATE:
+            return new org.apache.flink.table.data.vector.IntColumnVector() {
+              @Override
+              public int getInt(int i) {
+                return (int) constant;
+              }
+
+              @Override
+              public boolean isNullAt(int i) {
+                return false;
+              }
+            };
+
+          case TIME:
+            return new org.apache.flink.table.data.vector.IntColumnVector() {
+              @Override
+              public int getInt(int i) {
+                return (int) constant;
+              }
+
+              @Override
+              public boolean isNullAt(int i) {
+                return false;
+              }
+            };
+
+          case TIMESTAMP:
+            return new org.apache.flink.table.data.vector.TimestampColumnVector() {
+              @Override
+              public TimestampData getTimestamp(int i, int precision) {
+                return (TimestampData) constant;
+              }
+
+              @Override
+              public boolean isNullAt(int i) {
+                return false;
+              }
+            };
+
+          case STRING:
+            return new org.apache.flink.table.data.vector.BytesColumnVector() {
+              @Override
+              public Bytes getBytes(int i) {
+                BinaryStringData str = (BinaryStringData) constant;
+                return new Bytes(str.toBytes(), 0, str.getSizeInBytes());
+              }
+
+              @Override
+              public boolean isNullAt(int i) {
+                return false;
+              }
+            };
+
+
+          default:
+            throw new UnsupportedOperationException("Unsupported data type for constant.");
+        }
+      } else {
+        throw new UnsupportedOperationException("ConstantColumnVector only supports primitives.");
+      }
+    }
+
+    private static class FlinkRowColumnVector implements RowColumnVector {
+
+      private ColumnVector[] fieldVectors;
+      private StructColumnVector structVector;
+
+      FlinkRowColumnVector(ColumnVector[] fieldVectors,
+                           StructColumnVector structVector) {
+        this.fieldVectors = fieldVectors;
+        this.structVector = structVector;
+      }
+
+      @Override
+      public ColumnarRowData getRow(int i) {
+        VectorizedColumnBatch vectorizedColumnBatch = new VectorizedColumnBatch(fieldVectors);
+        return new ColumnarRowData(vectorizedColumnBatch, i);
+      }
+
+      @Override
+      public boolean isNullAt(int i) {
+        return structVector.isNull[i];
+      }
+
+      public ColumnVector[] getFieldVectors() {
+        return fieldVectors;
+      }
+    }

Review comment:
       This class should be an inner class of the outside VectorizedFlinkOrcReaders , rather than an inner static class of StructConverter ? 




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r629853943



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/BaseDataIterator.java
##########
@@ -0,0 +1,32 @@
+/*
+ * 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.iceberg.flink.source;
+
+import org.apache.flink.table.data.RowData;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.FileIO;
+
+abstract class BaseDataIterator extends DataIterator<RowData> {

Review comment:
       What is the purpose of introducing this new class? `DataIterator` is already an abstract base class.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r635812141



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        FlinkRowColumnVector cv = (FlinkRowColumnVector) converter.convert(
+            new StructColumnVector(batch.size, batch.cols), batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException("Unsupported vectorized read for map type.");
+    }
+
+    @Override
+    public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) {
+      return (vector, batchSize, batchOffsetInFile) -> {
+        if (vector instanceof LongColumnVector) {
+          return new OrcNoHiveLongVector((LongColumnVector) vector);
+        } else if (vector instanceof DoubleColumnVector) {
+          return new OrcNoHiveDoubleVector((DoubleColumnVector) vector);
+        } else if (vector instanceof BytesColumnVector) {
+          return new OrcNoHiveBytesVector((BytesColumnVector) vector);
+        } else if (vector instanceof DecimalColumnVector) {
+          return new OrcNoHiveDecimalVector((DecimalColumnVector) vector);
+        } else if (vector instanceof TimestampColumnVector) {
+          return new OrcNoHiveTimestampVector((TimestampColumnVector) vector);
+        } else {
+          throw new UnsupportedOperationException(
+              "Unsupported vector: " + vector.getClass().getName());
+        }
+      };
+    }
+  }
+
+  private static class RowPositionColumnVector implements org.apache.flink.table.data.vector.LongColumnVector {
+    private final long batchOffsetInFile;
+
+    RowPositionColumnVector(long batchOffsetInFile) {
+      this.batchOffsetInFile = batchOffsetInFile;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return false;
+    }
+
+    @Override
+    public long getLong(int i) {
+      return batchOffsetInFile + i;
+    }
+  }
+
+  private static class StructConverter implements Converter {
+    private final Types.StructType structType;
+    private final List<Converter> fieldConverters;
+    private final Map<Integer, ?> idToConstant;
+
+    private StructConverter(Types.StructType structType, List<Converter> fieldConverters,
+                            Map<Integer, ?> idToConstant) {
+      this.structType = structType;
+      this.fieldConverters = fieldConverters;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector vector, int batchSize,
+                                long batchOffsetInFile) {
+      StructColumnVector structVector = (StructColumnVector) vector;
+      List<Types.NestedField> fields = structType.fields();
+      ColumnVector[] fieldVectors = new ColumnVector[fields.size()];
+      for (int pos = 0; pos < fields.size(); pos++) {
+        Types.NestedField field = fields.get(pos);
+        if (idToConstant.containsKey(field.fieldId())) {
+          fieldVectors[pos] = toConstantColumnVector(field.type(), idToConstant.get(field.fieldId()));
+        } else if (field.equals(MetadataColumns.ROW_POSITION)) {
+          fieldVectors[pos] = new RowPositionColumnVector(batchOffsetInFile);
+        } else {
+          fieldVectors[pos] = fieldConverters.get(pos)
+              .convert(structVector.fields[pos], batchSize, batchOffsetInFile);
+        }
+      }
+
+      return new FlinkRowColumnVector(fieldVectors, structVector);
+    }
+
+    private ColumnVector toConstantColumnVector(Type type, Object constant) {
+      Type.TypeID typeID = type.typeId();
+      switch (typeID) {
+        case INTEGER:
+        case DATE:
+        case TIME:
+          return ConstantColumnVectors.ints(constant);
+
+        case LONG:
+          return ConstantColumnVectors.longs(constant);
+
+        case BOOLEAN:
+          return ConstantColumnVectors.booleans(constant);
+
+        case DOUBLE:
+          return ConstantColumnVectors.doubles(constant);
+
+        case FLOAT:
+          return ConstantColumnVectors.floats(constant);
+
+        case DECIMAL:
+          return ConstantColumnVectors.decimals(constant);
+
+        case TIMESTAMP:
+          return ConstantColumnVectors.timestamps(constant);
+
+        case FIXED:
+        case UUID:
+        case BINARY:

Review comment:
       I added the judgment that constant is byte[],and add the UT for byte[] data type




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r628708570



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        StructConverter.FlinkRowColumnVector cv =
+            (StructConverter.FlinkRowColumnVector) converter.convert(new StructColumnVector(batch.size, batch.cols),
+                batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException();

Review comment:
       flink has `RowColumnVector` and `ArrayColumnVector`, but it doesn't  has `MapColumnVector ` ,  we will add the `Map` support when flink implements `MapColumnVector`




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r636652610



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkTableOptions.java
##########
@@ -40,4 +40,10 @@ private FlinkTableOptions() {
           .intType()
           .defaultValue(100)
           .withDescription("Sets max infer parallelism for source operator.");
+
+  public static final ConfigOption<Boolean> ENABLE_VECTORIZED_READ =
+      ConfigOptions.key("enable.vectorized.read")

Review comment:
       I update it,but I read  the fields in [TableProperties](https://github.com/apache/iceberg/blob/master/core/src/main/java/org/apache/iceberg/TableProperties.java), `-` and  `.` are both being used




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r629024922



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,381 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        StructConverter.FlinkRowColumnVector cv =
+            (StructConverter.FlinkRowColumnVector) converter.convert(new StructColumnVector(batch.size, batch.cols),
+                batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) {
+      return (vector, batchSize, batchOffsetInFile) -> {
+        if (vector instanceof LongColumnVector) {
+          return new OrcNoHiveLongVector((LongColumnVector) vector);
+        } else if (vector instanceof DoubleColumnVector) {
+          return new OrcNoHiveDoubleVector((DoubleColumnVector) vector);
+        } else if (vector instanceof BytesColumnVector) {
+          return new OrcNoHiveBytesVector((BytesColumnVector) vector);
+        } else if (vector instanceof DecimalColumnVector) {
+          return new OrcNoHiveDecimalVector((DecimalColumnVector) vector);
+        } else if (vector instanceof TimestampColumnVector) {
+          return new OrcNoHiveTimestampVector((TimestampColumnVector) vector);
+        } else {
+          throw new UnsupportedOperationException(
+              "Unsupported vector: " + vector.getClass().getName());
+        }
+      };
+    }
+  }
+
+  private static class StructConverter implements Converter {
+    private final Types.StructType structType;
+    private final List<Converter> fieldConverters;
+    private final Map<Integer, ?> idToConstant;
+
+    private StructConverter(Types.StructType structType, List<Converter> fieldConverters,
+                            Map<Integer, ?> idToConstant) {
+      this.structType = structType;
+      this.fieldConverters = fieldConverters;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector vector, int batchSize,
+                                long batchOffsetInFile) {
+      StructColumnVector structVector = (StructColumnVector) vector;
+      List<Types.NestedField> fields = structType.fields();
+      ColumnVector[] fieldVectors = new ColumnVector[fields.size()];
+      for (int pos = 0; pos < fields.size(); pos++) {
+        Types.NestedField field = fields.get(pos);
+        if (idToConstant.containsKey(field.fieldId())) {
+          fieldVectors[pos] = getConstantVectors(field.type(), idToConstant.get(field.fieldId()));
+        } else if (field.equals(MetadataColumns.ROW_POSITION)) {
+          fieldVectors[pos] = new org.apache.flink.table.data.vector.LongColumnVector() {
+
+            @Override
+            public boolean isNullAt(int i) {
+              return false;
+            }
+
+            @Override
+            public long getLong(int i) {
+              return batchOffsetInFile + i;
+            }
+          };

Review comment:
       Maybe we could just introduce a `ConstantColumnVectors` class, and the just move the following static classes inside that class and provide the method to access them, similar to the class `OrcValueReaders`.
   
   * IntOrcColumnVector
   * LongOrcColumnVector;
   * BooleanOrcColumnVector;
   * DoubleOrcColumnVector
   * FloatOrcColumnVector
   * DecimalOrcColumnVector
   * ...




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
openinx commented on pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#issuecomment-835991435


   Thanks for the work,  I will take a look today. 


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r635797697



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/ConstantColumnVectors.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import org.apache.flink.table.data.vector.BooleanColumnVector;
+import org.apache.flink.table.data.vector.BytesColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.DecimalColumnVector;
+import org.apache.flink.table.data.vector.DoubleColumnVector;
+import org.apache.flink.table.data.vector.FloatColumnVector;
+import org.apache.flink.table.data.vector.IntColumnVector;
+import org.apache.flink.table.data.vector.LongColumnVector;
+import org.apache.flink.table.data.vector.TimestampColumnVector;
+
+class ConstantColumnVectors {
+  private ConstantColumnVectors() {
+  }
+
+  static ColumnVector ints(Object constant) {
+    return new ConstantIntColumnVector(constant);
+  }
+
+  static ColumnVector longs(Object constant) {
+    return new ConstantLongColumnVongector(constant);
+  }
+
+  static ColumnVector booleans(Object constant) {
+    return new ConstantBooleanColumnVector(constant);
+  }
+
+  static ColumnVector doubles(Object constant) {
+    return new ConstantDoubleColumnVector(constant);
+  }
+
+  static ColumnVector floats(Object constant) {
+    return new ConstantFloatColumnVector(constant);
+  }
+
+  static ColumnVector decimals(Object constant) {
+    return new ConstantDecimalColumnVector(constant);
+  }
+
+  static ColumnVector timestamps(Object constant) {
+    return new ConstantTimestampColumnVector(constant);
+  }
+
+  static ColumnVector bytes(Object constant) {
+    return new ConstantBytesColumnVector(constant);
+  }
+
+  private static class ConstantIntColumnVector implements IntColumnVector {
+
+    private final Object constant;
+
+    private ConstantIntColumnVector(Object constant) {
+      this.constant = constant;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return constant == null;
+    }
+
+    @Override
+    public int getInt(int i) {
+      return (int) constant;
+    }
+  }
+
+  private static class ConstantLongColumnVongector implements LongColumnVector {
+
+    private final Object constant;
+
+    private ConstantLongColumnVongector(Object constant) {
+      this.constant = constant;
+    }
+
+    @Override
+    public long getLong(int i) {
+      return (long) constant;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return constant == null;
+    }
+  }
+
+  private static class ConstantBooleanColumnVector implements BooleanColumnVector {
+    private final Object constant;
+
+    private ConstantBooleanColumnVector(Object constant) {
+      this.constant = constant;
+    }
+
+    @Override
+    public boolean getBoolean(int i) {
+      return (boolean) constant;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return constant == null;
+    }
+  }
+
+  private static class ConstantDoubleColumnVector implements DoubleColumnVector {
+    private final Object constant;
+
+    private ConstantDoubleColumnVector(Object constant) {
+      this.constant = constant;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return constant == null;
+    }
+
+    @Override
+    public double getDouble(int i) {
+      return (double) constant;
+    }
+  }
+
+  private static class ConstantFloatColumnVector implements FloatColumnVector {
+    private final Object constant;
+
+    private ConstantFloatColumnVector(Object constant) {
+      this.constant = constant;
+    }
+
+    @Override
+    public float getFloat(int i) {
+      return (float) constant;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return constant == null;
+    }
+  }
+
+  private static class ConstantDecimalColumnVector implements DecimalColumnVector {
+    private final Object constant;
+
+    private ConstantDecimalColumnVector(Object constant) {
+      this.constant = constant;
+    }
+
+    @Override
+    public DecimalData getDecimal(int i, int precision, int scale) {
+      return (DecimalData) constant;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return constant == null;
+    }
+  }
+
+  private static class ConstantTimestampColumnVector implements TimestampColumnVector {
+    private final Object constant;
+
+    private ConstantTimestampColumnVector(Object constant) {
+      this.constant = constant;
+    }
+
+    @Override
+    public TimestampData getTimestamp(int i, int precision) {
+      return (TimestampData) constant;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return constant == null;
+    }
+  }
+
+  private static class ConstantBytesColumnVector implements BytesColumnVector {
+    private final Object constant;
+
+    private ConstantBytesColumnVector(Object constant) {
+      this.constant = constant;
+    }
+
+    @Override
+    public Bytes getBytes(int i) {
+      BinaryStringData str = (BinaryStringData) constant;
+      return new Bytes(str.toBytes(), 0, str.getSizeInBytes());

Review comment:
       I read the code of `str.toBytes()` ,and I found that it is still more complicated, and it have done a lot of operations. I have not found a suitable method to assign different `BinaryStringData` to the same byte[].




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r636654755



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/BatchRowDataIterator.java
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.iceberg.flink.source;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileContent;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.flink.data.RowDataUtil;
+import org.apache.iceberg.flink.data.vectorized.VectorizedFlinkOrcReaders;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PartitionUtil;
+import org.jetbrains.annotations.NotNull;
+
+class BatchRowDataIterator extends DataIterator<RowData> {
+
+  private final Schema tableSchema;
+  private final Schema projectedSchema;
+  private final String nameMapping;
+  private final boolean caseSensitive;
+  private final DataType[] dataTypes;
+
+  BatchRowDataIterator(CombinedScanTask task, FileIO io, EncryptionManager encryption, Schema tableSchema,
+                       Schema projectedSchema, String nameMapping, boolean caseSensitive, DataType[] dataTypes) {
+    super(task, io, encryption);
+    this.tableSchema = tableSchema;
+    this.projectedSchema = projectedSchema;
+    this.nameMapping = nameMapping;
+    this.caseSensitive = caseSensitive;
+    this.dataTypes = dataTypes;

Review comment:
       I have a few questions about passing in `DataType[]` here to check if they contain any unsupported types
   
   1. I believe the array is only for top level columns. what if a nested field is unsupported type?
   2. Since the `DataType[]` is only extracted from table or projected schema in `FlinkSource`,  should such validation be done once in the  constructor of `FlinkInputFormat`?
   3. Will vectorized read be able to support all valid types so that we don't need to do this check in the future?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r636680107



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/BatchRowDataIterator.java
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.iceberg.flink.source;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileContent;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.flink.data.RowDataUtil;
+import org.apache.iceberg.flink.data.vectorized.VectorizedFlinkOrcReaders;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PartitionUtil;
+import org.jetbrains.annotations.NotNull;
+
+class BatchRowDataIterator extends DataIterator<RowData> {
+
+  private final Schema tableSchema;
+  private final Schema projectedSchema;
+  private final String nameMapping;
+  private final boolean caseSensitive;
+  private final DataType[] dataTypes;
+
+  BatchRowDataIterator(CombinedScanTask task, FileIO io, EncryptionManager encryption, Schema tableSchema,
+                       Schema projectedSchema, String nameMapping, boolean caseSensitive, DataType[] dataTypes) {
+    super(task, io, encryption);
+    this.tableSchema = tableSchema;
+    this.projectedSchema = projectedSchema;
+    this.nameMapping = nameMapping;
+    this.caseSensitive = caseSensitive;
+    this.dataTypes = dataTypes;

Review comment:
       > 3\. Will vectorized read be able to support all valid types so that we don't need to do this check in the future
   
   I copy the code from [flink](https://github.com/apache/flink/blob/master/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveTableInputFormat.java#L202), I guess that  flink should not support all types for vectorized read now.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r631509052



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/ConstantColumnVectors.java
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import org.apache.flink.table.data.vector.BooleanColumnVector;
+import org.apache.flink.table.data.vector.BytesColumnVector;
+import org.apache.flink.table.data.vector.DecimalColumnVector;
+import org.apache.flink.table.data.vector.DoubleColumnVector;
+import org.apache.flink.table.data.vector.FloatColumnVector;
+import org.apache.flink.table.data.vector.IntColumnVector;
+import org.apache.flink.table.data.vector.LongColumnVector;
+import org.apache.flink.table.data.vector.TimestampColumnVector;
+
+class ConstantColumnVectors {
+  private ConstantColumnVectors() {
+  }
+
+  static IntOrcColumnVector ints(Object constant) {

Review comment:
       I update the return value to `ColumnVector `




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r631586450



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java
##########
@@ -91,11 +105,85 @@ public void configure(Configuration parameters) {
 
   @Override
   public void open(FlinkInputSplit split) {
+    boolean enableVectorizedRead = readableConfig.get(FlinkTableOptions.ENABLE_VECTORIZED_READ);
+
+    if (enableVectorizedRead) {
+      if (useOrcVectorizedRead(split.getTask())) {

Review comment:
       > We also don't need the separate method of `setDefaultIterator`
   
   At first, I wanted to use non-vectorized read when  judge failed of vectorized read, so I extracted this method, but later I think it inappropriate.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r631505420



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        StructConverter.FlinkRowColumnVector cv =
+            (StructConverter.FlinkRowColumnVector) converter.convert(new StructColumnVector(batch.size, batch.cols),
+                batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException();

Review comment:
       yes ,I added the msg.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangjun0x01 commented on pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#issuecomment-835140457


   @stevenzwu Thanks for your response, could you help me review it ? 
   @rdblue @openinx 


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #2566: Flink : vectorized read of orc format in flink

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#discussion_r635812141



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/vectorized/VectorizedFlinkOrcReaders.java
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.iceberg.flink.data.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveBytesVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDecimalVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveDoubleVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveLongVector;
+import org.apache.flink.orc.nohive.vector.OrcNoHiveTimestampVector;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.ColumnarArrayData;
+import org.apache.flink.table.data.ColumnarRowData;
+import org.apache.flink.table.data.vector.ArrayColumnVector;
+import org.apache.flink.table.data.vector.ColumnVector;
+import org.apache.flink.table.data.vector.RowColumnVector;
+import org.apache.flink.table.data.vector.VectorizedColumnBatch;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.orc.OrcBatchReader;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class VectorizedFlinkOrcReaders {
+  private VectorizedFlinkOrcReaders() {
+  }
+
+  public static OrcBatchReader<VectorizedColumnBatch> buildReader(Schema expectedSchema, TypeDescription fileSchema,
+                                                                  Map<Integer, ?> idToConstant) {
+    Converter converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant));
+
+    return new OrcBatchReader<VectorizedColumnBatch>() {
+      private long batchOffsetInFile;
+
+      @Override
+      public VectorizedColumnBatch read(VectorizedRowBatch batch) {
+        FlinkRowColumnVector cv = (FlinkRowColumnVector) converter.convert(
+            new StructColumnVector(batch.size, batch.cols), batch.size, batchOffsetInFile);
+
+        VectorizedColumnBatch columnarBatch = new VectorizedColumnBatch(cv.getFieldVectors());
+        columnarBatch.setNumRows(batch.size);
+        return columnarBatch;
+      }
+
+      @Override
+      public void setBatchContext(long batchOffsetInFile) {
+        this.batchOffsetInFile = batchOffsetInFile;
+      }
+    };
+  }
+
+  private interface Converter {
+    ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, int batchSize,
+                         long batchOffsetInFile);
+  }
+
+  private static class ReadBuilder extends OrcSchemaWithTypeVisitor<Converter> {
+    private final Map<Integer, ?> idToConstant;
+
+    private ReadBuilder(Map<Integer, ?> idToConstant) {
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public Converter record(Types.StructType iStruct, TypeDescription record, List<String> names,
+                            List<Converter> fields) {
+      return new StructConverter(iStruct, fields, idToConstant);
+    }
+
+    @Override
+    public Converter list(Types.ListType iList, TypeDescription array, Converter element) {
+      return new StructConverter.ArrayConverter(element);
+    }
+
+    @Override
+    public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) {
+      throw new UnsupportedOperationException("Unsupported vectorized read for map type.");
+    }
+
+    @Override
+    public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) {
+      return (vector, batchSize, batchOffsetInFile) -> {
+        if (vector instanceof LongColumnVector) {
+          return new OrcNoHiveLongVector((LongColumnVector) vector);
+        } else if (vector instanceof DoubleColumnVector) {
+          return new OrcNoHiveDoubleVector((DoubleColumnVector) vector);
+        } else if (vector instanceof BytesColumnVector) {
+          return new OrcNoHiveBytesVector((BytesColumnVector) vector);
+        } else if (vector instanceof DecimalColumnVector) {
+          return new OrcNoHiveDecimalVector((DecimalColumnVector) vector);
+        } else if (vector instanceof TimestampColumnVector) {
+          return new OrcNoHiveTimestampVector((TimestampColumnVector) vector);
+        } else {
+          throw new UnsupportedOperationException(
+              "Unsupported vector: " + vector.getClass().getName());
+        }
+      };
+    }
+  }
+
+  private static class RowPositionColumnVector implements org.apache.flink.table.data.vector.LongColumnVector {
+    private final long batchOffsetInFile;
+
+    RowPositionColumnVector(long batchOffsetInFile) {
+      this.batchOffsetInFile = batchOffsetInFile;
+    }
+
+    @Override
+    public boolean isNullAt(int i) {
+      return false;
+    }
+
+    @Override
+    public long getLong(int i) {
+      return batchOffsetInFile + i;
+    }
+  }
+
+  private static class StructConverter implements Converter {
+    private final Types.StructType structType;
+    private final List<Converter> fieldConverters;
+    private final Map<Integer, ?> idToConstant;
+
+    private StructConverter(Types.StructType structType, List<Converter> fieldConverters,
+                            Map<Integer, ?> idToConstant) {
+      this.structType = structType;
+      this.fieldConverters = fieldConverters;
+      this.idToConstant = idToConstant;
+    }
+
+    @Override
+    public ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector vector, int batchSize,
+                                long batchOffsetInFile) {
+      StructColumnVector structVector = (StructColumnVector) vector;
+      List<Types.NestedField> fields = structType.fields();
+      ColumnVector[] fieldVectors = new ColumnVector[fields.size()];
+      for (int pos = 0; pos < fields.size(); pos++) {
+        Types.NestedField field = fields.get(pos);
+        if (idToConstant.containsKey(field.fieldId())) {
+          fieldVectors[pos] = toConstantColumnVector(field.type(), idToConstant.get(field.fieldId()));
+        } else if (field.equals(MetadataColumns.ROW_POSITION)) {
+          fieldVectors[pos] = new RowPositionColumnVector(batchOffsetInFile);
+        } else {
+          fieldVectors[pos] = fieldConverters.get(pos)
+              .convert(structVector.fields[pos], batchSize, batchOffsetInFile);
+        }
+      }
+
+      return new FlinkRowColumnVector(fieldVectors, structVector);
+    }
+
+    private ColumnVector toConstantColumnVector(Type type, Object constant) {
+      Type.TypeID typeID = type.typeId();
+      switch (typeID) {
+        case INTEGER:
+        case DATE:
+        case TIME:
+          return ConstantColumnVectors.ints(constant);
+
+        case LONG:
+          return ConstantColumnVectors.longs(constant);
+
+        case BOOLEAN:
+          return ConstantColumnVectors.booleans(constant);
+
+        case DOUBLE:
+          return ConstantColumnVectors.doubles(constant);
+
+        case FLOAT:
+          return ConstantColumnVectors.floats(constant);
+
+        case DECIMAL:
+          return ConstantColumnVectors.decimals(constant);
+
+        case TIMESTAMP:
+          return ConstantColumnVectors.timestamps(constant);
+
+        case FIXED:
+        case UUID:
+        case BINARY:

Review comment:
       I added the judgment that constant is byte[],and add the UT for byte[] data type




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on pull request #2566: Flink : Vectorize read of orc format in flink

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on pull request #2566:
URL: https://github.com/apache/iceberg/pull/2566#issuecomment-835118192


   @zhangjun0x01 I am very interested in this PR and the general support of vectorized reader. While working on FLIP-27 Flink source [1], one of the open question is if we should extend from Flink's `FileSourceSplit` and `BulkFormat` for vectorized reader [2]. Now if we are making the effort in Iceberg to support vectorized readers for Orc and Parquet. Then I think we shouldn't need to extend from Flink's vectorized reader.
   
   cc @openinx 
   
   [1] the uber PR for FLIP-27 Iceberg source that are broken down into smaller PRs: https://github.com/apache/iceberg/pull/2105
   [2] PR for the split reader: https://github.com/apache/iceberg/pull/2305 
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org