-
Notifications
You must be signed in to change notification settings - Fork 4.2k
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
[BEAM-6526] Add ReadFiles transform for AvroIO #7672
Changes from all commits
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 |
---|---|---|
|
@@ -297,6 +297,23 @@ public static <T> ReadAll<T> readAll(Class<T> recordClass) { | |
.build(); | ||
} | ||
|
||
/** | ||
* Like {@link #read}, but reads each file in a {@link PCollection} of {@link | ||
* FileIO.ReadableFile}, returned by {@link FileIO#readMatches}. | ||
*/ | ||
public static <T> ReadFiles<T> readFiles(Class<T> recordClass) { | ||
return new AutoValue_AvroIO_ReadFiles.Builder<T>() | ||
.setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.ALLOW_IF_WILDCARD)) | ||
.setRecordClass(recordClass) | ||
.setSchema(ReflectData.get().getSchema(recordClass)) | ||
.setInferBeamSchema(false) | ||
// 64MB is a reasonable value that allows to amortize the cost of opening files, | ||
// but is not so large as to exhaust a typical runner's maximum amount of output per | ||
// ProcessElement call. | ||
.setDesiredBundleSizeBytes(64 * 1024 * 1024L) | ||
.build(); | ||
} | ||
|
||
/** Reads Avro file(s) containing records of the specified schema. */ | ||
public static Read<GenericRecord> readGenericRecords(Schema schema) { | ||
return new AutoValue_AvroIO_Read.Builder<GenericRecord>() | ||
|
@@ -322,6 +339,17 @@ public static ReadAll<GenericRecord> readAllGenericRecords(Schema schema) { | |
.build(); | ||
} | ||
|
||
/** Like {@link #readGenericRecords(Schema)}, but for {@link FileIO.ReadableFile} collections. */ | ||
public static ReadFiles<GenericRecord> readFilesGenericRecords(Schema schema) { | ||
return new AutoValue_AvroIO_ReadFiles.Builder<GenericRecord>() | ||
.setMatchConfiguration(MatchConfiguration.create(EmptyMatchTreatment.ALLOW_IF_WILDCARD)) | ||
.setRecordClass(GenericRecord.class) | ||
.setSchema(schema) | ||
.setInferBeamSchema(false) | ||
.setDesiredBundleSizeBytes(64 * 1024 * 1024L) | ||
.build(); | ||
} | ||
|
||
/** | ||
* Reads Avro file(s) containing records of the specified schema. The schema is specified as a | ||
* JSON-encoded string. | ||
|
@@ -338,6 +366,11 @@ public static ReadAll<GenericRecord> readAllGenericRecords(String schema) { | |
return readAllGenericRecords(new Schema.Parser().parse(schema)); | ||
} | ||
|
||
/** Like {@link #readGenericRecords(String)}, but for {@link FileIO.ReadableFile} collections. */ | ||
public static ReadFiles<GenericRecord> readFilesGenericRecords(String schema) { | ||
return readFilesGenericRecords(new Schema.Parser().parse(schema)); | ||
} | ||
|
||
/** | ||
* Reads Avro file(s) containing records of an unspecified schema and converting each record to a | ||
* custom type. | ||
|
@@ -667,12 +700,7 @@ public PCollection<T> expand(PCollection<String> input) { | |
input | ||
.apply(FileIO.matchAll().withConfiguration(getMatchConfiguration())) | ||
.apply(FileIO.readMatches().withDirectoryTreatment(DirectoryTreatment.PROHIBIT)) | ||
.apply( | ||
"Read all via FileBasedSource", | ||
new ReadAllViaFileBasedSource<>( | ||
getDesiredBundleSizeBytes(), | ||
new CreateSourceFn<>(getRecordClass(), getSchema().toString()), | ||
AvroCoder.of(getRecordClass(), getSchema()))); | ||
.apply(readFiles(getRecordClass())); | ||
return getInferBeamSchema() ? setBeamSchema(read, getRecordClass(), getSchema()) : read; | ||
} | ||
|
||
|
@@ -705,6 +733,93 @@ public FileBasedSource<T> apply(String input) { | |
|
||
///////////////////////////////////////////////////////////////////////////// | ||
|
||
/** Implementation of {@link #readFiles}. */ | ||
@AutoValue | ||
public abstract static class ReadFiles<T> | ||
extends PTransform<PCollection<FileIO.ReadableFile>, PCollection<T>> { | ||
abstract MatchConfiguration getMatchConfiguration(); | ||
|
||
@Nullable | ||
abstract Class<T> getRecordClass(); | ||
|
||
@Nullable | ||
abstract Schema getSchema(); | ||
|
||
abstract long getDesiredBundleSizeBytes(); | ||
|
||
abstract boolean getInferBeamSchema(); | ||
|
||
abstract Builder<T> toBuilder(); | ||
|
||
@AutoValue.Builder | ||
abstract static class Builder<T> { | ||
abstract Builder<T> setMatchConfiguration(MatchConfiguration matchConfiguration); | ||
|
||
abstract Builder<T> setRecordClass(Class<T> recordClass); | ||
|
||
abstract Builder<T> setSchema(Schema schema); | ||
|
||
abstract Builder<T> setDesiredBundleSizeBytes(long desiredBundleSizeBytes); | ||
|
||
abstract Builder<T> setInferBeamSchema(boolean infer); | ||
|
||
abstract ReadFiles<T> build(); | ||
} | ||
|
||
/** Sets the {@link MatchConfiguration}. */ | ||
public ReadFiles<T> withMatchConfiguration(MatchConfiguration configuration) { | ||
return toBuilder().setMatchConfiguration(configuration).build(); | ||
} | ||
|
||
/** Like {@link Read#withEmptyMatchTreatment}. */ | ||
public ReadFiles<T> withEmptyMatchTreatment(EmptyMatchTreatment treatment) { | ||
return withMatchConfiguration(getMatchConfiguration().withEmptyMatchTreatment(treatment)); | ||
} | ||
|
||
/** Like {@link Read#watchForNewFiles}. */ | ||
@Experimental(Kind.SPLITTABLE_DO_FN) | ||
public ReadFiles<T> watchForNewFiles( | ||
Duration pollInterval, TerminationCondition<String, ?> terminationCondition) { | ||
return withMatchConfiguration( | ||
getMatchConfiguration().continuously(pollInterval, terminationCondition)); | ||
} | ||
|
||
@VisibleForTesting | ||
ReadFiles<T> withDesiredBundleSizeBytes(long desiredBundleSizeBytes) { | ||
return toBuilder().setDesiredBundleSizeBytes(desiredBundleSizeBytes).build(); | ||
} | ||
|
||
/** | ||
* If set to true, a Beam schema will be inferred from the AVRO schema. This allows the output | ||
* to be used by SQL and by the schema-transform library. | ||
*/ | ||
@Experimental(Kind.SCHEMAS) | ||
public ReadFiles<T> withBeamSchemas(boolean withBeamSchemas) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 👍 I like that. It seems to be possible to use it in ParquetIO in the same, right (asking for the sake of future PRs)? People seem to wish it frequently There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes, it makes sense for ParquetIO too. Worth to fill a JIRA IMO. Notice that I just created that method to be consistent with the existing There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. ok. fyi, there already is a JIRA: https://issues.apache.org/jira/browse/BEAM-4812. |
||
return toBuilder().setInferBeamSchema(withBeamSchemas).build(); | ||
} | ||
|
||
@Override | ||
public PCollection<T> expand(PCollection<FileIO.ReadableFile> input) { | ||
checkNotNull(getSchema(), "schema"); | ||
PCollection<T> read = | ||
input.apply( | ||
"Read all via FileBasedSource", | ||
new ReadAllViaFileBasedSource<>( | ||
getDesiredBundleSizeBytes(), | ||
new CreateSourceFn<>(getRecordClass(), getSchema().toString()), | ||
AvroCoder.of(getRecordClass(), getSchema()))); | ||
return getInferBeamSchema() ? setBeamSchema(read, getRecordClass(), getSchema()) : read; | ||
} | ||
|
||
@Override | ||
public void populateDisplayData(DisplayData.Builder builder) { | ||
super.populateDisplayData(builder); | ||
builder.include("matchConfiguration", getMatchConfiguration()); | ||
} | ||
} | ||
|
||
///////////////////////////////////////////////////////////////////////////// | ||
|
||
/** Implementation of {@link #parseGenericRecords}. */ | ||
@AutoValue | ||
public abstract static class Parse<T> extends PTransform<PBegin, PCollection<T>> { | ||
|
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.
Do you think it makes sense to extract a constant with the size and give the comment above it? Currently, multiple places use the same value (not only from here) but it's documented only in two of them.
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.
Yes good idea, I will extract the constant and move the doc there.