-
Notifications
You must be signed in to change notification settings - Fork 4.5k
[BEAM-121] Add DisplayData for IO transforms #123
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 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 |
|---|---|---|
|
|
@@ -20,6 +20,7 @@ | |
| import org.apache.beam.sdk.annotations.Experimental; | ||
| import org.apache.beam.sdk.coders.Coder; | ||
| import org.apache.beam.sdk.options.PipelineOptions; | ||
| import org.apache.beam.sdk.transforms.display.DisplayData; | ||
|
|
||
| import com.google.common.base.Preconditions; | ||
| import com.google.common.io.ByteStreams; | ||
|
|
@@ -320,6 +321,22 @@ public final boolean producesSortedKeys(PipelineOptions options) throws Exceptio | |
| return sourceDelegate.producesSortedKeys(options); | ||
| } | ||
|
|
||
| @Override | ||
| public void populateDisplayData(DisplayData.Builder builder) { | ||
|
Contributor
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. // We explicitly do not include the parent data, instead we use the delegate inner source.or something like that. |
||
| // We explicitly do not register base-class data, instead we use the delegate inner source. | ||
| builder | ||
| .include(sourceDelegate) | ||
| .add("source", sourceDelegate.getClass()); | ||
|
|
||
| if (channelFactory instanceof Enum) { | ||
| // GZIP and BZIP are implemented as enums; Enum classes are anonymous, so use the .name() | ||
| // value instead | ||
| builder.add("compressionMode", ((Enum) channelFactory).name()); | ||
| } else { | ||
| builder.add("compressionMode", channelFactory.getClass()); | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Returns the delegate source's default output coder. | ||
| */ | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -36,6 +36,7 @@ | |
| import org.apache.beam.sdk.io.Sink.Writer; | ||
| import org.apache.beam.sdk.options.GcpOptions; | ||
| import org.apache.beam.sdk.options.PipelineOptions; | ||
| import org.apache.beam.sdk.transforms.display.DisplayData; | ||
| import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; | ||
| import org.apache.beam.sdk.util.RetryHttpRequestInitializer; | ||
| import org.apache.beam.sdk.values.PCollection; | ||
|
|
@@ -386,6 +387,19 @@ public long getEstimatedSizeBytes(PipelineOptions options) throws Exception { | |
| return getPropertyMap(entity).get("entity_bytes").getIntegerValue(); | ||
| } | ||
|
|
||
| @Override | ||
| public void populateDisplayData(DisplayData.Builder builder) { | ||
| super.populateDisplayData(builder); | ||
| builder | ||
| .addIfNotDefault("host", host, DEFAULT_HOST) | ||
| .addIfNotNull("dataset", datasetId) | ||
| .addIfNotNull("namespace", namespace); | ||
|
|
||
| if (query != null) { | ||
| builder.add("query", query.toString()); | ||
|
Contributor
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. is
Contributor
Author
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. We handle a small set of explicit types for display data (string, integer, floating point, java class, duration, timestamp, boolean), and we have We chose not to add a catch-all for |
||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public String toString() { | ||
| return MoreObjects.toStringHelper(getClass()) | ||
|
|
@@ -587,6 +601,13 @@ public void validate(PipelineOptions options) { | |
| public DatastoreWriteOperation createWriteOperation(PipelineOptions options) { | ||
| return new DatastoreWriteOperation(this); | ||
| } | ||
|
|
||
| @Override | ||
| public void populateDisplayData(DisplayData.Builder builder) { | ||
| builder | ||
| .addIfNotDefault("host", host, DEFAULT_HOST) | ||
| .addIfNotNull("dataset", datasetId); | ||
| } | ||
| } | ||
|
|
||
| /** | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -21,6 +21,7 @@ | |
| import org.apache.beam.sdk.coders.SerializableCoder; | ||
| import org.apache.beam.sdk.options.GcsOptions; | ||
| import org.apache.beam.sdk.options.PipelineOptions; | ||
| import org.apache.beam.sdk.transforms.display.DisplayData; | ||
| import org.apache.beam.sdk.util.FileIOChannelFactory; | ||
| import org.apache.beam.sdk.util.GcsIOChannelFactory; | ||
| import org.apache.beam.sdk.util.IOChannelFactory; | ||
|
|
@@ -135,6 +136,30 @@ public void validate(PipelineOptions options) {} | |
| @Override | ||
| public abstract FileBasedWriteOperation<T> createWriteOperation(PipelineOptions options); | ||
|
|
||
| @Override | ||
| public void populateDisplayData(DisplayData.Builder builder) { | ||
| super.populateDisplayData(builder); | ||
|
|
||
| String fileNamePattern = String.format("%s%s%s", | ||
|
Contributor
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. add (For people who copy and paste later.) |
||
| baseOutputFilename, fileNamingTemplate, getFileExtension(extension)); | ||
| builder.add("fileNamePattern", fileNamePattern); | ||
| } | ||
|
|
||
| /** | ||
| * Returns the file extension to be used. If the user did not request a file | ||
| * extension then this method returns the empty string. Otherwise this method | ||
| * adds a {@code "."} to the beginning of the users extension if one is not present. | ||
| */ | ||
| private static String getFileExtension(String usersExtension) { | ||
| if (usersExtension == null || usersExtension.isEmpty()) { | ||
| return ""; | ||
| } | ||
| if (usersExtension.startsWith(".")) { | ||
| return usersExtension; | ||
| } | ||
| return "." + usersExtension; | ||
| } | ||
|
|
||
| /** | ||
| * Abstract {@link Sink.WriteOperation} that manages the process of writing to a | ||
| * {@link FileBasedSink}. | ||
|
|
@@ -356,21 +381,6 @@ protected final List<String> generateDestinationFilenames(int numFiles) { | |
| return destFilenames; | ||
| } | ||
|
|
||
| /** | ||
| * Returns the file extension to be used. If the user did not request a file | ||
| * extension then this method returns the empty string. Otherwise this method | ||
| * adds a {@code "."} to the beginning of the users extension if one is not present. | ||
| */ | ||
| private String getFileExtension(String usersExtension) { | ||
| if (usersExtension == null || usersExtension.isEmpty()) { | ||
| return ""; | ||
| } | ||
| if (usersExtension.startsWith(".")) { | ||
| return usersExtension; | ||
| } | ||
| return "." + usersExtension; | ||
| } | ||
|
|
||
| /** | ||
| * Removes temporary output files. Uses the temporary filename to find files to remove. | ||
| * | ||
|
|
||
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.
probably similar to
CompressedSource.