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 2020/07/29 08:01:01 UTC

[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1254: Parquet: Add row position reader

chenjunjiedada commented on a change in pull request #1254:
URL: https://github.com/apache/iceberg/pull/1254#discussion_r461321137



##########
File path: spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.data;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.ParquetSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.ParquetReadOptions;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.ParquetFileWriter;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+@RunWith(Parameterized.class)
+public class TestSparkParquetReadMetadataColumns {
+  private static final Schema DATA_SCHEMA = new Schema(
+      required(100, "id", Types.LongType.get()),
+      required(101, "data", Types.StringType.get())
+  );
+
+  private static final Schema PROJECTION_SCHEMA = new Schema(
+      required(100, "id", Types.LongType.get()),
+      required(101, "data", Types.StringType.get()),
+      MetadataColumns.ROW_POSITION
+  );
+
+  private static final int NUM_ROWS = 1000;
+  private static final List<InternalRow> DATA_ROWS;
+  private static final List<InternalRow> EXPECTED_ROWS;
+  private static final int NUM_ROW_GROUPS = 10;
+  private static final int ROWS_PER_SPLIT = NUM_ROWS / NUM_ROW_GROUPS;
+
+  static {
+    DATA_ROWS = Lists.newArrayListWithCapacity(NUM_ROWS);
+    for (long i = 0; i < NUM_ROWS; i++) {
+      InternalRow row = new GenericInternalRow(DATA_SCHEMA.columns().size());
+      row.update(0, i);
+      row.update(1, UTF8String.fromString("str" + i));
+      DATA_ROWS.add(row);
+    }
+
+    EXPECTED_ROWS = Lists.newArrayListWithCapacity(NUM_ROWS);
+    for (long i = 0; i < NUM_ROWS; i++) {
+      InternalRow row = new GenericInternalRow(PROJECTION_SCHEMA.columns().size());
+      row.update(0, i);
+      row.update(1, UTF8String.fromString("str" + i));
+      row.update(2, i);
+      EXPECTED_ROWS.add(row);
+    }
+  }
+
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { false },
+        //new Object[] { true }
+    };
+  }
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private boolean vectorized;
+  private File testFile;
+
+  public TestSparkParquetReadMetadataColumns(boolean vectorized) {
+    this.vectorized = vectorized;
+  }
+
+  @Before
+  public void writeFile() throws IOException {
+    List<Path> fileSplits = new ArrayList<>();
+    StructType struct = SparkSchemaUtil.convert(DATA_SCHEMA);
+    Configuration conf = new Configuration();
+
+    testFile = temp.newFile();
+    Assert.assertTrue("Delete should succeed", testFile.delete());
+    ParquetFileWriter parquetFileWriter = new ParquetFileWriter(
+        conf,
+        ParquetSchemaUtil.convert(DATA_SCHEMA, "testSchema"),
+        new Path(testFile.getAbsolutePath())
+    );
+
+    parquetFileWriter.start();
+    for (int i = 0; i < NUM_ROW_GROUPS; i += 1) {
+      File split = temp.newFile();
+      Assert.assertTrue("Delete should succeed", split.delete());
+      fileSplits.add(new Path(split.getAbsolutePath()));
+      try (FileAppender<InternalRow> writer = Parquet.write(Files.localOutput(split))
+          .createWriterFunc(msgType -> SparkParquetWriters.buildWriter(struct, msgType))
+          .schema(DATA_SCHEMA)
+          .overwrite()
+          .build()) {
+        writer.addAll(DATA_ROWS.subList(i * ROWS_PER_SPLIT, (i + 1) * ROWS_PER_SPLIT));
+      }
+      parquetFileWriter.appendFile(HadoopInputFile.fromPath(new Path(split.getAbsolutePath()), conf));
+    }
+    parquetFileWriter
+        .end(ParquetFileWriter.mergeMetadataFiles(fileSplits, conf).getFileMetaData().getKeyValueMetaData());
+  }
+
+  @Test
+  public void testReadRowNumbers() throws IOException {
+    readAndValidate(null, null, null, EXPECTED_ROWS);
+  }
+
+  @Test
+  public void testReadRowNumbersWithFilter() throws IOException {
+    // current iceberg support row group filter.
+    for (int i = 0; i < NUM_ROW_GROUPS; i += 1) {
+      readAndValidate(Expressions.greaterThanOrEqual("id", i * ROWS_PER_SPLIT), null, null,
+          EXPECTED_ROWS.subList(i * ROWS_PER_SPLIT, 1000));
+    }
+  }
+
+  @Test
+  public void testReadRowNumbersWithSplits() throws IOException {
+    ParquetFileReader fileReader = new ParquetFileReader(
+        HadoopInputFile.fromPath(new Path(testFile.getAbsolutePath()), new Configuration()),
+        ParquetReadOptions.builder().build());
+    List<BlockMetaData> rowGroups = fileReader.getRowGroups();
+    for (int i = 0; i < NUM_ROW_GROUPS; i += 1) {
+      readAndValidate(null,
+          rowGroups.get(i).getColumns().get(1).getStartingPos(),
+          rowGroups.get(i).getCompressedSize(),
+          EXPECTED_ROWS.subList(i * ROWS_PER_SPLIT, (i + 1) * ROWS_PER_SPLIT));
+    }
+  }
+
+  private void readAndValidate(Expression filter, Long splitStart, Long splitLength, List<InternalRow> expected)
+      throws IOException {
+    Schema projectionWithoutMetadataFields = TypeUtil.selectNot(PROJECTION_SCHEMA, MetadataColumns.metadataFieldIds());

Review comment:
       Hmm, this is a copy-paste error. let me remove this at first. At least, I don't use any related logic right now.

##########
File path: spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.data;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.ParquetSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.ParquetReadOptions;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.ParquetFileWriter;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+@RunWith(Parameterized.class)
+public class TestSparkParquetReadMetadataColumns {
+  private static final Schema DATA_SCHEMA = new Schema(
+      required(100, "id", Types.LongType.get()),
+      required(101, "data", Types.StringType.get())
+  );
+
+  private static final Schema PROJECTION_SCHEMA = new Schema(
+      required(100, "id", Types.LongType.get()),
+      required(101, "data", Types.StringType.get()),
+      MetadataColumns.ROW_POSITION
+  );
+
+  private static final int NUM_ROWS = 1000;
+  private static final List<InternalRow> DATA_ROWS;
+  private static final List<InternalRow> EXPECTED_ROWS;
+  private static final int NUM_ROW_GROUPS = 10;
+  private static final int ROWS_PER_SPLIT = NUM_ROWS / NUM_ROW_GROUPS;
+
+  static {
+    DATA_ROWS = Lists.newArrayListWithCapacity(NUM_ROWS);
+    for (long i = 0; i < NUM_ROWS; i++) {
+      InternalRow row = new GenericInternalRow(DATA_SCHEMA.columns().size());
+      row.update(0, i);

Review comment:
       Nice catch! The split start should use the first column's starting pos other than the second one. I updated the UT to cover both `row.update(0, i)` and `row.update(0, 2*i)`.

##########
File path: parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java
##########
@@ -157,6 +164,23 @@ ParquetFileReader reader() {
     return shouldSkip;
   }
 
+  private Map<Long, Long> generateRowGroupsStartRowPos() {

Review comment:
       The existing loop of row groups is based on the row group set that had been filtered with `options`.  So we need to read the Parquet file without any filter to get each starting row position of row group.

##########
File path: parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java
##########
@@ -157,6 +164,23 @@ ParquetFileReader reader() {
     return shouldSkip;
   }
 
+  private Map<Long, Long> generateRowGroupsStartRowPos() {
+    ParquetFileReader fileReader = newReader(this.file, ParquetReadOptions.builder().build());
+    Map<Long, Long> offsetToStartRowPosMap = new HashMap<>();
+    long curRowCount = 0;
+    for (int i = 0; i < fileReader.getRowGroups().size(); i += 1) {
+      BlockMetaData meta = fileReader.getRowGroups().get(i);
+      offsetToStartRowPosMap.put(meta.getStartingPos(), curRowCount);
+      curRowCount += meta.getRowCount();
+    }
+
+    return offsetToStartRowPosMap;
+  }
+
+  long[] getRowGroupsStartRowPos() {

Review comment:
       `startPositions` may confuse with `rowGroup.startingPosition`, how about `startRowPosititions`?

##########
File path: parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReader.java
##########
@@ -30,4 +30,6 @@
   List<TripleIterator<?>> columns();
 
   void setPageSource(PageReadStore pageStore);
+
+  default void setRowOffsetForRowGroup(long position) {}

Review comment:
       I can change to that. Just one thing that do we mind to change the function signature in the public API?

##########
File path: spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.data;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.ParquetSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.ParquetReadOptions;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.ParquetFileWriter;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+@RunWith(Parameterized.class)
+public class TestSparkParquetReadMetadataColumns {
+  private static final Schema DATA_SCHEMA = new Schema(
+      required(100, "id", Types.LongType.get()),
+      required(101, "data", Types.StringType.get())
+  );
+
+  private static final Schema PROJECTION_SCHEMA = new Schema(
+      required(100, "id", Types.LongType.get()),
+      required(101, "data", Types.StringType.get()),
+      MetadataColumns.ROW_POSITION
+  );
+
+  private static final int NUM_ROWS = 1000;
+  private static final List<InternalRow> DATA_ROWS;
+  private static final List<InternalRow> EXPECTED_ROWS;
+  private static final int NUM_ROW_GROUPS = 10;
+  private static final int ROWS_PER_SPLIT = NUM_ROWS / NUM_ROW_GROUPS;
+
+  static {
+    DATA_ROWS = Lists.newArrayListWithCapacity(NUM_ROWS);
+    for (long i = 0; i < NUM_ROWS; i += 1) {
+      InternalRow row = new GenericInternalRow(DATA_SCHEMA.columns().size());
+      if (i >= 500) {
+        row.update(0, 2 * i);
+      } else {
+        row.update(0, i);
+      }
+      row.update(1, UTF8String.fromString("str" + i));
+      DATA_ROWS.add(row);
+    }
+
+    EXPECTED_ROWS = Lists.newArrayListWithCapacity(NUM_ROWS);
+    for (long i = 0; i < NUM_ROWS; i += 1) {
+      InternalRow row = new GenericInternalRow(PROJECTION_SCHEMA.columns().size());
+      if (i >= 500) {
+        row.update(0, 2 * i);
+      } else {
+        row.update(0, i);
+      }
+      row.update(1, UTF8String.fromString("str" + i));
+      row.update(2, i);
+      EXPECTED_ROWS.add(row);
+    }
+  }
+
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { false },
+        //new Object[] { true }
+    };
+  }
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private boolean vectorized;
+  private File testFile;
+
+  public TestSparkParquetReadMetadataColumns(boolean vectorized) {
+    this.vectorized = vectorized;
+  }
+
+  @Before
+  public void writeFile() throws IOException {
+    List<Path> fileSplits = new ArrayList<>();
+    StructType struct = SparkSchemaUtil.convert(DATA_SCHEMA);
+    Configuration conf = new Configuration();
+
+    testFile = temp.newFile();
+    Assert.assertTrue("Delete should succeed", testFile.delete());
+    ParquetFileWriter parquetFileWriter = new ParquetFileWriter(
+        conf,
+        ParquetSchemaUtil.convert(DATA_SCHEMA, "testSchema"),
+        new Path(testFile.getAbsolutePath())
+    );
+
+    parquetFileWriter.start();
+    for (int i = 0; i < NUM_ROW_GROUPS; i += 1) {
+      File split = temp.newFile();
+      Assert.assertTrue("Delete should succeed", split.delete());
+      fileSplits.add(new Path(split.getAbsolutePath()));
+      try (FileAppender<InternalRow> writer = Parquet.write(Files.localOutput(split))
+          .createWriterFunc(msgType -> SparkParquetWriters.buildWriter(struct, msgType))
+          .schema(DATA_SCHEMA)
+          .overwrite()
+          .build()) {
+        writer.addAll(DATA_ROWS.subList(i * ROWS_PER_SPLIT, (i + 1) * ROWS_PER_SPLIT));
+      }
+      parquetFileWriter.appendFile(HadoopInputFile.fromPath(new Path(split.getAbsolutePath()), conf));
+    }
+    parquetFileWriter
+        .end(ParquetFileWriter.mergeMetadataFiles(fileSplits, conf).getFileMetaData().getKeyValueMetaData());
+  }
+
+  @Test
+  public void testReadRowNumbers() throws IOException {
+    readAndValidate(null, null, null, EXPECTED_ROWS);
+  }
+
+  @Test
+  public void testReadRowNumbersWithFilter() throws IOException {
+    // current iceberg supports row group filter.
+    for (int i = 1; i < 5; i += 1) {
+      readAndValidate(
+          Expressions.and(Expressions.lessThan("id", 500),
+              Expressions.greaterThanOrEqual("id", i * ROWS_PER_SPLIT)),
+          null,
+          null,
+          EXPECTED_ROWS.subList(i * ROWS_PER_SPLIT, 500));
+    }
+  }
+
+  @Test
+  public void testReadRowNumbersWithSplits() throws IOException {
+    ParquetFileReader fileReader = new ParquetFileReader(
+        HadoopInputFile.fromPath(new Path(testFile.getAbsolutePath()), new Configuration()),
+        ParquetReadOptions.builder().build());
+    List<BlockMetaData> rowGroups = fileReader.getRowGroups();
+    for (int i = 0; i < NUM_ROW_GROUPS; i += 1) {
+      readAndValidate(null,
+          rowGroups.get(i).getColumns().get(0).getStartingPos(),
+          rowGroups.get(i).getCompressedSize(),
+          EXPECTED_ROWS.subList(i * ROWS_PER_SPLIT, (i + 1) * ROWS_PER_SPLIT));
+    }
+  }
+
+  private void readAndValidate(Expression filter, Long splitStart, Long splitLength, List<InternalRow> expected)
+      throws IOException {
+    CloseableIterable<InternalRow> reader = null;
+    try {
+      Parquet.ReadBuilder builder = Parquet.read(Files.localInput(testFile))
+          .project(PROJECTION_SCHEMA);
+
+      if (vectorized) {
+        builder.createBatchedReaderFunc(fileSchema -> VectorizedSparkParquetReaders.buildReader(PROJECTION_SCHEMA,
+            fileSchema, NullCheckingForGet.NULL_CHECKING_ENABLED));
+      } else {
+        builder = builder.createReaderFunc(msgType -> SparkParquetReaders.buildReader(PROJECTION_SCHEMA, msgType));
+      }
+
+      if (filter != null) {
+        builder = builder.filter(filter);
+      }
+
+      if (splitStart != null && splitLength != null) {
+        builder = builder.split(splitStart, splitLength);
+      }
+
+      reader = builder.build();
+
+      final Iterator<InternalRow> actualRows = reader.iterator();
+
+      for (InternalRow internalRow : expected) {
+        Assert.assertTrue("Should have expected number of rows", actualRows.hasNext());
+        TestHelpers.assertEquals(PROJECTION_SCHEMA, internalRow, actualRows.next());
+      }
+      Assert.assertFalse("Should not have extra rows", actualRows.hasNext());
+    } finally {
+      if (reader != null) {
+        reader.close();

Review comment:
       Done.

##########
File path: parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReaders.java
##########
@@ -137,6 +137,41 @@ public void setPageSource(PageReadStore pageStore) {
     }
   }
 
+  static class PositionReader implements ParquetValueReader<Long> {
+    private long rowOffsetInCurrentRowGroup = -1;
+    private long rowGroupRowOffsetInFile;

Review comment:
       Done.

##########
File path: parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReaders.java
##########
@@ -137,6 +137,41 @@ public void setPageSource(PageReadStore pageStore) {
     }
   }
 
+  static class PositionReader implements ParquetValueReader<Long> {
+    private long rowOffsetInCurrentRowGroup = -1;
+    private long rowGroupRowOffsetInFile;
+
+    @Override
+    public Long read(Long reuse) {
+      rowOffsetInCurrentRowGroup = rowOffsetInCurrentRowGroup + 1;
+      return rowGroupRowOffsetInFile + rowOffsetInCurrentRowGroup;
+    }
+
+    @Override
+    public TripleIterator<?> column() {
+      return NullReader.NULL_COLUMN;
+    }
+
+    @Override
+    public List<TripleIterator<?>> columns() {
+      return NullReader.COLUMNS;
+    }
+
+    @Override
+    public void setPageSource(PageReadStore pageStore) {
+    }
+
+    @Override
+    public void setRowOffsetForRowGroup(long rowGroupStartPos) {
+      this.rowGroupRowOffsetInFile = rowGroupStartPos;
+      this.rowOffsetInCurrentRowGroup = -1;
+    }
+  }
+
+  public static ParquetValueReader<Long> position() {
+    return new PositionReader();
+  }

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.

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