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



##########
File path: 
sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java
##########
@@ -229,15 +279,259 @@ 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 =
+            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.debug("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) {

Review comment:
       I have tried that once and it might have some issue when reading out of 
the range, the original ParquetReader also tracked the row number when reading.




----------------------------------------------------------------
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:
us...@infra.apache.org


Reply via email to