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 04:15:44 UTC

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

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



##########
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) {
+            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 (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.debug("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)) {
+          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 {
+        CountAndSize recordCountAndSize = getRecordCountAndSize(file, restriction);
+        return new BlockTracker(
+            restriction,
+            Math.round(recordCountAndSize.getSize()),
+            Math.round(recordCountAndSize.getCount()));
+      }
+
+      @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).getSize();
+      }
+
+      public CountAndSize getRecordCountAndSize(

Review comment:
       Yes, private should be used here.




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