parthchandra commented on code in PR #13786:
URL: https://github.com/apache/iceberg/pull/13786#discussion_r2487266122


##########
parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java:
##########
@@ -1289,6 +1290,20 @@ public ReadBuilder setCustomType(int fieldId, Class<? 
extends StructLike> struct
       throw new UnsupportedOperationException("Custom types are not yet 
supported");
     }
 
+    /**
+     * @deprecated Use {@link #set(String, String)} with 
"read.parquet.vectorized-reader.factory" =
+     *     "comet" instead
+     */
+    @Deprecated

Review Comment:
   Sorry, I got confused between all these revisions and thought this was 
already part of the API! Updated the comment.



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometTypeUtils.java:
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.spark.parquet;
+
+import java.util.Map;
+import org.apache.comet.parquet.ParquetColumnSpec;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.schema.LogicalTypeAnnotation;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.apache.parquet.schema.Types;
+
+public class CometTypeUtils {
+
+  private CometTypeUtils() {}
+
+  public static ParquetColumnSpec 
descriptorToParquetColumnSpec(ColumnDescriptor descriptor) {
+
+    String[] path = descriptor.getPath();
+    PrimitiveType primitiveType = descriptor.getPrimitiveType();
+    String physicalType = primitiveType.getPrimitiveTypeName().name();
+
+    int typeLength =
+        primitiveType.getPrimitiveTypeName() == 
PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY
+            ? primitiveType.getTypeLength()
+            : 0;
+
+    boolean isRepeated = primitiveType.getRepetition() == 
Type.Repetition.REPEATED;
+
+    String logicalTypeName = null;
+    Map<String, String> logicalTypeParams = Maps.newHashMap();
+    LogicalTypeAnnotation logicalType = 
primitiveType.getLogicalTypeAnnotation();
+
+    if (logicalType != null) {
+      logicalTypeName = logicalType.getClass().getSimpleName();
+
+      // Handle specific logical types
+      if (logicalType instanceof 
LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) {
+        LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimal =
+            (LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) logicalType;
+        logicalTypeParams.put("precision", 
String.valueOf(decimal.getPrecision()));
+        logicalTypeParams.put("scale", String.valueOf(decimal.getScale()));
+      } else if (logicalType instanceof 
LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) {
+        LogicalTypeAnnotation.TimestampLogicalTypeAnnotation timestamp =
+            (LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) logicalType;
+        logicalTypeParams.put("isAdjustedToUTC", 
String.valueOf(timestamp.isAdjustedToUTC()));
+        logicalTypeParams.put("unit", timestamp.getUnit().name());
+      } else if (logicalType instanceof 
LogicalTypeAnnotation.TimeLogicalTypeAnnotation) {
+        LogicalTypeAnnotation.TimeLogicalTypeAnnotation time =
+            (LogicalTypeAnnotation.TimeLogicalTypeAnnotation) logicalType;
+        logicalTypeParams.put("isAdjustedToUTC", 
String.valueOf(time.isAdjustedToUTC()));
+        logicalTypeParams.put("unit", time.getUnit().name());
+      } else if (logicalType instanceof 
LogicalTypeAnnotation.IntLogicalTypeAnnotation) {
+        LogicalTypeAnnotation.IntLogicalTypeAnnotation intType =
+            (LogicalTypeAnnotation.IntLogicalTypeAnnotation) logicalType;
+        logicalTypeParams.put("isSigned", String.valueOf(intType.isSigned()));
+        logicalTypeParams.put("bitWidth", 
String.valueOf(intType.getBitWidth()));
+      }
+    }
+
+    int id = -1;
+    Type type = descriptor.getPrimitiveType();
+    if (type != null && type.getId() != null) {
+      id = type.getId().intValue();
+    }
+
+    return new ParquetColumnSpec(
+        id,
+        path,
+        physicalType,
+        typeLength,
+        isRepeated,
+        descriptor.getMaxDefinitionLevel(),
+        descriptor.getMaxRepetitionLevel(),
+        logicalTypeName,
+        logicalTypeParams);
+  }
+
+  public static ColumnDescriptor buildColumnDescriptor(ParquetColumnSpec 
columnSpec) {
+    PrimitiveType.PrimitiveTypeName primType =
+        PrimitiveType.PrimitiveTypeName.valueOf(columnSpec.getPhysicalType());
+
+    Type.Repetition repetition;
+    if (columnSpec.getMaxRepetitionLevel() > 0) {
+      repetition = Type.Repetition.REPEATED;
+    } else if (columnSpec.getMaxDefinitionLevel() > 0) {
+      repetition = Type.Repetition.OPTIONAL;
+    } else {
+      repetition = Type.Repetition.REQUIRED;
+    }
+
+    String name = columnSpec.getPath()[columnSpec.getPath().length - 1];
+    // Reconstruct the logical type from parameters
+    LogicalTypeAnnotation logicalType = null;
+    if (columnSpec.getLogicalTypeName() != null) {
+      logicalType =
+          reconstructLogicalType(
+              columnSpec.getLogicalTypeName(), 
columnSpec.getLogicalTypeParams());
+    }
+
+    PrimitiveType primitiveType;
+    if (primType == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) {
+      primitiveType =
+          org.apache.parquet.schema.Types.primitive(primType, repetition)

Review Comment:
   Done



##########
parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java:
##########
@@ -1352,6 +1367,32 @@ public <D> CloseableIterable<D> build() {
         }
 
         if (batchedReaderFunc != null) {
+          // Try to load custom vectorized reader factory from properties
+          String readerName = 
properties.get("read.parquet.vectorized-reader.factory");

Review Comment:
   Done



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometTypeUtils.java:
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.spark.parquet;
+
+import java.util.Map;
+import org.apache.comet.parquet.ParquetColumnSpec;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.schema.LogicalTypeAnnotation;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.apache.parquet.schema.Types;
+
+public class CometTypeUtils {
+
+  private CometTypeUtils() {}
+
+  public static ParquetColumnSpec 
descriptorToParquetColumnSpec(ColumnDescriptor descriptor) {
+
+    String[] path = descriptor.getPath();
+    PrimitiveType primitiveType = descriptor.getPrimitiveType();
+    String physicalType = primitiveType.getPrimitiveTypeName().name();
+
+    int typeLength =
+        primitiveType.getPrimitiveTypeName() == 
PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY
+            ? primitiveType.getTypeLength()
+            : 0;
+
+    boolean isRepeated = primitiveType.getRepetition() == 
Type.Repetition.REPEATED;
+
+    String logicalTypeName = null;
+    Map<String, String> logicalTypeParams = Maps.newHashMap();
+    LogicalTypeAnnotation logicalType = 
primitiveType.getLogicalTypeAnnotation();
+
+    if (logicalType != null) {
+      logicalTypeName = logicalType.getClass().getSimpleName();
+
+      // Handle specific logical types
+      if (logicalType instanceof 
LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) {
+        LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimal =
+            (LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) logicalType;
+        logicalTypeParams.put("precision", 
String.valueOf(decimal.getPrecision()));
+        logicalTypeParams.put("scale", String.valueOf(decimal.getScale()));
+      } else if (logicalType instanceof 
LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) {
+        LogicalTypeAnnotation.TimestampLogicalTypeAnnotation timestamp =
+            (LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) logicalType;
+        logicalTypeParams.put("isAdjustedToUTC", 
String.valueOf(timestamp.isAdjustedToUTC()));
+        logicalTypeParams.put("unit", timestamp.getUnit().name());
+      } else if (logicalType instanceof 
LogicalTypeAnnotation.TimeLogicalTypeAnnotation) {
+        LogicalTypeAnnotation.TimeLogicalTypeAnnotation time =
+            (LogicalTypeAnnotation.TimeLogicalTypeAnnotation) logicalType;
+        logicalTypeParams.put("isAdjustedToUTC", 
String.valueOf(time.isAdjustedToUTC()));
+        logicalTypeParams.put("unit", time.getUnit().name());
+      } else if (logicalType instanceof 
LogicalTypeAnnotation.IntLogicalTypeAnnotation) {
+        LogicalTypeAnnotation.IntLogicalTypeAnnotation intType =
+            (LogicalTypeAnnotation.IntLogicalTypeAnnotation) logicalType;
+        logicalTypeParams.put("isSigned", String.valueOf(intType.isSigned()));
+        logicalTypeParams.put("bitWidth", 
String.valueOf(intType.getBitWidth()));
+      }
+    }
+
+    int id = -1;
+    Type type = descriptor.getPrimitiveType();
+    if (type != null && type.getId() != null) {
+      id = type.getId().intValue();
+    }
+
+    return new ParquetColumnSpec(
+        id,
+        path,
+        physicalType,
+        typeLength,
+        isRepeated,
+        descriptor.getMaxDefinitionLevel(),
+        descriptor.getMaxRepetitionLevel(),
+        logicalTypeName,
+        logicalTypeParams);
+  }
+
+  public static ColumnDescriptor buildColumnDescriptor(ParquetColumnSpec 
columnSpec) {
+    PrimitiveType.PrimitiveTypeName primType =
+        PrimitiveType.PrimitiveTypeName.valueOf(columnSpec.getPhysicalType());
+
+    Type.Repetition repetition;
+    if (columnSpec.getMaxRepetitionLevel() > 0) {
+      repetition = Type.Repetition.REPEATED;
+    } else if (columnSpec.getMaxDefinitionLevel() > 0) {
+      repetition = Type.Repetition.OPTIONAL;
+    } else {
+      repetition = Type.Repetition.REQUIRED;
+    }
+
+    String name = columnSpec.getPath()[columnSpec.getPath().length - 1];
+    // Reconstruct the logical type from parameters
+    LogicalTypeAnnotation logicalType = null;
+    if (columnSpec.getLogicalTypeName() != null) {
+      logicalType =
+          reconstructLogicalType(
+              columnSpec.getLogicalTypeName(), 
columnSpec.getLogicalTypeParams());
+    }
+
+    PrimitiveType primitiveType;
+    if (primType == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) {
+      primitiveType =
+          org.apache.parquet.schema.Types.primitive(primType, repetition)
+              .length(columnSpec.getTypeLength())
+              .as(logicalType)
+              .id(columnSpec.getFieldId())
+              .named(name);
+    } else {
+      primitiveType =
+          Types.primitive(primType, repetition)
+              .as(logicalType)
+              .id(columnSpec.getFieldId())
+              .named(name);
+    }
+
+    return new ColumnDescriptor(
+        columnSpec.getPath(),
+        primitiveType,
+        columnSpec.getMaxRepetitionLevel(),
+        columnSpec.getMaxDefinitionLevel());
+  }
+
+  private static LogicalTypeAnnotation reconstructLogicalType(
+      String logicalTypeName, java.util.Map<String, String> params) {

Review Comment:
   Done



##########
spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java:
##########
@@ -0,0 +1,257 @@
+/*
+ * 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.spark.parquet;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.function.Function;
+import org.apache.comet.parquet.FileReader;
+import org.apache.comet.parquet.ParquetColumnSpec;
+import org.apache.comet.parquet.ReadOptions;
+import org.apache.comet.parquet.RowGroupReader;
+import org.apache.comet.parquet.WrappedInputFile;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMapping;
+import org.apache.iceberg.parquet.ReadConf;
+import org.apache.iceberg.parquet.VectorizedReader;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.util.ByteBuffers;
+import org.apache.parquet.ParquetReadOptions;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.schema.MessageType;
+
+public class CometVectorizedParquetReader<T> extends CloseableGroup
+    implements CloseableIterable<T> {
+  private final InputFile input;
+  private final ParquetReadOptions options;
+  private final Schema expectedSchema;
+  private final Function<MessageType, VectorizedReader<?>> batchReaderFunc;
+  private final Expression filter;
+  private final boolean reuseContainers;
+  private final boolean caseSensitive;
+  private final int batchSize;
+  private final NameMapping nameMapping;
+  private final Map<String, String> properties;
+  private Long start = null;
+  private Long length = null;
+  private ByteBuffer fileEncryptionKey = null;
+  private ByteBuffer fileAADPrefix = null;
+
+  public CometVectorizedParquetReader(
+      InputFile input,
+      Schema expectedSchema,
+      ParquetReadOptions options,
+      Function<MessageType, VectorizedReader<?>> readerFunc,
+      NameMapping nameMapping,
+      Expression filter,
+      boolean reuseContainers,
+      boolean caseSensitive,
+      int maxRecordsPerBatch,
+      Map<String, String> properties,
+      Long start,
+      Long length,
+      ByteBuffer fileEncryptionKey,
+      ByteBuffer fileAADPrefix) {
+    this.input = input;
+    this.expectedSchema = expectedSchema;
+    this.options = options;
+    this.batchReaderFunc = readerFunc;
+    // replace alwaysTrue with null to avoid extra work evaluating a trivial 
filter
+    this.filter = filter == Expressions.alwaysTrue() ? null : filter;
+    this.reuseContainers = reuseContainers;
+    this.caseSensitive = caseSensitive;
+    this.batchSize = maxRecordsPerBatch;
+    this.nameMapping = nameMapping;
+    this.properties = properties;
+    this.start = start;
+    this.length = length;
+    this.fileEncryptionKey = fileEncryptionKey;
+    this.fileAADPrefix = fileAADPrefix;
+  }
+
+  private ReadConf conf = null;
+
+  private ReadConf init() {
+    if (conf == null) {
+      ReadConf readConf =
+          new ReadConf(
+              input,
+              options,
+              expectedSchema,
+              filter,
+              null,
+              batchReaderFunc,
+              nameMapping,
+              reuseContainers,
+              caseSensitive,
+              batchSize);
+      this.conf = readConf.copy();
+      return readConf;
+    }
+    return conf;
+  }
+
+  @Override
+  public CloseableIterator<T> iterator() {
+    FileIterator<T> iter =
+        new FileIterator<>(init(), properties, start, length, 
fileEncryptionKey, fileAADPrefix);
+    addCloseable(iter);
+    return iter;
+  }
+
+  private static class FileIterator<T> implements CloseableIterator<T> {
+    // private final ParquetFileReader reader;

Review Comment:
   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.

To unsubscribe, e-mail: [email protected]

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