boyuanzz commented on a change in pull request #12223:
URL: https://github.com/apache/beam/pull/12223#discussion_r479523764



##########
File path: 
sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java
##########
@@ -230,14 +279,255 @@ public ReadFiles withAvroDataModel(GenericData model) {
       return toBuilder().setAvroDataModel(model).build();
     }
 
+    public ReadFiles withSplit() {
+      return toBuilder().setSplittable(true).build();
+    }
+
     @Override
     public PCollection<GenericRecord> expand(PCollection<FileIO.ReadableFile> 
input) {
       checkNotNull(getSchema(), "Schema can not be null");
+      if (isSplittable()) {
+        return input
+            .apply(ParDo.of(new SplitReadFn(getAvroDataModel())))
+            .setCoder(AvroCoder.of(getSchema()));
+      }
       return input
           .apply(ParDo.of(new ReadFn(getAvroDataModel())))
           .setCoder(AvroCoder.of(getSchema()));
     }
 
+    @DoFn.BoundedPerElement
+    static class SplitReadFn extends DoFn<FileIO.ReadableFile, GenericRecord> {
+      private Class<? extends GenericData> modelClass;
+      private static final Logger LOG = 
LoggerFactory.getLogger(SplitReadFn.class);
+      // Default initial splitting the file into blocks of 64MB. Unit of 
SPLIT_LIMIT is byte.
+      private static final long SPLIT_LIMIT = 64000000;
+
+      SplitReadFn(GenericData model) {
+
+        this.modelClass = model != null ? model.getClass() : null;
+      }
+
+      ParquetFileReader getParquetFileReader(FileIO.ReadableFile file) throws 
Exception {
+        ParquetReadOptions options = 
HadoopReadOptions.builder(getConfWithModelClass()).build();
+        return ParquetFileReader.open(new 
BeamParquetInputFile(file.openSeekable()), options);
+      }
+
+      @ProcessElement
+      public void processElement(
+          @Element FileIO.ReadableFile file,
+          RestrictionTracker<OffsetRange, Long> tracker,
+          OutputReceiver<GenericRecord> outputReceiver)
+          throws Exception {
+        LOG.info(
+            "start "
+                + tracker.currentRestriction().getFrom()
+                + " to "
+                + tracker.currentRestriction().getTo());
+        ParquetReadOptions options = 
HadoopReadOptions.builder(getConfWithModelClass()).build();
+        ParquetFileReader reader =
+            ParquetFileReader.open(new 
BeamParquetInputFile(file.openSeekable()), options);
+        GenericData model = null;
+        if (modelClass != null) {
+          model = (GenericData) modelClass.getMethod("get").invoke(null);
+        }
+        ReadSupport<GenericRecord> readSupport = new 
AvroReadSupport<GenericRecord>(model);
+
+        Filter filter = checkNotNull(options.getRecordFilter(), "filter");
+        Configuration hadoopConf = ((HadoopReadOptions) options).getConf();
+        FileMetaData parquetFileMetadata = 
reader.getFooter().getFileMetaData();
+        MessageType fileSchema = parquetFileMetadata.getSchema();
+        Map<String, String> fileMetadata = 
parquetFileMetadata.getKeyValueMetaData();
+
+        ReadSupport.ReadContext readContext =
+            readSupport.init(
+                new InitContext(
+                    hadoopConf, Maps.transformValues(fileMetadata, 
ImmutableSet::of), fileSchema));
+        ColumnIOFactory columnIOFactory = new 
ColumnIOFactory(parquetFileMetadata.getCreatedBy());
+        MessageType requestedSchema = readContext.getRequestedSchema();
+        RecordMaterializer<GenericRecord> recordConverter =
+            readSupport.prepareForRead(hadoopConf, fileMetadata, fileSchema, 
readContext);
+        reader.setRequestedSchema(requestedSchema);
+        MessageColumnIO columnIO = 
columnIOFactory.getColumnIO(requestedSchema, fileSchema, true);
+        long currentBlock = tracker.currentRestriction().getFrom();
+        for (int i = 0; i < currentBlock; i++) {
+          reader.skipNextRowGroup();
+        }
+
+        while ((tracker).tryClaim(currentBlock)) {
+          PageReadStore pages = reader.readNextRowGroup();
+          LOG.info("block {} read in memory. row count = {}", currentBlock, 
pages.getRowCount());

Review comment:
       Same above. I'm thinking about whether we really these `INFO` log since 
it looks too much if per element per offset.

##########
File path: 
sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java
##########
@@ -133,15 +161,18 @@
    * pattern).
    */
   public static Read read(Schema schema) {
-    return new AutoValue_ParquetIO_Read.Builder().setSchema(schema).build();
+    return new 
AutoValue_ParquetIO_Read.Builder().setSchema(schema).setSplittable(false).build();

Review comment:
       Please add javadoc above to explain what `withSplit()` will do.

##########
File path: 
sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java
##########
@@ -230,14 +279,255 @@ public ReadFiles withAvroDataModel(GenericData model) {
       return toBuilder().setAvroDataModel(model).build();
     }
 
+    public ReadFiles withSplit() {
+      return toBuilder().setSplittable(true).build();
+    }
+
     @Override
     public PCollection<GenericRecord> expand(PCollection<FileIO.ReadableFile> 
input) {
       checkNotNull(getSchema(), "Schema can not be null");
+      if (isSplittable()) {
+        return input
+            .apply(ParDo.of(new SplitReadFn(getAvroDataModel())))
+            .setCoder(AvroCoder.of(getSchema()));
+      }
       return input
           .apply(ParDo.of(new ReadFn(getAvroDataModel())))
           .setCoder(AvroCoder.of(getSchema()));
     }
 
+    @DoFn.BoundedPerElement
+    static class SplitReadFn extends DoFn<FileIO.ReadableFile, GenericRecord> {
+      private Class<? extends GenericData> modelClass;
+      private static final Logger LOG = 
LoggerFactory.getLogger(SplitReadFn.class);
+      // Default initial splitting the file into blocks of 64MB. Unit of 
SPLIT_LIMIT is byte.
+      private static final long SPLIT_LIMIT = 64000000;
+
+      SplitReadFn(GenericData model) {
+
+        this.modelClass = model != null ? model.getClass() : null;
+      }
+
+      ParquetFileReader getParquetFileReader(FileIO.ReadableFile file) throws 
Exception {
+        ParquetReadOptions options = 
HadoopReadOptions.builder(getConfWithModelClass()).build();
+        return ParquetFileReader.open(new 
BeamParquetInputFile(file.openSeekable()), options);
+      }
+
+      @ProcessElement
+      public void processElement(
+          @Element FileIO.ReadableFile file,
+          RestrictionTracker<OffsetRange, Long> tracker,
+          OutputReceiver<GenericRecord> outputReceiver)
+          throws Exception {
+        LOG.info(

Review comment:
       Do we really need this log?

##########
File path: 
sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java
##########
@@ -230,14 +279,255 @@ public ReadFiles withAvroDataModel(GenericData model) {
       return toBuilder().setAvroDataModel(model).build();
     }
 
+    public ReadFiles withSplit() {
+      return toBuilder().setSplittable(true).build();
+    }
+
     @Override
     public PCollection<GenericRecord> expand(PCollection<FileIO.ReadableFile> 
input) {
       checkNotNull(getSchema(), "Schema can not be null");
+      if (isSplittable()) {
+        return input
+            .apply(ParDo.of(new SplitReadFn(getAvroDataModel())))
+            .setCoder(AvroCoder.of(getSchema()));
+      }
       return input
           .apply(ParDo.of(new ReadFn(getAvroDataModel())))
           .setCoder(AvroCoder.of(getSchema()));
     }
 
+    @DoFn.BoundedPerElement
+    static class SplitReadFn extends DoFn<FileIO.ReadableFile, GenericRecord> {
+      private Class<? extends GenericData> modelClass;
+      private static final Logger LOG = 
LoggerFactory.getLogger(SplitReadFn.class);
+      // Default initial splitting the file into blocks of 64MB. Unit of 
SPLIT_LIMIT is byte.
+      private static final long SPLIT_LIMIT = 64000000;
+
+      SplitReadFn(GenericData model) {
+
+        this.modelClass = model != null ? model.getClass() : null;
+      }
+
+      ParquetFileReader getParquetFileReader(FileIO.ReadableFile file) throws 
Exception {
+        ParquetReadOptions options = 
HadoopReadOptions.builder(getConfWithModelClass()).build();
+        return ParquetFileReader.open(new 
BeamParquetInputFile(file.openSeekable()), options);
+      }
+
+      @ProcessElement
+      public void processElement(
+          @Element FileIO.ReadableFile file,
+          RestrictionTracker<OffsetRange, Long> tracker,
+          OutputReceiver<GenericRecord> outputReceiver)
+          throws Exception {
+        LOG.info(
+            "start "
+                + tracker.currentRestriction().getFrom()
+                + " to "
+                + tracker.currentRestriction().getTo());
+        ParquetReadOptions options = 
HadoopReadOptions.builder(getConfWithModelClass()).build();
+        ParquetFileReader reader =
+            ParquetFileReader.open(new 
BeamParquetInputFile(file.openSeekable()), options);
+        GenericData model = null;
+        if (modelClass != null) {
+          model = (GenericData) modelClass.getMethod("get").invoke(null);
+        }
+        ReadSupport<GenericRecord> readSupport = new 
AvroReadSupport<GenericRecord>(model);
+
+        Filter filter = checkNotNull(options.getRecordFilter(), "filter");
+        Configuration hadoopConf = ((HadoopReadOptions) options).getConf();
+        FileMetaData parquetFileMetadata = 
reader.getFooter().getFileMetaData();
+        MessageType fileSchema = parquetFileMetadata.getSchema();
+        Map<String, String> fileMetadata = 
parquetFileMetadata.getKeyValueMetaData();
+
+        ReadSupport.ReadContext readContext =
+            readSupport.init(
+                new InitContext(
+                    hadoopConf, Maps.transformValues(fileMetadata, 
ImmutableSet::of), fileSchema));
+        ColumnIOFactory columnIOFactory = new 
ColumnIOFactory(parquetFileMetadata.getCreatedBy());
+        MessageType requestedSchema = readContext.getRequestedSchema();
+        RecordMaterializer<GenericRecord> recordConverter =
+            readSupport.prepareForRead(hadoopConf, fileMetadata, fileSchema, 
readContext);
+        reader.setRequestedSchema(requestedSchema);
+        MessageColumnIO columnIO = 
columnIOFactory.getColumnIO(requestedSchema, fileSchema, true);
+        long currentBlock = tracker.currentRestriction().getFrom();
+        for (int i = 0; i < currentBlock; i++) {
+          reader.skipNextRowGroup();
+        }
+
+        while ((tracker).tryClaim(currentBlock)) {
+          PageReadStore pages = reader.readNextRowGroup();
+          LOG.info("block {} read in memory. row count = {}", currentBlock, 
pages.getRowCount());
+          currentBlock += 1;
+          RecordReader<GenericRecord> recordReader =
+              columnIO.getRecordReader(
+                  pages, recordConverter, options.useRecordFilter() ? filter : 
FilterCompat.NOOP);
+          long currentRow = 0;
+          long totalRows = pages.getRowCount();
+          while (currentRow < totalRows) {
+            try {
+              GenericRecord record;
+              currentRow += 1;
+              try {
+                record = recordReader.read();
+              } catch (RecordMaterializer.RecordMaterializationException e) {
+                LOG.debug("skipping a corrupt record");
+                continue;
+              }
+              if (record == null) {
+                // only happens with FilteredRecordReader at end of block
+                LOG.debug("filtered record reader reached end of block");
+                break;
+              }
+              if (tracker instanceof BlockTracker) {
+                ((BlockTracker) tracker).makeProgress();
+              }
+              if (recordReader.shouldSkipCurrentRecord()) {
+                // this record is being filtered via the filter2 package
+                LOG.debug("skipping record");
+                continue;
+              }
+              outputReceiver.output(record);
+            } catch (RuntimeException e) {
+
+              throw new ParquetDecodingException(
+                  format(
+                      "Can not read value at %d in block %d in file %s",
+                      currentRow, currentBlock, file.toString()),
+                  e);
+            }
+          }
+          LOG.info("Finish processing " + currentRow + " rows from block " + 
(currentBlock - 1));
+        }
+      }
+
+      public Configuration getConfWithModelClass() throws Exception {
+        Configuration conf = new Configuration();
+        GenericData model = null;
+        if (modelClass != null) {
+          model = (GenericData) modelClass.getMethod("get").invoke(null);
+        }
+        if (model != null
+            && (model.getClass() == GenericData.class || model.getClass() == 
SpecificData.class)) {
+          conf.setBoolean(AvroReadSupport.AVRO_COMPATIBILITY, true);
+        } else {
+          conf.setBoolean(AvroReadSupport.AVRO_COMPATIBILITY, false);
+        }
+        return conf;
+      }
+
+      @GetInitialRestriction
+      public OffsetRange getInitialRestriction(@Element FileIO.ReadableFile 
file) throws Exception {
+        ParquetFileReader reader = getParquetFileReader(file);
+        return new OffsetRange(0, reader.getRowGroups().size());
+      }
+
+      @SplitRestriction
+      public void split(
+          @Restriction OffsetRange restriction,
+          OutputReceiver<OffsetRange> out,
+          @Element FileIO.ReadableFile file)
+          throws Exception {
+        ParquetFileReader reader = getParquetFileReader(file);
+        List<BlockMetaData> rowGroups = reader.getRowGroups();
+        for (OffsetRange offsetRange :
+            splitBlockWithLimit(
+                restriction.getFrom(), restriction.getTo(), rowGroups, 
SPLIT_LIMIT / 1000)) {

Review comment:
       Any reason to use `SPLIT_LIMIT / 1000`?

##########
File path: 
sdks/java/io/parquet/src/test/java/org/apache/beam/sdk/io/parquet/ParquetIOTest.java
##########
@@ -69,6 +76,44 @@
         "Faraday", "Newton", "Bohr", "Galilei", "Maxwell"
       };
 
+  @Test
+  public void testBlockTracker() throws Exception {
+    OffsetRange range = new OffsetRange(0, 1);
+    ParquetIO.ReadFiles.BlockTracker tracker = new 
ParquetIO.ReadFiles.BlockTracker(range, 7, 3);
+    assertTrue(Math.abs(tracker.getProgress().getWorkRemaining() - 6) < 0.01);

Review comment:
       To compare 2 double, you can use `assertEquals(double expected, double 
actual, double epsilon)`

##########
File path: 
sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java
##########
@@ -230,14 +279,255 @@ public ReadFiles withAvroDataModel(GenericData model) {
       return toBuilder().setAvroDataModel(model).build();
     }
 
+    public ReadFiles withSplit() {
+      return toBuilder().setSplittable(true).build();
+    }
+
     @Override
     public PCollection<GenericRecord> expand(PCollection<FileIO.ReadableFile> 
input) {
       checkNotNull(getSchema(), "Schema can not be null");
+      if (isSplittable()) {
+        return input
+            .apply(ParDo.of(new SplitReadFn(getAvroDataModel())))
+            .setCoder(AvroCoder.of(getSchema()));
+      }
       return input
           .apply(ParDo.of(new ReadFn(getAvroDataModel())))
           .setCoder(AvroCoder.of(getSchema()));
     }
 
+    @DoFn.BoundedPerElement
+    static class SplitReadFn extends DoFn<FileIO.ReadableFile, GenericRecord> {
+      private Class<? extends GenericData> modelClass;
+      private static final Logger LOG = 
LoggerFactory.getLogger(SplitReadFn.class);
+      // Default initial splitting the file into blocks of 64MB. Unit of 
SPLIT_LIMIT is byte.
+      private static final long SPLIT_LIMIT = 64000000;
+
+      SplitReadFn(GenericData model) {
+
+        this.modelClass = model != null ? model.getClass() : null;
+      }
+
+      ParquetFileReader getParquetFileReader(FileIO.ReadableFile file) throws 
Exception {
+        ParquetReadOptions options = 
HadoopReadOptions.builder(getConfWithModelClass()).build();
+        return ParquetFileReader.open(new 
BeamParquetInputFile(file.openSeekable()), options);
+      }
+
+      @ProcessElement
+      public void processElement(
+          @Element FileIO.ReadableFile file,
+          RestrictionTracker<OffsetRange, Long> tracker,
+          OutputReceiver<GenericRecord> outputReceiver)
+          throws Exception {
+        LOG.info(
+            "start "
+                + tracker.currentRestriction().getFrom()
+                + " to "
+                + tracker.currentRestriction().getTo());
+        ParquetReadOptions options = 
HadoopReadOptions.builder(getConfWithModelClass()).build();
+        ParquetFileReader reader =
+            ParquetFileReader.open(new 
BeamParquetInputFile(file.openSeekable()), options);
+        GenericData model = null;
+        if (modelClass != null) {
+          model = (GenericData) modelClass.getMethod("get").invoke(null);
+        }
+        ReadSupport<GenericRecord> readSupport = new 
AvroReadSupport<GenericRecord>(model);
+
+        Filter filter = checkNotNull(options.getRecordFilter(), "filter");
+        Configuration hadoopConf = ((HadoopReadOptions) options).getConf();
+        FileMetaData parquetFileMetadata = 
reader.getFooter().getFileMetaData();
+        MessageType fileSchema = parquetFileMetadata.getSchema();
+        Map<String, String> fileMetadata = 
parquetFileMetadata.getKeyValueMetaData();
+
+        ReadSupport.ReadContext readContext =
+            readSupport.init(
+                new InitContext(
+                    hadoopConf, Maps.transformValues(fileMetadata, 
ImmutableSet::of), fileSchema));
+        ColumnIOFactory columnIOFactory = new 
ColumnIOFactory(parquetFileMetadata.getCreatedBy());
+        MessageType requestedSchema = readContext.getRequestedSchema();
+        RecordMaterializer<GenericRecord> recordConverter =
+            readSupport.prepareForRead(hadoopConf, fileMetadata, fileSchema, 
readContext);
+        reader.setRequestedSchema(requestedSchema);
+        MessageColumnIO columnIO = 
columnIOFactory.getColumnIO(requestedSchema, fileSchema, true);
+        long currentBlock = tracker.currentRestriction().getFrom();
+        for (int i = 0; i < currentBlock; i++) {
+          reader.skipNextRowGroup();
+        }
+
+        while ((tracker).tryClaim(currentBlock)) {
+          PageReadStore pages = reader.readNextRowGroup();
+          LOG.info("block {} read in memory. row count = {}", currentBlock, 
pages.getRowCount());
+          currentBlock += 1;
+          RecordReader<GenericRecord> recordReader =
+              columnIO.getRecordReader(
+                  pages, recordConverter, options.useRecordFilter() ? filter : 
FilterCompat.NOOP);
+          long currentRow = 0;
+          long totalRows = pages.getRowCount();
+          while (currentRow < totalRows) {
+            try {
+              GenericRecord record;
+              currentRow += 1;
+              try {
+                record = recordReader.read();
+              } catch (RecordMaterializer.RecordMaterializationException e) {
+                LOG.debug("skipping a corrupt record");
+                continue;
+              }
+              if (record == null) {
+                // only happens with FilteredRecordReader at end of block
+                LOG.debug("filtered record reader reached end of block");
+                break;
+              }
+              if (tracker instanceof BlockTracker) {
+                ((BlockTracker) tracker).makeProgress();
+              }
+              if (recordReader.shouldSkipCurrentRecord()) {
+                // this record is being filtered via the filter2 package
+                LOG.debug("skipping record");
+                continue;
+              }
+              outputReceiver.output(record);
+            } catch (RuntimeException e) {
+
+              throw new ParquetDecodingException(
+                  format(
+                      "Can not read value at %d in block %d in file %s",
+                      currentRow, currentBlock, file.toString()),
+                  e);
+            }
+          }
+          LOG.info("Finish processing " + currentRow + " rows from block " + 
(currentBlock - 1));
+        }
+      }
+
+      public Configuration getConfWithModelClass() throws Exception {
+        Configuration conf = new Configuration();
+        GenericData model = null;
+        if (modelClass != null) {
+          model = (GenericData) modelClass.getMethod("get").invoke(null);
+        }
+        if (model != null
+            && (model.getClass() == GenericData.class || model.getClass() == 
SpecificData.class)) {
+          conf.setBoolean(AvroReadSupport.AVRO_COMPATIBILITY, true);
+        } else {
+          conf.setBoolean(AvroReadSupport.AVRO_COMPATIBILITY, false);
+        }
+        return conf;
+      }
+
+      @GetInitialRestriction
+      public OffsetRange getInitialRestriction(@Element FileIO.ReadableFile 
file) throws Exception {
+        ParquetFileReader reader = getParquetFileReader(file);
+        return new OffsetRange(0, reader.getRowGroups().size());
+      }
+
+      @SplitRestriction
+      public void split(
+          @Restriction OffsetRange restriction,
+          OutputReceiver<OffsetRange> out,
+          @Element FileIO.ReadableFile file)
+          throws Exception {
+        ParquetFileReader reader = getParquetFileReader(file);
+        List<BlockMetaData> rowGroups = reader.getRowGroups();
+        for (OffsetRange offsetRange :
+            splitBlockWithLimit(
+                restriction.getFrom(), restriction.getTo(), rowGroups, 
SPLIT_LIMIT / 1000)) {
+          out.output(offsetRange);
+        }
+      }
+
+      public ArrayList<OffsetRange> splitBlockWithLimit(
+          long start, long end, List<BlockMetaData> blockList, long limit) {
+        ArrayList<OffsetRange> offsetList = new ArrayList<OffsetRange>();
+        long totalSize = 0;
+        long rangeStart = start;
+        long rangeEnd = start;
+        for (long i = start; i < end; i++) {
+          totalSize += blockList.get((int) i).getTotalByteSize();
+          rangeEnd += 1;
+          if (totalSize >= limit) {
+            offsetList.add(new OffsetRange(rangeStart, rangeEnd));
+            rangeStart = rangeEnd;
+            totalSize = 0;
+          }
+        }
+        if (totalSize != 0) {
+          offsetList.add(new OffsetRange(rangeStart, rangeEnd));
+        }
+        return offsetList;
+      }
+
+      @NewTracker
+      public RestrictionTracker<OffsetRange, Long> newTracker(
+          @Restriction OffsetRange restriction, @Element FileIO.ReadableFile 
file)
+          throws Exception {
+        List<Double> recordCountAndSize = getRecordCountAndSize(file, 
restriction);
+        return new BlockTracker(
+            restriction,
+            Math.round(recordCountAndSize.get(1)),
+            Math.round(recordCountAndSize.get(0)));
+      }
+
+      @GetRestrictionCoder
+      public OffsetRange.Coder getRestrictionCoder() {
+        return new OffsetRange.Coder();
+      }
+
+      @GetSize
+      public double getSize(@Element FileIO.ReadableFile file, @Restriction 
OffsetRange restriction)
+          throws Exception {
+        return getRecordCountAndSize(file, restriction).get(1);
+      }
+
+      public List<Double> getRecordCountAndSize(
+          @Element FileIO.ReadableFile file, @Restriction OffsetRange 
restriction)
+          throws Exception {
+        ParquetFileReader reader = getParquetFileReader(file);
+        double size = 0;
+        double recordCount = 0;
+        for (long i = restriction.getFrom(); i < restriction.getTo(); i++) {
+          BlockMetaData block = reader.getRowGroups().get((int) i);
+          recordCount += block.getRowCount();
+          size += block.getTotalByteSize();
+        }
+        List<Double> countAndSize = new ArrayList<>();
+        countAndSize.add(recordCount);
+        countAndSize.add(size);
+        return countAndSize;
+      }
+    }
+
+    public static class BlockTracker extends OffsetRangeTracker {
+      private long totalWork;
+      private long progress;
+      private long approximateRecordSize;
+
+      public BlockTracker(OffsetRange range, long totalByteSize, long 
recordCount) {
+        super(range);
+        if (recordCount != 0) {
+          this.approximateRecordSize = totalByteSize / recordCount;
+          this.totalWork = approximateRecordSize * recordCount;
+          this.progress = 0;
+        }
+      }
+
+      public void makeProgress() throws Exception {
+        progress += approximateRecordSize;
+        if (progress > totalWork) {
+          throw new IOException("Making progress out of range");
+        }
+      }
+
+      @Override
+      public Progress getProgress() {

Review comment:
       Also if `makeProgress()` is never called because of the guard you have, 
the progress will always be (0, totalWork), which is not correct to some extent.

##########
File path: 
sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java
##########
@@ -230,14 +279,255 @@ public ReadFiles withAvroDataModel(GenericData model) {
       return toBuilder().setAvroDataModel(model).build();
     }
 
+    public ReadFiles withSplit() {
+      return toBuilder().setSplittable(true).build();
+    }
+
     @Override
     public PCollection<GenericRecord> expand(PCollection<FileIO.ReadableFile> 
input) {
       checkNotNull(getSchema(), "Schema can not be null");
+      if (isSplittable()) {
+        return input
+            .apply(ParDo.of(new SplitReadFn(getAvroDataModel())))
+            .setCoder(AvroCoder.of(getSchema()));
+      }
       return input
           .apply(ParDo.of(new ReadFn(getAvroDataModel())))
           .setCoder(AvroCoder.of(getSchema()));
     }
 
+    @DoFn.BoundedPerElement
+    static class SplitReadFn extends DoFn<FileIO.ReadableFile, GenericRecord> {
+      private Class<? extends GenericData> modelClass;
+      private static final Logger LOG = 
LoggerFactory.getLogger(SplitReadFn.class);
+      // Default initial splitting the file into blocks of 64MB. Unit of 
SPLIT_LIMIT is byte.
+      private static final long SPLIT_LIMIT = 64000000;
+
+      SplitReadFn(GenericData model) {
+
+        this.modelClass = model != null ? model.getClass() : null;
+      }
+
+      ParquetFileReader getParquetFileReader(FileIO.ReadableFile file) throws 
Exception {
+        ParquetReadOptions options = 
HadoopReadOptions.builder(getConfWithModelClass()).build();
+        return ParquetFileReader.open(new 
BeamParquetInputFile(file.openSeekable()), options);
+      }
+
+      @ProcessElement
+      public void processElement(
+          @Element FileIO.ReadableFile file,
+          RestrictionTracker<OffsetRange, Long> tracker,
+          OutputReceiver<GenericRecord> outputReceiver)
+          throws Exception {
+        LOG.info(
+            "start "
+                + tracker.currentRestriction().getFrom()
+                + " to "
+                + tracker.currentRestriction().getTo());
+        ParquetReadOptions options = 
HadoopReadOptions.builder(getConfWithModelClass()).build();
+        ParquetFileReader reader =
+            ParquetFileReader.open(new 
BeamParquetInputFile(file.openSeekable()), options);
+        GenericData model = null;
+        if (modelClass != null) {
+          model = (GenericData) modelClass.getMethod("get").invoke(null);
+        }
+        ReadSupport<GenericRecord> readSupport = new 
AvroReadSupport<GenericRecord>(model);
+
+        Filter filter = checkNotNull(options.getRecordFilter(), "filter");
+        Configuration hadoopConf = ((HadoopReadOptions) options).getConf();
+        FileMetaData parquetFileMetadata = 
reader.getFooter().getFileMetaData();
+        MessageType fileSchema = parquetFileMetadata.getSchema();
+        Map<String, String> fileMetadata = 
parquetFileMetadata.getKeyValueMetaData();
+
+        ReadSupport.ReadContext readContext =
+            readSupport.init(
+                new InitContext(
+                    hadoopConf, Maps.transformValues(fileMetadata, 
ImmutableSet::of), fileSchema));
+        ColumnIOFactory columnIOFactory = new 
ColumnIOFactory(parquetFileMetadata.getCreatedBy());
+        MessageType requestedSchema = readContext.getRequestedSchema();
+        RecordMaterializer<GenericRecord> recordConverter =
+            readSupport.prepareForRead(hadoopConf, fileMetadata, fileSchema, 
readContext);
+        reader.setRequestedSchema(requestedSchema);
+        MessageColumnIO columnIO = 
columnIOFactory.getColumnIO(requestedSchema, fileSchema, true);
+        long currentBlock = tracker.currentRestriction().getFrom();
+        for (int i = 0; i < currentBlock; i++) {
+          reader.skipNextRowGroup();
+        }
+
+        while ((tracker).tryClaim(currentBlock)) {
+          PageReadStore pages = reader.readNextRowGroup();
+          LOG.info("block {} read in memory. row count = {}", currentBlock, 
pages.getRowCount());
+          currentBlock += 1;
+          RecordReader<GenericRecord> recordReader =
+              columnIO.getRecordReader(
+                  pages, recordConverter, options.useRecordFilter() ? filter : 
FilterCompat.NOOP);
+          long currentRow = 0;
+          long totalRows = pages.getRowCount();
+          while (currentRow < totalRows) {
+            try {
+              GenericRecord record;
+              currentRow += 1;
+              try {
+                record = recordReader.read();
+              } catch (RecordMaterializer.RecordMaterializationException e) {
+                LOG.debug("skipping a corrupt record");
+                continue;
+              }
+              if (record == null) {
+                // only happens with FilteredRecordReader at end of block
+                LOG.debug("filtered record reader reached end of block");
+                break;
+              }
+              if (tracker instanceof BlockTracker) {
+                ((BlockTracker) tracker).makeProgress();
+              }
+              if (recordReader.shouldSkipCurrentRecord()) {
+                // this record is being filtered via the filter2 package
+                LOG.debug("skipping record");
+                continue;
+              }
+              outputReceiver.output(record);
+            } catch (RuntimeException e) {
+
+              throw new ParquetDecodingException(
+                  format(
+                      "Can not read value at %d in block %d in file %s",
+                      currentRow, currentBlock, file.toString()),
+                  e);
+            }
+          }
+          LOG.info("Finish processing " + currentRow + " rows from block " + 
(currentBlock - 1));

Review comment:
       Same above.

##########
File path: 
sdks/java/io/parquet/src/test/java/org/apache/beam/sdk/io/parquet/ParquetIOTest.java
##########
@@ -69,6 +76,44 @@
         "Faraday", "Newton", "Bohr", "Galilei", "Maxwell"
       };
 
+  @Test
+  public void testBlockTracker() throws Exception {
+    OffsetRange range = new OffsetRange(0, 1);
+    ParquetIO.ReadFiles.BlockTracker tracker = new 
ParquetIO.ReadFiles.BlockTracker(range, 7, 3);
+    assertTrue(Math.abs(tracker.getProgress().getWorkRemaining() - 6) < 0.01);
+    assertTrue(Math.abs(tracker.getProgress().getWorkCompleted()) < 0.01);
+    tracker.tryClaim((long) 0);

Review comment:
       you can use `0L` to represent that `0` is a `long`. 

##########
File path: 
sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java
##########
@@ -230,14 +279,255 @@ public ReadFiles withAvroDataModel(GenericData model) {
       return toBuilder().setAvroDataModel(model).build();
     }
 
+    public ReadFiles withSplit() {
+      return toBuilder().setSplittable(true).build();
+    }
+
     @Override
     public PCollection<GenericRecord> expand(PCollection<FileIO.ReadableFile> 
input) {
       checkNotNull(getSchema(), "Schema can not be null");
+      if (isSplittable()) {
+        return input
+            .apply(ParDo.of(new SplitReadFn(getAvroDataModel())))
+            .setCoder(AvroCoder.of(getSchema()));
+      }
       return input
           .apply(ParDo.of(new ReadFn(getAvroDataModel())))
           .setCoder(AvroCoder.of(getSchema()));
     }
 
+    @DoFn.BoundedPerElement
+    static class SplitReadFn extends DoFn<FileIO.ReadableFile, GenericRecord> {
+      private Class<? extends GenericData> modelClass;
+      private static final Logger LOG = 
LoggerFactory.getLogger(SplitReadFn.class);
+      // Default initial splitting the file into blocks of 64MB. Unit of 
SPLIT_LIMIT is byte.
+      private static final long SPLIT_LIMIT = 64000000;
+
+      SplitReadFn(GenericData model) {
+
+        this.modelClass = model != null ? model.getClass() : null;
+      }
+
+      ParquetFileReader getParquetFileReader(FileIO.ReadableFile file) throws 
Exception {
+        ParquetReadOptions options = 
HadoopReadOptions.builder(getConfWithModelClass()).build();
+        return ParquetFileReader.open(new 
BeamParquetInputFile(file.openSeekable()), options);
+      }
+
+      @ProcessElement
+      public void processElement(
+          @Element FileIO.ReadableFile file,
+          RestrictionTracker<OffsetRange, Long> tracker,
+          OutputReceiver<GenericRecord> outputReceiver)
+          throws Exception {
+        LOG.info(
+            "start "
+                + tracker.currentRestriction().getFrom()
+                + " to "
+                + tracker.currentRestriction().getTo());
+        ParquetReadOptions options = 
HadoopReadOptions.builder(getConfWithModelClass()).build();
+        ParquetFileReader reader =
+            ParquetFileReader.open(new 
BeamParquetInputFile(file.openSeekable()), options);
+        GenericData model = null;
+        if (modelClass != null) {
+          model = (GenericData) modelClass.getMethod("get").invoke(null);
+        }
+        ReadSupport<GenericRecord> readSupport = new 
AvroReadSupport<GenericRecord>(model);
+
+        Filter filter = checkNotNull(options.getRecordFilter(), "filter");
+        Configuration hadoopConf = ((HadoopReadOptions) options).getConf();
+        FileMetaData parquetFileMetadata = 
reader.getFooter().getFileMetaData();
+        MessageType fileSchema = parquetFileMetadata.getSchema();
+        Map<String, String> fileMetadata = 
parquetFileMetadata.getKeyValueMetaData();
+
+        ReadSupport.ReadContext readContext =
+            readSupport.init(
+                new InitContext(
+                    hadoopConf, Maps.transformValues(fileMetadata, 
ImmutableSet::of), fileSchema));
+        ColumnIOFactory columnIOFactory = new 
ColumnIOFactory(parquetFileMetadata.getCreatedBy());
+        MessageType requestedSchema = readContext.getRequestedSchema();
+        RecordMaterializer<GenericRecord> recordConverter =
+            readSupport.prepareForRead(hadoopConf, fileMetadata, fileSchema, 
readContext);
+        reader.setRequestedSchema(requestedSchema);
+        MessageColumnIO columnIO = 
columnIOFactory.getColumnIO(requestedSchema, fileSchema, true);
+        long currentBlock = tracker.currentRestriction().getFrom();
+        for (int i = 0; i < currentBlock; i++) {
+          reader.skipNextRowGroup();
+        }
+
+        while ((tracker).tryClaim(currentBlock)) {
+          PageReadStore pages = reader.readNextRowGroup();
+          LOG.info("block {} read in memory. row count = {}", currentBlock, 
pages.getRowCount());
+          currentBlock += 1;
+          RecordReader<GenericRecord> recordReader =
+              columnIO.getRecordReader(
+                  pages, recordConverter, options.useRecordFilter() ? filter : 
FilterCompat.NOOP);
+          long currentRow = 0;
+          long totalRows = pages.getRowCount();
+          while (currentRow < totalRows) {
+            try {
+              GenericRecord record;
+              currentRow += 1;
+              try {
+                record = recordReader.read();
+              } catch (RecordMaterializer.RecordMaterializationException e) {
+                LOG.debug("skipping a corrupt record");
+                continue;
+              }
+              if (record == null) {
+                // only happens with FilteredRecordReader at end of block
+                LOG.debug("filtered record reader reached end of block");
+                break;
+              }
+              if (tracker instanceof BlockTracker) {
+                ((BlockTracker) tracker).makeProgress();
+              }
+              if (recordReader.shouldSkipCurrentRecord()) {
+                // this record is being filtered via the filter2 package
+                LOG.debug("skipping record");
+                continue;
+              }
+              outputReceiver.output(record);
+            } catch (RuntimeException e) {
+
+              throw new ParquetDecodingException(
+                  format(
+                      "Can not read value at %d in block %d in file %s",
+                      currentRow, currentBlock, file.toString()),
+                  e);
+            }
+          }
+          LOG.info("Finish processing " + currentRow + " rows from block " + 
(currentBlock - 1));
+        }
+      }
+
+      public Configuration getConfWithModelClass() throws Exception {
+        Configuration conf = new Configuration();
+        GenericData model = null;
+        if (modelClass != null) {
+          model = (GenericData) modelClass.getMethod("get").invoke(null);
+        }
+        if (model != null
+            && (model.getClass() == GenericData.class || model.getClass() == 
SpecificData.class)) {
+          conf.setBoolean(AvroReadSupport.AVRO_COMPATIBILITY, true);
+        } else {
+          conf.setBoolean(AvroReadSupport.AVRO_COMPATIBILITY, false);
+        }
+        return conf;
+      }
+
+      @GetInitialRestriction
+      public OffsetRange getInitialRestriction(@Element FileIO.ReadableFile 
file) throws Exception {
+        ParquetFileReader reader = getParquetFileReader(file);
+        return new OffsetRange(0, reader.getRowGroups().size());
+      }
+
+      @SplitRestriction
+      public void split(
+          @Restriction OffsetRange restriction,
+          OutputReceiver<OffsetRange> out,
+          @Element FileIO.ReadableFile file)
+          throws Exception {
+        ParquetFileReader reader = getParquetFileReader(file);
+        List<BlockMetaData> rowGroups = reader.getRowGroups();
+        for (OffsetRange offsetRange :
+            splitBlockWithLimit(
+                restriction.getFrom(), restriction.getTo(), rowGroups, 
SPLIT_LIMIT / 1000)) {
+          out.output(offsetRange);
+        }
+      }
+
+      public ArrayList<OffsetRange> splitBlockWithLimit(
+          long start, long end, List<BlockMetaData> blockList, long limit) {
+        ArrayList<OffsetRange> offsetList = new ArrayList<OffsetRange>();
+        long totalSize = 0;
+        long rangeStart = start;
+        long rangeEnd = start;
+        for (long i = start; i < end; i++) {
+          totalSize += blockList.get((int) i).getTotalByteSize();
+          rangeEnd += 1;
+          if (totalSize >= limit) {
+            offsetList.add(new OffsetRange(rangeStart, rangeEnd));
+            rangeStart = rangeEnd;
+            totalSize = 0;
+          }
+        }
+        if (totalSize != 0) {
+          offsetList.add(new OffsetRange(rangeStart, rangeEnd));
+        }
+        return offsetList;
+      }
+
+      @NewTracker
+      public RestrictionTracker<OffsetRange, Long> newTracker(
+          @Restriction OffsetRange restriction, @Element FileIO.ReadableFile 
file)
+          throws Exception {
+        List<Double> recordCountAndSize = getRecordCountAndSize(file, 
restriction);
+        return new BlockTracker(
+            restriction,
+            Math.round(recordCountAndSize.get(1)),
+            Math.round(recordCountAndSize.get(0)));
+      }
+
+      @GetRestrictionCoder
+      public OffsetRange.Coder getRestrictionCoder() {
+        return new OffsetRange.Coder();
+      }
+
+      @GetSize
+      public double getSize(@Element FileIO.ReadableFile file, @Restriction 
OffsetRange restriction)
+          throws Exception {
+        return getRecordCountAndSize(file, restriction).get(1);
+      }
+
+      public List<Double> getRecordCountAndSize(
+          @Element FileIO.ReadableFile file, @Restriction OffsetRange 
restriction)
+          throws Exception {
+        ParquetFileReader reader = getParquetFileReader(file);
+        double size = 0;
+        double recordCount = 0;
+        for (long i = restriction.getFrom(); i < restriction.getTo(); i++) {
+          BlockMetaData block = reader.getRowGroups().get((int) i);
+          recordCount += block.getRowCount();
+          size += block.getTotalByteSize();
+        }
+        List<Double> countAndSize = new ArrayList<>();

Review comment:
       Can we have a simple `AutoValue` class for `countAndSize`?

##########
File path: 
sdks/java/io/parquet/src/test/java/org/apache/beam/sdk/io/parquet/ParquetIOTest.java
##########
@@ -69,6 +76,44 @@
         "Faraday", "Newton", "Bohr", "Galilei", "Maxwell"
       };
 
+  @Test
+  public void testBlockTracker() throws Exception {
+    OffsetRange range = new OffsetRange(0, 1);
+    ParquetIO.ReadFiles.BlockTracker tracker = new 
ParquetIO.ReadFiles.BlockTracker(range, 7, 3);
+    assertTrue(Math.abs(tracker.getProgress().getWorkRemaining() - 6) < 0.01);
+    assertTrue(Math.abs(tracker.getProgress().getWorkCompleted()) < 0.01);
+    tracker.tryClaim((long) 0);
+    tracker.makeProgress();
+    assertTrue(Math.abs(tracker.getProgress().getWorkRemaining() - 4) < 0.01);
+    assertTrue(Math.abs(tracker.getProgress().getWorkCompleted() - 2) < 0.01);
+    assertThrows(RuntimeException.class, () -> tracker.tryClaim((long) 0));
+    tracker.makeProgress();
+    tracker.makeProgress();
+    assertTrue(Math.abs(tracker.getProgress().getWorkRemaining() - 0) < 0.01);
+    assertTrue(Math.abs(tracker.getProgress().getWorkCompleted() - 6) < 0.01);
+    assertThrows("Making progress out of range", IOException.class, () -> 
tracker.makeProgress());
+  }
+
+  @Test
+  public void testSplitBlockWithLimit() {
+    ParquetIO.ReadFiles.SplitReadFn testFn = new 
ParquetIO.ReadFiles.SplitReadFn(null);
+    ArrayList<BlockMetaData> blockList = new ArrayList<BlockMetaData>();
+    ArrayList<OffsetRange> rangeList;
+    BlockMetaData testBlock = mock(BlockMetaData.class);
+    when(testBlock.getTotalByteSize()).thenReturn((long) 60);
+    rangeList = testFn.splitBlockWithLimit(0, blockList.size(), blockList, 
200);
+    assertTrue(rangeList.isEmpty());
+    for (int i = 0; i < 6; i++) {
+      blockList.add(testBlock);
+    }
+    rangeList = testFn.splitBlockWithLimit(1, blockList.size(), blockList, 
200);
+    assertTrue(rangeList.get(0).getFrom() == (long) 1);

Review comment:
       `assertEquals`?

##########
File path: 
sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java
##########
@@ -230,14 +279,255 @@ public ReadFiles withAvroDataModel(GenericData model) {
       return toBuilder().setAvroDataModel(model).build();
     }
 
+    public ReadFiles withSplit() {
+      return toBuilder().setSplittable(true).build();
+    }
+
     @Override
     public PCollection<GenericRecord> expand(PCollection<FileIO.ReadableFile> 
input) {
       checkNotNull(getSchema(), "Schema can not be null");
+      if (isSplittable()) {
+        return input
+            .apply(ParDo.of(new SplitReadFn(getAvroDataModel())))
+            .setCoder(AvroCoder.of(getSchema()));
+      }
       return input
           .apply(ParDo.of(new ReadFn(getAvroDataModel())))
           .setCoder(AvroCoder.of(getSchema()));
     }
 
+    @DoFn.BoundedPerElement
+    static class SplitReadFn extends DoFn<FileIO.ReadableFile, GenericRecord> {
+      private Class<? extends GenericData> modelClass;
+      private static final Logger LOG = 
LoggerFactory.getLogger(SplitReadFn.class);
+      // Default initial splitting the file into blocks of 64MB. Unit of 
SPLIT_LIMIT is byte.
+      private static final long SPLIT_LIMIT = 64000000;
+
+      SplitReadFn(GenericData model) {
+
+        this.modelClass = model != null ? model.getClass() : null;
+      }
+
+      ParquetFileReader getParquetFileReader(FileIO.ReadableFile file) throws 
Exception {
+        ParquetReadOptions options = 
HadoopReadOptions.builder(getConfWithModelClass()).build();
+        return ParquetFileReader.open(new 
BeamParquetInputFile(file.openSeekable()), options);
+      }
+
+      @ProcessElement
+      public void processElement(
+          @Element FileIO.ReadableFile file,
+          RestrictionTracker<OffsetRange, Long> tracker,
+          OutputReceiver<GenericRecord> outputReceiver)
+          throws Exception {
+        LOG.info(
+            "start "
+                + tracker.currentRestriction().getFrom()
+                + " to "
+                + tracker.currentRestriction().getTo());
+        ParquetReadOptions options = 
HadoopReadOptions.builder(getConfWithModelClass()).build();
+        ParquetFileReader reader =
+            ParquetFileReader.open(new 
BeamParquetInputFile(file.openSeekable()), options);
+        GenericData model = null;
+        if (modelClass != null) {
+          model = (GenericData) modelClass.getMethod("get").invoke(null);
+        }
+        ReadSupport<GenericRecord> readSupport = new 
AvroReadSupport<GenericRecord>(model);
+
+        Filter filter = checkNotNull(options.getRecordFilter(), "filter");
+        Configuration hadoopConf = ((HadoopReadOptions) options).getConf();
+        FileMetaData parquetFileMetadata = 
reader.getFooter().getFileMetaData();
+        MessageType fileSchema = parquetFileMetadata.getSchema();
+        Map<String, String> fileMetadata = 
parquetFileMetadata.getKeyValueMetaData();
+
+        ReadSupport.ReadContext readContext =
+            readSupport.init(
+                new InitContext(
+                    hadoopConf, Maps.transformValues(fileMetadata, 
ImmutableSet::of), fileSchema));
+        ColumnIOFactory columnIOFactory = new 
ColumnIOFactory(parquetFileMetadata.getCreatedBy());
+        MessageType requestedSchema = readContext.getRequestedSchema();
+        RecordMaterializer<GenericRecord> recordConverter =
+            readSupport.prepareForRead(hadoopConf, fileMetadata, fileSchema, 
readContext);
+        reader.setRequestedSchema(requestedSchema);
+        MessageColumnIO columnIO = 
columnIOFactory.getColumnIO(requestedSchema, fileSchema, true);
+        long currentBlock = tracker.currentRestriction().getFrom();
+        for (int i = 0; i < currentBlock; i++) {
+          reader.skipNextRowGroup();
+        }
+
+        while ((tracker).tryClaim(currentBlock)) {
+          PageReadStore pages = reader.readNextRowGroup();
+          LOG.info("block {} read in memory. row count = {}", currentBlock, 
pages.getRowCount());
+          currentBlock += 1;
+          RecordReader<GenericRecord> recordReader =
+              columnIO.getRecordReader(
+                  pages, recordConverter, options.useRecordFilter() ? filter : 
FilterCompat.NOOP);
+          long currentRow = 0;
+          long totalRows = pages.getRowCount();
+          while (currentRow < totalRows) {
+            try {
+              GenericRecord record;
+              currentRow += 1;
+              try {
+                record = recordReader.read();
+              } catch (RecordMaterializer.RecordMaterializationException e) {
+                LOG.debug("skipping a corrupt record");
+                continue;
+              }
+              if (record == null) {
+                // only happens with FilteredRecordReader at end of block
+                LOG.debug("filtered record reader reached end of block");
+                break;
+              }
+              if (tracker instanceof BlockTracker) {
+                ((BlockTracker) tracker).makeProgress();
+              }
+              if (recordReader.shouldSkipCurrentRecord()) {
+                // this record is being filtered via the filter2 package
+                LOG.debug("skipping record");
+                continue;
+              }
+              outputReceiver.output(record);
+            } catch (RuntimeException e) {
+
+              throw new ParquetDecodingException(
+                  format(
+                      "Can not read value at %d in block %d in file %s",
+                      currentRow, currentBlock, file.toString()),
+                  e);
+            }
+          }
+          LOG.info("Finish processing " + currentRow + " rows from block " + 
(currentBlock - 1));
+        }
+      }
+
+      public Configuration getConfWithModelClass() throws Exception {
+        Configuration conf = new Configuration();
+        GenericData model = null;
+        if (modelClass != null) {
+          model = (GenericData) modelClass.getMethod("get").invoke(null);
+        }
+        if (model != null
+            && (model.getClass() == GenericData.class || model.getClass() == 
SpecificData.class)) {
+          conf.setBoolean(AvroReadSupport.AVRO_COMPATIBILITY, true);
+        } else {
+          conf.setBoolean(AvroReadSupport.AVRO_COMPATIBILITY, false);
+        }
+        return conf;
+      }
+
+      @GetInitialRestriction
+      public OffsetRange getInitialRestriction(@Element FileIO.ReadableFile 
file) throws Exception {
+        ParquetFileReader reader = getParquetFileReader(file);
+        return new OffsetRange(0, reader.getRowGroups().size());
+      }
+
+      @SplitRestriction
+      public void split(
+          @Restriction OffsetRange restriction,
+          OutputReceiver<OffsetRange> out,
+          @Element FileIO.ReadableFile file)
+          throws Exception {
+        ParquetFileReader reader = getParquetFileReader(file);
+        List<BlockMetaData> rowGroups = reader.getRowGroups();
+        for (OffsetRange offsetRange :
+            splitBlockWithLimit(
+                restriction.getFrom(), restriction.getTo(), rowGroups, 
SPLIT_LIMIT / 1000)) {
+          out.output(offsetRange);
+        }
+      }
+
+      public ArrayList<OffsetRange> splitBlockWithLimit(
+          long start, long end, List<BlockMetaData> blockList, long limit) {
+        ArrayList<OffsetRange> offsetList = new ArrayList<OffsetRange>();
+        long totalSize = 0;
+        long rangeStart = start;
+        long rangeEnd = start;
+        for (long i = start; i < end; i++) {
+          totalSize += blockList.get((int) i).getTotalByteSize();
+          rangeEnd += 1;
+          if (totalSize >= limit) {
+            offsetList.add(new OffsetRange(rangeStart, rangeEnd));
+            rangeStart = rangeEnd;
+            totalSize = 0;
+          }
+        }
+        if (totalSize != 0) {
+          offsetList.add(new OffsetRange(rangeStart, rangeEnd));
+        }
+        return offsetList;
+      }
+
+      @NewTracker
+      public RestrictionTracker<OffsetRange, Long> newTracker(
+          @Restriction OffsetRange restriction, @Element FileIO.ReadableFile 
file)
+          throws Exception {
+        List<Double> recordCountAndSize = getRecordCountAndSize(file, 
restriction);
+        return new BlockTracker(
+            restriction,
+            Math.round(recordCountAndSize.get(1)),
+            Math.round(recordCountAndSize.get(0)));
+      }
+
+      @GetRestrictionCoder
+      public OffsetRange.Coder getRestrictionCoder() {
+        return new OffsetRange.Coder();
+      }
+
+      @GetSize
+      public double getSize(@Element FileIO.ReadableFile file, @Restriction 
OffsetRange restriction)
+          throws Exception {
+        return getRecordCountAndSize(file, restriction).get(1);
+      }
+
+      public List<Double> getRecordCountAndSize(
+          @Element FileIO.ReadableFile file, @Restriction OffsetRange 
restriction)
+          throws Exception {
+        ParquetFileReader reader = getParquetFileReader(file);
+        double size = 0;
+        double recordCount = 0;
+        for (long i = restriction.getFrom(); i < restriction.getTo(); i++) {
+          BlockMetaData block = reader.getRowGroups().get((int) i);
+          recordCount += block.getRowCount();
+          size += block.getTotalByteSize();
+        }
+        List<Double> countAndSize = new ArrayList<>();
+        countAndSize.add(recordCount);
+        countAndSize.add(size);
+        return countAndSize;
+      }
+    }
+
+    public static class BlockTracker extends OffsetRangeTracker {
+      private long totalWork;
+      private long progress;
+      private long approximateRecordSize;
+
+      public BlockTracker(OffsetRange range, long totalByteSize, long 
recordCount) {
+        super(range);
+        if (recordCount != 0) {
+          this.approximateRecordSize = totalByteSize / recordCount;
+          this.totalWork = approximateRecordSize * recordCount;
+          this.progress = 0;
+        }
+      }
+
+      public void makeProgress() throws Exception {
+        progress += approximateRecordSize;

Review comment:
       It seems like in the DoFn body, you will call `tracker.makeProgress()` 
for each record. It doesn't work under fnapi context since we only expose limit 
APIs: 
https://github.com/apache/beam/blob/7fb07ff78e9895fb486f25f2ee477b875b112c3b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/splittabledofn/RestrictionTrackers.java#L91-L104




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


Reply via email to