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


##########
parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java:
##########
@@ -1352,16 +1358,35 @@ public <D> CloseableIterable<D> build() {
         }
 
         if (batchedReaderFunc != null) {
-          return new VectorizedParquetReader<>(
-              file,
-              schema,
-              options,
-              batchedReaderFunc,
-              mapping,
-              filter,
-              reuseContainers,
-              caseSensitive,
-              maxRecordsPerBatch);
+          if (isComet) {
+            LOG.info("Comet enabled");

Review Comment:
   changed



##########
parquet/src/main/java/org/apache/iceberg/parquet/CometVectorizedParquetReader.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.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.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(

Review Comment:
   I created a Comet specific `CometReadConf` that does not have a 
`ParquetFileReader` but uses the Comet `FileReader` instead and changed 
`CometVectorizedParquetReader` to reuse the FileReader created in 
`CometReadConf`.
   Unfortunately, `CometVectorizedParquetReader` needs the file schema and 
`RowGroups`  from `CometReadConf` and using Comet's `FileReader` to get this 
information in Iceberg violates the requirement to not use any Comet API that 
has a Parquet class in the signature (doing so breaks the implementation when 
Parquet is  shaded by Iceberg). Also, the methods to get schema and row groups 
are also (rightly) made package-private in Comet so the change won't even 
compile. 
   I think for the moment this will have to do. We will revisit this 
integration once again as we move to a more native implementation in Comet.



##########
parquet/src/main/java/org/apache/iceberg/parquet/CometVectorizedParquetReader.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.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.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;
+    private final boolean[] shouldSkip;
+    private final VectorizedReader<T> model;
+    private final long totalValues;
+    private final int batchSize;
+    private final List<Map<ColumnPath, ColumnChunkMetaData>> 
columnChunkMetadata;
+    private final boolean reuseContainers;
+    private int nextRowGroup = 0;
+    private long nextRowGroupStart = 0;
+    private long valuesRead = 0;
+    private T last = null;
+    private final FileReader cometReader;
+
+    FileIterator(
+        ReadConf conf,
+        Map<String, String> properties,
+        Long start,
+        Long length,
+        ByteBuffer fileEncryptionKey,
+        ByteBuffer fileAADPrefix) {
+      this.shouldSkip = conf.shouldSkip();
+      this.totalValues = conf.totalValues();
+      this.reuseContainers = conf.reuseContainers();
+      this.model = conf.vectorizedModel();
+      this.batchSize = conf.batchSize();
+      this.model.setBatchSize(this.batchSize);
+      this.columnChunkMetadata = conf.columnChunkMetadataForRowGroups();
+      this.cometReader =
+          newCometReader(
+              conf.file(),
+              conf.projection(),
+              properties,
+              start,
+              length,
+              fileEncryptionKey,
+              fileAADPrefix);
+    }
+
+    private FileReader newCometReader(
+        InputFile file,
+        MessageType projection,
+        Map<String, String> properties,
+        Long start,
+        Long length,
+        ByteBuffer fileEncryptionKey,
+        ByteBuffer fileAADPrefix) {
+      try {
+        ReadOptions cometOptions = ReadOptions.builder(new 
Configuration()).build();
+
+        FileReader fileReader =
+            new FileReader(
+                new WrappedInputFile(file),
+                cometOptions,
+                properties,
+                start,
+                length,
+                ByteBuffers.toByteArray(fileEncryptionKey),
+                ByteBuffers.toByteArray(fileAADPrefix));
+
+        List<ColumnDescriptor> columnDescriptors = projection.getColumns();
+
+        List<ParquetColumnSpec> specs = Lists.newArrayList();
+
+        for (ColumnDescriptor descriptor : columnDescriptors) {
+          ParquetColumnSpec spec = 
CometTypeUtils.descriptorToParquetColumnSpec(descriptor);
+          specs.add(spec);
+        }
+
+        fileReader.setRequestedSchemaFromSpecs(specs);
+        return fileReader;
+      } catch (IOException e) {
+        throw new UncheckedIOException("Failed to open Parquet file: " + 
file.location(), e);
+      }
+    }
+
+    @Override
+    public boolean hasNext() {
+      return valuesRead < totalValues;
+    }
+
+    @Override
+    public T next() {
+      if (!hasNext()) {
+        throw new NoSuchElementException();
+      }
+      if (valuesRead >= nextRowGroupStart) {
+        advance();
+      }
+
+      // batchSize is an integer, so casting to integer is safe
+      int numValuesToRead = (int) Math.min(nextRowGroupStart - valuesRead, 
batchSize);
+      if (reuseContainers) {
+        this.last = model.read(last, numValuesToRead);
+      } else {
+        this.last = model.read(null, numValuesToRead);
+      }
+      valuesRead += numValuesToRead;
+
+      return last;
+    }
+
+    private void advance() {
+      while (shouldSkip[nextRowGroup]) {
+        nextRowGroup += 1;
+        cometReader.skipNextRowGroup();
+      }
+      RowGroupReader pages;
+      try {
+        pages = cometReader.readNextRowGroup();

Review Comment:
   In this case, `advance` is called only by `next` which is implementing the 
`iterator` interface and which explicitly checks for `hasNext` so 
`readNextRowGroup` always returns a not null value.
   This could be done better, but I'll leave this as it is because - 
    i) it is not really clear what the behavior will be if `advance` silently 
does not do anything. 
    ii) the code (and behavior) will be consistent with the implementation in 
`VectorizedParquetReader`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

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


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

Reply via email to