[
https://issues.apache.org/jira/browse/BEAM-11460?focusedWorklogId=528163&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-528163
]
ASF GitHub Bot logged work on BEAM-11460:
-----------------------------------------
Author: ASF GitHub Bot
Created on: 24/Dec/20 14:35
Start Date: 24/Dec/20 14:35
Worklog Time Spent: 10m
Work Description: iemejia commented on a change in pull request #13554:
URL: https://github.com/apache/beam/pull/13554#discussion_r548530734
##########
File path:
sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java
##########
@@ -152,6 +162,38 @@
* *
* }</pre>
*
+ * <h3>Reading records of an unknown schema</h3>
+ *
+ * <p>To read records from files whose schema is unknown at pipeline
construction time or differs
+ * between files, use {@link #parseGenericRecords(SerializableFunction)} - in
this case, you will
+ * need to specify a parsing function for converting each {@link
GenericRecord} into a value of your
+ * custom type.
+ *
+ * <p>For example:
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<Foo> records =
+ * p.apply(ParquetIO.parseGenericRecords(new
SerializableFunction<GenericRecord, Foo>() {
+ * public Foo apply(GenericRecord record) {
+ * // If needed, access the schema of the record using
record.getSchema()
+ * return ...;
+ * }}));
+ *
+ * // For reading from filepatterns
+ * PCollection<String> filepatterns = p.apply(...);
Review comment:
`PCollection<FileIO.ReadableFile> files = p.apply(...)`
##########
File path:
sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java
##########
@@ -838,6 +1031,23 @@ public void close() throws IOException {
}
}
+ /**
+ * Passthrough function to provide seamless backward compatibility to
ParquetIO's functionality.
+ */
+ @VisibleForTesting
+ static class GenericRecordPassthroughFn
+ implements SerializableFunction<GenericRecord, GenericRecord> {
+
+ static GenericRecordPassthroughFn create() {
+ return new GenericRecordPassthroughFn();
Review comment:
Can you please make this a singleton.
##########
File path:
sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java
##########
@@ -618,12 +807,16 @@ public Progress getProgress() {
}
}
- static class ReadFn extends DoFn<FileIO.ReadableFile, GenericRecord> {
+ static class ReadFn<T> extends DoFn<FileIO.ReadableFile, T> {
private Class<? extends GenericData> modelClass;
- ReadFn(GenericData model) {
+ private final SerializableFunction<GenericRecord, T> parseFn;
+
+ ReadFn(GenericData model, SerializableFunction<GenericRecord, T>
parseFn) {
+
Review comment:
nit: remove space
##########
File path:
sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java
##########
@@ -152,6 +162,38 @@
* *
* }</pre>
*
+ * <h3>Reading records of an unknown schema</h3>
+ *
+ * <p>To read records from files whose schema is unknown at pipeline
construction time or differs
+ * between files, use {@link #parseGenericRecords(SerializableFunction)} - in
this case, you will
+ * need to specify a parsing function for converting each {@link
GenericRecord} into a value of your
+ * custom type.
+ *
+ * <p>For example:
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<Foo> records =
+ * p.apply(ParquetIO.parseGenericRecords(new
SerializableFunction<GenericRecord, Foo>() {
+ * public Foo apply(GenericRecord record) {
+ * // If needed, access the schema of the record using
record.getSchema()
+ * return ...;
+ * }}));
+ *
+ * // For reading from filepatterns
+ * PCollection<String> filepatterns = p.apply(...);
+ *
+ * PCollection<Foo> records =
+ * filepatterns
Review comment:
files (for coherence with above doc)
##########
File path:
sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java
##########
@@ -152,6 +162,38 @@
* *
* }</pre>
*
+ * <h3>Reading records of an unknown schema</h3>
+ *
+ * <p>To read records from files whose schema is unknown at pipeline
construction time or differs
+ * between files, use {@link #parseGenericRecords(SerializableFunction)} - in
this case, you will
+ * need to specify a parsing function for converting each {@link
GenericRecord} into a value of your
+ * custom type.
+ *
+ * <p>For example:
+ *
+ * <pre>{@code
+ * Pipeline p = ...;
+ *
+ * PCollection<Foo> records =
+ * p.apply(ParquetIO.parseGenericRecords(new
SerializableFunction<GenericRecord, Foo>() {
+ * public Foo apply(GenericRecord record) {
+ * // If needed, access the schema of the record using
record.getSchema()
+ * return ...;
+ * }}));
+ *
+ * // For reading from filepatterns
+ * PCollection<String> filepatterns = p.apply(...);
+ *
+ * PCollection<Foo> records =
+ * filepatterns
+ * .apply(ParquetIO.parseFilesGenericRecords(new
SerializableFunction<GenericRecord, Foo>() {
+ * public Foo apply(GenericRecord record) {
+ * // If needed, access the schema of the record using
record.getSchema()
+ * return ...;
Review comment:
extra indent on comment + return
----------------------------------------------------------------
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]
Issue Time Tracking
-------------------
Worklog Id: (was: 528163)
Time Spent: 40m (was: 0.5h)
> Support reading Parquet files with unknown schema
> -------------------------------------------------
>
> Key: BEAM-11460
> URL: https://issues.apache.org/jira/browse/BEAM-11460
> Project: Beam
> Issue Type: New Feature
> Components: io-java-parquet
> Reporter: Anant Damle
> Priority: P1
> Labels: Parquet
> Original Estimate: 336h
> Time Spent: 40m
> Remaining Estimate: 0h
>
> Data engineers encounter times when schema of Parquet file is unknown at the
> time of writing the pipeline or multiple schema may be present in different
> files. Reading Parquet files using ParquetIO requires providing an Avro
> (equivalent) schema, Many a times its not possible to know the schema of the
> Parquet files.
> On the other hand
> [AvroIO|https://beam.apache.org/releases/javadoc/2.26.0/org/apache/beam/sdk/io/AvroIO.html]
> supports reading unknow schema files by providing a parse function :
> {{*#parseGenericRecords(SerializableFunction<GenericRecord,T>)*}}
> Supporting this functionality in ParquetIO is simple and requires minimal
> changes to the ParquetIO surface.
> {code}
> Pipeline p = ...;
> PCollection<String> filepatterns = p.apply(...);
> PCollection<Foo> records =
> filepatterns
> .apply(FileIO.matchAll())
> .apply(FileIO.readMatches())
> .apply(ParquetIO.parseGenericRecords(new
> SerializableFunction<GenericRecord, Foo>() {
> public Foo apply(GenericRecord record) {
> // If needed, access the schema of the record using
> record.getSchema()
> return ...;
> }
> }));
> {code}
--
This message was sent by Atlassian Jira
(v8.3.4#803005)