You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2020/09/01 22:11:49 UTC

[GitHub] [beam] ihji commented on a change in pull request #12223: [Beam-4379] Make ParquetIO read splittable

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



##########
File path: sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java
##########
@@ -229,15 +290,258 @@ public void populateDisplayData(DisplayData.Builder builder) {
     public ReadFiles withAvroDataModel(GenericData model) {
       return toBuilder().setAvroDataModel(model).build();
     }
+    /** Enable the Splittable reading. */
+    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.debug(
+            "start "
+                + tracker.currentRestriction().getFrom()
+                + " to "
+                + tracker.currentRestriction().getTo());
+        ParquetReadOptions options = HadoopReadOptions.builder(getConfWithModelClass()).build();
+        ParquetFileReader reader =

Review comment:
       looks like these two lines could be also replaced with `getParquetFileReader()`.

##########
File path: sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java
##########
@@ -230,14 +281,284 @@ public ReadFiles withAvroDataModel(GenericData model) {
       return toBuilder().setAvroDataModel(model).build();
     }
 
+    public ReadFiles withSplit() {
+      return toBuilder().setSplit(true).build();
+    }
+
     @Override
     public PCollection<GenericRecord> expand(PCollection<FileIO.ReadableFile> input) {
       checkNotNull(getSchema(), "Schema can not be null");
+      if (getSplit()) {
+        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);
+      private static final long SPLIT_LIMIT = 64000000;
+      ReadSupport<GenericRecord> readSupport;
+
+      SplitReadFn(GenericData model) {
+        this.modelClass = model != null ? model.getClass() : null;
+      }
+
+      private static <K, V> Map<K, Set<V>> toSetMultiMap(Map<K, V> map) {
+        Map<K, Set<V>> setMultiMap = new HashMap<K, Set<V>>();
+        for (Map.Entry<K, V> entry : map.entrySet()) {
+          Set<V> set = new HashSet<V>();
+          set.add(entry.getValue());
+          setMultiMap.put(entry.getKey(), Collections.unmodifiableSet(set));
+        }
+        return Collections.unmodifiableMap(setMultiMap);
+      }
+
+      public InputFile getInputFile(FileIO.ReadableFile file) throws IOException {
+        if (!file.getMetadata().isReadSeekEfficient()) {
+          throw new RuntimeException(
+              String.format("File has to be seekable: %s", file.getMetadata().resourceId()));
+        }
+        return new BeamParquetInputFile(file.openSeekable());
+      }
+
+      @ProcessElement
+      public void processElement(
+          @Element FileIO.ReadableFile file,
+          RestrictionTracker<OffsetRange, Long> tracker,
+          OutputReceiver<GenericRecord> outputReceiver)
+          throws Exception {
+        ReadSupport<GenericRecord> readSupport;
+        InputFile inputFile = getInputFile(file);
+        Configuration conf = setConf();
+        GenericData model = null;
+        if (modelClass != null) {
+          model = (GenericData) modelClass.getMethod("get").invoke(null);
+        }
+        readSupport = new AvroReadSupport<GenericRecord>(model);
+        ParquetReadOptions options = HadoopReadOptions.builder(conf).build();
+        ParquetFileReader reader = ParquetFileReader.open(inputFile, options);
+        Filter filter = checkNotNull(options.getRecordFilter(), "filter");
+        conf = ((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(conf, toSetMultiMap(fileMetadata), fileSchema));
+        ColumnIOFactory columnIOFactory = new ColumnIOFactory(parquetFileMetadata.getCreatedBy());
+        MessageType requestedSchema = readContext.getRequestedSchema();
+        RecordMaterializer<GenericRecord> recordConverter =
+            readSupport.prepareForRead(conf, fileMetadata, fileSchema, readContext);
+        boolean strictTypeChecking = options.isEnabled(STRICT_TYPE_CHECKING, true);
+        boolean filterRecords = options.useRecordFilter();
+        reader.setRequestedSchema(requestedSchema);
+        MessageColumnIO columnIO =
+            columnIOFactory.getColumnIO(requestedSchema, fileSchema, strictTypeChecking);
+        long currentBlock = tracker.currentRestriction().getFrom();
+        for (int i = 0; i < currentBlock; i++) {
+          reader.skipNextRowGroup();
+        }
+
+        while ((tracker).tryClaim(currentBlock)) {
+          LOG.info("reading block" + currentBlock);
+          PageReadStore pages = reader.readNextRowGroup();
+          currentBlock += 1;
+          RecordReader<GenericRecord> recordReader =
+              columnIO.getRecordReader(
+                  pages, recordConverter, filterRecords ? 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");

Review comment:
       I think this should be `warn` not `debug`. WDYT @chamikaramj ?

##########
File path: sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java
##########
@@ -230,14 +281,284 @@ public ReadFiles withAvroDataModel(GenericData model) {
       return toBuilder().setAvroDataModel(model).build();
     }
 
+    public ReadFiles withSplit() {
+      return toBuilder().setSplit(true).build();
+    }
+
     @Override
     public PCollection<GenericRecord> expand(PCollection<FileIO.ReadableFile> input) {
       checkNotNull(getSchema(), "Schema can not be null");
+      if (getSplit()) {
+        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);
+      private static final long SPLIT_LIMIT = 64000000;
+      ReadSupport<GenericRecord> readSupport;
+
+      SplitReadFn(GenericData model) {
+        this.modelClass = model != null ? model.getClass() : null;
+      }
+
+      private static <K, V> Map<K, Set<V>> toSetMultiMap(Map<K, V> map) {
+        Map<K, Set<V>> setMultiMap = new HashMap<K, Set<V>>();
+        for (Map.Entry<K, V> entry : map.entrySet()) {
+          Set<V> set = new HashSet<V>();
+          set.add(entry.getValue());
+          setMultiMap.put(entry.getKey(), Collections.unmodifiableSet(set));
+        }
+        return Collections.unmodifiableMap(setMultiMap);
+      }
+
+      public InputFile getInputFile(FileIO.ReadableFile file) throws IOException {
+        if (!file.getMetadata().isReadSeekEfficient()) {
+          throw new RuntimeException(
+              String.format("File has to be seekable: %s", file.getMetadata().resourceId()));
+        }
+        return new BeamParquetInputFile(file.openSeekable());
+      }
+
+      @ProcessElement
+      public void processElement(
+          @Element FileIO.ReadableFile file,
+          RestrictionTracker<OffsetRange, Long> tracker,
+          OutputReceiver<GenericRecord> outputReceiver)
+          throws Exception {
+        ReadSupport<GenericRecord> readSupport;
+        InputFile inputFile = getInputFile(file);
+        Configuration conf = setConf();
+        GenericData model = null;
+        if (modelClass != null) {
+          model = (GenericData) modelClass.getMethod("get").invoke(null);
+        }
+        readSupport = new AvroReadSupport<GenericRecord>(model);
+        ParquetReadOptions options = HadoopReadOptions.builder(conf).build();
+        ParquetFileReader reader = ParquetFileReader.open(inputFile, options);
+        Filter filter = checkNotNull(options.getRecordFilter(), "filter");
+        conf = ((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(conf, toSetMultiMap(fileMetadata), fileSchema));
+        ColumnIOFactory columnIOFactory = new ColumnIOFactory(parquetFileMetadata.getCreatedBy());
+        MessageType requestedSchema = readContext.getRequestedSchema();
+        RecordMaterializer<GenericRecord> recordConverter =
+            readSupport.prepareForRead(conf, fileMetadata, fileSchema, readContext);
+        boolean strictTypeChecking = options.isEnabled(STRICT_TYPE_CHECKING, true);
+        boolean filterRecords = options.useRecordFilter();
+        reader.setRequestedSchema(requestedSchema);
+        MessageColumnIO columnIO =
+            columnIOFactory.getColumnIO(requestedSchema, fileSchema, strictTypeChecking);
+        long currentBlock = tracker.currentRestriction().getFrom();
+        for (int i = 0; i < currentBlock; i++) {
+          reader.skipNextRowGroup();
+        }
+
+        while ((tracker).tryClaim(currentBlock)) {
+          LOG.info("reading block" + currentBlock);
+          PageReadStore pages = reader.readNextRowGroup();
+          currentBlock += 1;
+          RecordReader<GenericRecord> recordReader =
+              columnIO.getRecordReader(
+                  pages, recordConverter, filterRecords ? 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");

Review comment:
       We need to print out the block number with the log message at least. Otherwise it would be hard to debug with this string constant message.

##########
File path: sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java
##########
@@ -230,14 +281,284 @@ public ReadFiles withAvroDataModel(GenericData model) {
       return toBuilder().setAvroDataModel(model).build();
     }
 
+    public ReadFiles withSplit() {
+      return toBuilder().setSplit(true).build();
+    }
+
     @Override
     public PCollection<GenericRecord> expand(PCollection<FileIO.ReadableFile> input) {
       checkNotNull(getSchema(), "Schema can not be null");
+      if (getSplit()) {
+        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);
+      private static final long SPLIT_LIMIT = 64000000;
+      ReadSupport<GenericRecord> readSupport;
+
+      SplitReadFn(GenericData model) {
+        this.modelClass = model != null ? model.getClass() : null;
+      }
+
+      private static <K, V> Map<K, Set<V>> toSetMultiMap(Map<K, V> map) {
+        Map<K, Set<V>> setMultiMap = new HashMap<K, Set<V>>();
+        for (Map.Entry<K, V> entry : map.entrySet()) {
+          Set<V> set = new HashSet<V>();
+          set.add(entry.getValue());
+          setMultiMap.put(entry.getKey(), Collections.unmodifiableSet(set));
+        }
+        return Collections.unmodifiableMap(setMultiMap);
+      }
+
+      public InputFile getInputFile(FileIO.ReadableFile file) throws IOException {
+        if (!file.getMetadata().isReadSeekEfficient()) {
+          throw new RuntimeException(
+              String.format("File has to be seekable: %s", file.getMetadata().resourceId()));
+        }
+        return new BeamParquetInputFile(file.openSeekable());
+      }
+
+      @ProcessElement
+      public void processElement(
+          @Element FileIO.ReadableFile file,
+          RestrictionTracker<OffsetRange, Long> tracker,
+          OutputReceiver<GenericRecord> outputReceiver)
+          throws Exception {
+        ReadSupport<GenericRecord> readSupport;
+        InputFile inputFile = getInputFile(file);
+        Configuration conf = setConf();
+        GenericData model = null;
+        if (modelClass != null) {
+          model = (GenericData) modelClass.getMethod("get").invoke(null);
+        }
+        readSupport = new AvroReadSupport<GenericRecord>(model);
+        ParquetReadOptions options = HadoopReadOptions.builder(conf).build();
+        ParquetFileReader reader = ParquetFileReader.open(inputFile, options);
+        Filter filter = checkNotNull(options.getRecordFilter(), "filter");
+        conf = ((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(conf, toSetMultiMap(fileMetadata), fileSchema));
+        ColumnIOFactory columnIOFactory = new ColumnIOFactory(parquetFileMetadata.getCreatedBy());
+        MessageType requestedSchema = readContext.getRequestedSchema();
+        RecordMaterializer<GenericRecord> recordConverter =
+            readSupport.prepareForRead(conf, fileMetadata, fileSchema, readContext);
+        boolean strictTypeChecking = options.isEnabled(STRICT_TYPE_CHECKING, true);
+        boolean filterRecords = options.useRecordFilter();
+        reader.setRequestedSchema(requestedSchema);
+        MessageColumnIO columnIO =
+            columnIOFactory.getColumnIO(requestedSchema, fileSchema, strictTypeChecking);
+        long currentBlock = tracker.currentRestriction().getFrom();
+        for (int i = 0; i < currentBlock; i++) {
+          reader.skipNextRowGroup();
+        }
+
+        while ((tracker).tryClaim(currentBlock)) {
+          LOG.info("reading block" + currentBlock);
+          PageReadStore pages = reader.readNextRowGroup();
+          currentBlock += 1;
+          RecordReader<GenericRecord> recordReader =
+              columnIO.getRecordReader(
+                  pages, recordConverter, filterRecords ? 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");

Review comment:
       ditto.

##########
File path: sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java
##########
@@ -230,14 +281,284 @@ public ReadFiles withAvroDataModel(GenericData model) {
       return toBuilder().setAvroDataModel(model).build();
     }
 
+    public ReadFiles withSplit() {
+      return toBuilder().setSplit(true).build();
+    }
+
     @Override
     public PCollection<GenericRecord> expand(PCollection<FileIO.ReadableFile> input) {
       checkNotNull(getSchema(), "Schema can not be null");
+      if (getSplit()) {
+        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);
+      private static final long SPLIT_LIMIT = 64000000;
+      ReadSupport<GenericRecord> readSupport;
+
+      SplitReadFn(GenericData model) {
+        this.modelClass = model != null ? model.getClass() : null;
+      }
+
+      private static <K, V> Map<K, Set<V>> toSetMultiMap(Map<K, V> map) {
+        Map<K, Set<V>> setMultiMap = new HashMap<K, Set<V>>();
+        for (Map.Entry<K, V> entry : map.entrySet()) {
+          Set<V> set = new HashSet<V>();
+          set.add(entry.getValue());
+          setMultiMap.put(entry.getKey(), Collections.unmodifiableSet(set));
+        }
+        return Collections.unmodifiableMap(setMultiMap);
+      }
+
+      public InputFile getInputFile(FileIO.ReadableFile file) throws IOException {
+        if (!file.getMetadata().isReadSeekEfficient()) {
+          throw new RuntimeException(
+              String.format("File has to be seekable: %s", file.getMetadata().resourceId()));
+        }
+        return new BeamParquetInputFile(file.openSeekable());
+      }
+
+      @ProcessElement
+      public void processElement(
+          @Element FileIO.ReadableFile file,
+          RestrictionTracker<OffsetRange, Long> tracker,
+          OutputReceiver<GenericRecord> outputReceiver)
+          throws Exception {
+        ReadSupport<GenericRecord> readSupport;
+        InputFile inputFile = getInputFile(file);
+        Configuration conf = setConf();
+        GenericData model = null;
+        if (modelClass != null) {
+          model = (GenericData) modelClass.getMethod("get").invoke(null);
+        }
+        readSupport = new AvroReadSupport<GenericRecord>(model);
+        ParquetReadOptions options = HadoopReadOptions.builder(conf).build();
+        ParquetFileReader reader = ParquetFileReader.open(inputFile, options);
+        Filter filter = checkNotNull(options.getRecordFilter(), "filter");
+        conf = ((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(conf, toSetMultiMap(fileMetadata), fileSchema));
+        ColumnIOFactory columnIOFactory = new ColumnIOFactory(parquetFileMetadata.getCreatedBy());
+        MessageType requestedSchema = readContext.getRequestedSchema();
+        RecordMaterializer<GenericRecord> recordConverter =
+            readSupport.prepareForRead(conf, fileMetadata, fileSchema, readContext);
+        boolean strictTypeChecking = options.isEnabled(STRICT_TYPE_CHECKING, true);
+        boolean filterRecords = options.useRecordFilter();
+        reader.setRequestedSchema(requestedSchema);
+        MessageColumnIO columnIO =
+            columnIOFactory.getColumnIO(requestedSchema, fileSchema, strictTypeChecking);
+        long currentBlock = tracker.currentRestriction().getFrom();
+        for (int i = 0; i < currentBlock; i++) {
+          reader.skipNextRowGroup();
+        }
+
+        while ((tracker).tryClaim(currentBlock)) {
+          LOG.info("reading block" + currentBlock);
+          PageReadStore pages = reader.readNextRowGroup();
+          currentBlock += 1;
+          RecordReader<GenericRecord> recordReader =
+              columnIO.getRecordReader(
+                  pages, recordConverter, filterRecords ? 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 read " + currentRow + " rows");
+        }
+      }
+
+      public Configuration setConf() 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 {
+        InputFile inputFile = getInputFile(file);
+        Configuration conf = setConf();
+        ParquetReadOptions options = HadoopReadOptions.builder(conf).build();
+        ParquetFileReader reader = ParquetFileReader.open(inputFile, options);
+        return new OffsetRange(0, reader.getRowGroups().size());
+      }
+
+      @SplitRestriction
+      public void split(
+          @Restriction OffsetRange restriction,
+          OutputReceiver<OffsetRange> out,
+          @Element FileIO.ReadableFile file)
+          throws Exception {
+        InputFile inputFile = getInputFile(file);
+        Configuration conf = setConf();
+        ParquetReadOptions options = HadoopReadOptions.builder(conf).build();
+        ParquetFileReader reader = ParquetFileReader.open(inputFile, options);
+        List<BlockMetaData> rowGroups = reader.getRowGroups();
+        for (OffsetRange offsetRange :
+            splitBlockWithLimit(
+                restriction.getFrom(), restriction.getTo(), rowGroups, SPLIT_LIMIT / 3)) {
+          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;

Review comment:
       How about renaming `i` to `rangeEnd` and removing the separate `rangeEnd` variable?

##########
File path: sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java
##########
@@ -230,14 +281,284 @@ public ReadFiles withAvroDataModel(GenericData model) {
       return toBuilder().setAvroDataModel(model).build();
     }
 
+    public ReadFiles withSplit() {
+      return toBuilder().setSplit(true).build();
+    }
+
     @Override
     public PCollection<GenericRecord> expand(PCollection<FileIO.ReadableFile> input) {
       checkNotNull(getSchema(), "Schema can not be null");
+      if (getSplit()) {
+        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);
+      private static final long SPLIT_LIMIT = 64000000;
+      ReadSupport<GenericRecord> readSupport;
+
+      SplitReadFn(GenericData model) {
+        this.modelClass = model != null ? model.getClass() : null;
+      }
+
+      private static <K, V> Map<K, Set<V>> toSetMultiMap(Map<K, V> map) {
+        Map<K, Set<V>> setMultiMap = new HashMap<K, Set<V>>();
+        for (Map.Entry<K, V> entry : map.entrySet()) {
+          Set<V> set = new HashSet<V>();
+          set.add(entry.getValue());
+          setMultiMap.put(entry.getKey(), Collections.unmodifiableSet(set));
+        }
+        return Collections.unmodifiableMap(setMultiMap);
+      }
+
+      public InputFile getInputFile(FileIO.ReadableFile file) throws IOException {
+        if (!file.getMetadata().isReadSeekEfficient()) {
+          throw new RuntimeException(
+              String.format("File has to be seekable: %s", file.getMetadata().resourceId()));
+        }
+        return new BeamParquetInputFile(file.openSeekable());
+      }
+
+      @ProcessElement
+      public void processElement(
+          @Element FileIO.ReadableFile file,
+          RestrictionTracker<OffsetRange, Long> tracker,
+          OutputReceiver<GenericRecord> outputReceiver)
+          throws Exception {
+        ReadSupport<GenericRecord> readSupport;
+        InputFile inputFile = getInputFile(file);
+        Configuration conf = setConf();
+        GenericData model = null;
+        if (modelClass != null) {
+          model = (GenericData) modelClass.getMethod("get").invoke(null);
+        }
+        readSupport = new AvroReadSupport<GenericRecord>(model);
+        ParquetReadOptions options = HadoopReadOptions.builder(conf).build();
+        ParquetFileReader reader = ParquetFileReader.open(inputFile, options);
+        Filter filter = checkNotNull(options.getRecordFilter(), "filter");
+        conf = ((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(conf, toSetMultiMap(fileMetadata), fileSchema));
+        ColumnIOFactory columnIOFactory = new ColumnIOFactory(parquetFileMetadata.getCreatedBy());
+        MessageType requestedSchema = readContext.getRequestedSchema();
+        RecordMaterializer<GenericRecord> recordConverter =
+            readSupport.prepareForRead(conf, fileMetadata, fileSchema, readContext);
+        boolean strictTypeChecking = options.isEnabled(STRICT_TYPE_CHECKING, true);
+        boolean filterRecords = options.useRecordFilter();
+        reader.setRequestedSchema(requestedSchema);
+        MessageColumnIO columnIO =
+            columnIOFactory.getColumnIO(requestedSchema, fileSchema, strictTypeChecking);
+        long currentBlock = tracker.currentRestriction().getFrom();
+        for (int i = 0; i < currentBlock; i++) {
+          reader.skipNextRowGroup();
+        }
+
+        while ((tracker).tryClaim(currentBlock)) {
+          LOG.info("reading block" + currentBlock);
+          PageReadStore pages = reader.readNextRowGroup();
+          currentBlock += 1;
+          RecordReader<GenericRecord> recordReader =
+              columnIO.getRecordReader(
+                  pages, recordConverter, filterRecords ? 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 read " + currentRow + " rows");
+        }
+      }
+
+      public Configuration setConf() 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 {
+        InputFile inputFile = getInputFile(file);
+        Configuration conf = setConf();
+        ParquetReadOptions options = HadoopReadOptions.builder(conf).build();
+        ParquetFileReader reader = ParquetFileReader.open(inputFile, options);
+        return new OffsetRange(0, reader.getRowGroups().size());
+      }
+
+      @SplitRestriction
+      public void split(
+          @Restriction OffsetRange restriction,
+          OutputReceiver<OffsetRange> out,
+          @Element FileIO.ReadableFile file)
+          throws Exception {
+        InputFile inputFile = getInputFile(file);
+        Configuration conf = setConf();
+        ParquetReadOptions options = HadoopReadOptions.builder(conf).build();
+        ParquetFileReader reader = ParquetFileReader.open(inputFile, options);
+        List<BlockMetaData> rowGroups = reader.getRowGroups();
+        for (OffsetRange offsetRange :
+            splitBlockWithLimit(
+                restriction.getFrom(), restriction.getTo(), rowGroups, SPLIT_LIMIT / 3)) {
+          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 {
+        return new BlockTracker(
+            restriction, (long) getSize(file, restriction), getRecordCount(file, restriction));
+      }
+
+      @GetRestrictionCoder
+      public OffsetRange.Coder getRestrictionCoder() {
+        return new OffsetRange.Coder();
+      }
+
+      public long getRecordCount(
+          @Element FileIO.ReadableFile file, @Restriction OffsetRange restriction)
+          throws Exception {
+        InputFile inputFile = getInputFile(file);
+        Configuration conf = setConf();
+        ParquetReadOptions options = HadoopReadOptions.builder(conf).build();
+        ParquetFileReader reader = ParquetFileReader.open(inputFile, options);
+        long start = 0;
+        long end = 0;
+        start = restriction.getFrom();
+        end = restriction.getTo();
+        long recordCount = 0;
+        for (long i = start; i < end; i++) {
+          recordCount += reader.getRowGroups().get((int) i).getRowCount();
+        }
+        return recordCount;
+      }
+
+      public double getSize(@Element FileIO.ReadableFile file, @Restriction OffsetRange restriction)
+          throws Exception {
+        InputFile inputFile = getInputFile(file);
+        Configuration conf = setConf();
+        ParquetReadOptions options = HadoopReadOptions.builder(conf).build();
+        ParquetFileReader reader = ParquetFileReader.open(inputFile, options);
+        double size = 0;
+        for (long i = restriction.getFrom(); i < restriction.getTo(); i++) {
+          size += reader.getRowGroups().get((int) i).getTotalByteSize();
+        }
+        return size;
+      }
+    }
+
+    public static class BlockTracker extends OffsetRangeTracker {
+      private static final Logger LOG = LoggerFactory.getLogger(BlockTracker.class);
+      private long totalWork;
+      private long progress;
+      private long approximateRecordSize;
+
+      public BlockTracker(OffsetRange range, long work, long recordCount) {
+        super(range);
+        if (recordCount != 0) {
+          this.approximateRecordSize = work / recordCount;
+          this.totalWork = approximateRecordSize * recordCount;

Review comment:
       Please put a comment then.




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