-
Notifications
You must be signed in to change notification settings - Fork 4.5k
[BEAM-6394] Add support to write protobuf data using ProtoParquetReader #34063
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 2 commits
7edc73b
73712e6
2196798
9d3e8e4
6659033
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -64,6 +64,7 @@ | |
| import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; | ||
| import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; | ||
| import org.apache.hadoop.conf.Configuration; | ||
| import org.apache.hadoop.fs.Path; | ||
| import org.apache.parquet.HadoopReadOptions; | ||
| import org.apache.parquet.ParquetReadOptions; | ||
| import org.apache.parquet.avro.AvroParquetReader; | ||
|
|
@@ -73,6 +74,7 @@ | |
| import org.apache.parquet.filter2.compat.FilterCompat; | ||
| import org.apache.parquet.filter2.compat.FilterCompat.Filter; | ||
| import org.apache.parquet.hadoop.ParquetFileReader; | ||
| import org.apache.parquet.hadoop.ParquetReader; | ||
| import org.apache.parquet.hadoop.ParquetWriter; | ||
| import org.apache.parquet.hadoop.api.InitContext; | ||
| import org.apache.parquet.hadoop.api.ReadSupport; | ||
|
|
@@ -89,6 +91,8 @@ | |
| import org.apache.parquet.io.RecordReader; | ||
| import org.apache.parquet.io.SeekableInputStream; | ||
| import org.apache.parquet.io.api.RecordMaterializer; | ||
| import org.apache.parquet.proto.ProtoParquetReader; | ||
| import org.apache.parquet.proto.ProtoReadSupport; | ||
| import org.apache.parquet.schema.MessageType; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
@@ -263,6 +267,7 @@ public static ReadFiles readFiles(Schema schema) { | |
| return new AutoValue_ParquetIO_ReadFiles.Builder() | ||
| .setSchema(schema) | ||
| .setInferBeamSchema(false) | ||
| .setUseProtoReader(false) // NEW: default to false | ||
| .build(); | ||
| } | ||
|
|
||
|
|
@@ -616,6 +621,9 @@ public abstract static class ReadFiles | |
|
|
||
| abstract Builder toBuilder(); | ||
|
|
||
| // New: flag to indicate using ProtoParquetReader for protobuf data. | ||
| abstract boolean getUseProtoReader(); | ||
|
|
||
| @AutoValue.Builder | ||
| abstract static class Builder { | ||
| abstract Builder setSchema(Schema schema); | ||
|
|
@@ -630,6 +638,8 @@ abstract static class Builder { | |
|
|
||
| abstract Builder setInferBeamSchema(boolean inferBeamSchema); | ||
|
|
||
| abstract Builder setUseProtoReader(boolean useProtoReader); | ||
|
|
||
| abstract ReadFiles build(); | ||
| } | ||
|
|
||
|
|
@@ -663,6 +673,11 @@ public ReadFiles withBeamSchemas(boolean inferBeamSchema) { | |
| return toBuilder().setInferBeamSchema(inferBeamSchema).build(); | ||
| } | ||
|
|
||
| // New: method to opt into using ProtoParquetReader. | ||
| public ReadFiles withProtoReader() { | ||
| return toBuilder().setUseProtoReader(true).build(); | ||
| } | ||
|
|
||
| @Override | ||
| public PCollection<GenericRecord> expand(PCollection<ReadableFile> input) { | ||
| checkNotNull(getSchema(), "Schema can not be null"); | ||
|
|
@@ -673,7 +688,9 @@ public PCollection<GenericRecord> expand(PCollection<ReadableFile> input) { | |
| getAvroDataModel(), | ||
| getProjectionSchema(), | ||
| GenericRecordPassthroughFn.create(), | ||
| getConfiguration()))) | ||
| getConfiguration(), | ||
| getUseProtoReader() // New: pass the new flag here | ||
| ))) | ||
| .setCoder(getCollectionCoder()); | ||
| } | ||
|
|
||
|
|
@@ -686,7 +703,10 @@ public void populateDisplayData(DisplayData.Builder builder) { | |
| DisplayData.item("inferBeamSchema", getInferBeamSchema()) | ||
| .withLabel("Infer Beam Schema")) | ||
| .addIfNotNull(DisplayData.item("projectionSchema", String.valueOf(getProjectionSchema()))) | ||
| .addIfNotNull(DisplayData.item("avroDataModel", String.valueOf(getAvroDataModel()))); | ||
| .addIfNotNull(DisplayData.item("avroDataModel", String.valueOf(getAvroDataModel()))) | ||
| .add( | ||
| DisplayData.item("useProtoReader", getUseProtoReader()) | ||
| .withLabel("Use ProtoParquetReader")); | ||
| if (this.getConfiguration() != null) { | ||
| Configuration configuration = this.getConfiguration().get(); | ||
| for (Entry<String, String> entry : configuration) { | ||
|
|
@@ -718,16 +738,30 @@ static class SplitReadFn<T> extends DoFn<ReadableFile, T> { | |
|
|
||
| private final SerializableFunction<GenericRecord, T> parseFn; | ||
|
|
||
| private final boolean useProtoReader; // New: new flag for protobuf | ||
|
|
||
| SplitReadFn( | ||
| GenericData model, | ||
| Schema requestSchema, | ||
| SerializableFunction<GenericRecord, T> parseFn, | ||
| @Nullable SerializableConfiguration configuration) { | ||
| @Nullable SerializableConfiguration configuration, | ||
| boolean useProtoReader // New: add flag here | ||
| ) { | ||
|
|
||
| this.modelClass = model != null ? model.getClass() : null; | ||
| this.requestSchemaString = requestSchema != null ? requestSchema.toString() : null; | ||
| this.parseFn = checkNotNull(parseFn, "GenericRecord parse function can't be null"); | ||
| this.configuration = configuration; | ||
| this.useProtoReader = useProtoReader; // New: assign the flag | ||
| } | ||
|
|
||
| // New: Overloaded constructor for backward compatibility: | ||
| SplitReadFn( | ||
| GenericData model, | ||
| Schema requestSchema, | ||
| SerializableFunction<GenericRecord, T> parseFn, | ||
| @Nullable SerializableConfiguration configuration) { | ||
| this(model, requestSchema, parseFn, configuration, false); | ||
| } | ||
|
|
||
| private ParquetFileReader getParquetFileReader(ReadableFile file) throws Exception { | ||
|
|
@@ -746,6 +780,31 @@ public void processElement( | |
| tracker.currentRestriction().getFrom(), | ||
| tracker.currentRestriction().getTo()); | ||
| Configuration conf = getConfWithModelClass(); | ||
|
|
||
| if (useProtoReader) { | ||
| // Use ProtoParquetReader to read protobuf data. | ||
| // Derive a Hadoop Path from the file metadata. Adjust as needed. | ||
| Path path = new Path(file.getMetadata().resourceId().toString()); | ||
|
|
||
| // Get the configuration and set the property using the literal. | ||
| Configuration conf2 = getConfWithModelClass(); | ||
| conf2.set("parquet.proto.ignore.unknown.fields", "TRUE"); | ||
|
||
|
|
||
| // Use the builder overload that takes a ReadSupport and a Path. | ||
| try (ParquetReader<GenericRecord> reader = | ||
| ProtoParquetReader.<GenericRecord>builder(new ProtoReadSupport(), path).build()) { | ||
| GenericRecord message; | ||
| while ((message = reader.read()) != null) { | ||
| // Cast through Object so that parseFn (which expects GenericRecord) | ||
| // can accept the DynamicMessage. | ||
| outputReceiver.output(parseFn.apply((GenericRecord) (Object) message)); | ||
| } | ||
| } | ||
|
|
||
| return; // exit after using the proto path | ||
|
||
| } | ||
|
|
||
| // ELSE: existing logic using ParquetFileReader for Avro | ||
| GenericData model = null; | ||
| if (modelClass != null) { | ||
| model = (GenericData) modelClass.getMethod("get").invoke(null); | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
consider using a enum type parameter (AVRO / PROTO) and defaults to AVRO? This allows flexibility for future additions.