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



##########
File path: 
sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java
##########
@@ -235,12 +284,151 @@ 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");
-      return input
-          .apply(ParDo.of(new ReadFn(getAvroDataModel())))
-          .setCoder(AvroCoder.of(getSchema()));
+      if (getSplit() != null) {
+        return input
+            .apply(ParDo.of(new SplitReadFn(getAvroDataModel())))
+            .setCoder(AvroCoder.of(getSchema()));
+      } else {
+        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);
+      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);
+      }
+
+      @ProcessElement
+      public void processElement(
+          @Element FileIO.ReadableFile file,
+          RestrictionTracker<OffsetRange, Long> tracker,
+          OutputReceiver<GenericRecord> outputReceiver)
+          throws Exception {
+        if (!file.getMetadata().isReadSeekEfficient()) {
+          ResourceId filename = file.getMetadata().resourceId();
+          throw new RuntimeException(String.format("File has to be seekable: 
%s", filename));
+        }
+
+        SeekableByteChannel seekableByteChannel = file.openSeekable();
+        ReadSupport<GenericRecord> readSupport;
+        InputFile inputFile = new BeamParquetInputFile(seekableByteChannel);
+        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();
+        for (String property : options.getPropertyNames()) {
+          conf.set(property, options.getProperty(property));
+        }
+        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);
+        for (int i = 0; i < reader.getRowGroups().size(); i++) {

Review comment:
       Ok, I have made the relevant change.




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