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:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to