Skip to content

[BEAM-2624] Allow access to created filenames from WriteFiles#3573

Closed
reuvenlax wants to merge 9 commits intoapache:masterfrom
reuvenlax:fbs_results
Closed

[BEAM-2624] Allow access to created filenames from WriteFiles#3573
reuvenlax wants to merge 9 commits intoapache:masterfrom
reuvenlax:fbs_results

Conversation

@reuvenlax
Copy link
Copy Markdown
Contributor

@reuvenlax reuvenlax commented Jul 17, 2017

We use the new TypedWrite transforms to provide the new return type. Since these transforms have not yet been part of a Beam release (they are slated for 2.2), we can change their return types without worrying about backwards compatibility.

R: @kennknowles
R: @jkff

@reuvenlax
Copy link
Copy Markdown
Contributor Author

Adding @jkff as @kennknowles is on vacation right now.

Map<TupleTag<?>, PValue> outputs, WriteFilesResult newOutput) {
// We must connect the new output from WriteFilesResult to the outputs provided by the original
// transform.
Map.Entry<TupleTag<?>, PValue> original = Iterables.getOnlyElement(outputs.entrySet());
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think there exists a helper that does all this for you; take a look at other implementations of mapOutputs?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

import org.apache.beam.sdk.values.TupleTag;

/** The result of a {@link WriteFiles} transform. */
public class WriteFilesResult implements POutput {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  • It should probably be destination-dependent, i.e. PCollection<KV<DestinationT, String>>
  • Not a fan of the POutput boilerplate: why do we need a WriteFilesResult in general, why not just return a PCollection - is that because you want to make it forward-compatible?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is what we did in BigQueryIO for exactly that reason - it is forward compatible, and I can imagine wanting to return additional metadata in the future, as well as other things (e.g. dead-letter type accessors like in BigQueryIO).

Added a second getter to return KV<DestinationT, String>(simple use cases of sinks do not set a custom destination type, so want there to be a simple accessor)

public Void apply(Iterable<String> values) {
try {
String pattern = baseFilename.toString() + "*";
Iterable<String> matches =
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  • This will look better as a simple loop
  • You can put the creation of expected values into runTestWrite, and use regular PAssert.that(collection).contains(those values).

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

runTestWrite doesn't necessarily know the expected values a priori (e.g. if dynamic sharding is used). That's why we match the files written to FS.

Changed to a loop.

@reuvenlax
Copy link
Copy Markdown
Contributor Author

I just remembered this PR was pending. It changes an API introduced in 2.2.0, so we need to ensure this goes in before 2.2.0 is cut

@kennknowles
Copy link
Copy Markdown
Member

I would like to defer this to @jkff if possible; my impression is that review is well underway.

R: -@kennknowles

@reuvenlax reuvenlax force-pushed the fbs_results branch 2 times, most recently from 8f6e407 to 5c838f4 Compare August 31, 2017 21:30
@asfgit asfgit closed this in 7d53878 Sep 1, 2017
@reuvenlax reuvenlax deleted the fbs_results branch December 9, 2018 23:01
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants