Skip to content

Conversation

@reuvenlax
Copy link
Contributor

@reuvenlax reuvenlax commented May 16, 2017

This is similar to the fix for BEAM-2154.

R: @jkff

@coveralls
Copy link

Coverage Status

Coverage decreased (-0.06%) to 70.159% when pulling 675b4fa on reuvenlax:windowed_file_scalability into 529f251 on apache:master.

Copy link
Contributor

@jkff jkff left a comment

Choose a reason for hiding this comment

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

Thanks! Didn't look into the tests in too much detail yet.

new ViewOverrideFactory())) /* Uses pardos and GBKs */
ImmutableList.Builder<PTransformOverride> builder =
ImmutableList.<PTransformOverride>builder();
if (!options.isUnitTest()) {
Copy link
Contributor

Choose a reason for hiding this comment

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

I would prefer the other alternative, where the transform itself has an option whether or not a runner should be allowed to set fixed sharding on it (or more like, perhaps introduce a distinction between "no sharding" and "runner-chosen sharding", and use "no sharding" in the test). Having different behavior for test and non-test code is dangerous.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This approach was suggested by @tgroh, who felt that the transform should never be overridden in unit tests.

Copy link
Member

Choose a reason for hiding this comment

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

Specifically, tests executions for the sdk and runners-core modules (e.g. the validates-runner execution) don't need to modify any behavior of the original transforms, and I'd like to avoid having the transform specify how it is permitted to be overridden.

I would like to pull any test-related options out of DirectOptions and mark them as @Internal and @Hidden, however.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@tgroh where should the options go?

Copy link
Member

Choose a reason for hiding this comment

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

Either in a new DirectTestOptions or in TestPipelineOptions

Copy link
Member

Choose a reason for hiding this comment

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

If you add DirectTestOptions I believe you will have to register them in the DirectPipelineRegistrar as well

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done

* for each shard have been collected into a single iterable.
*/
private class WriteShardedBundles extends DoFn<KV<Integer, Iterable<T>>, FileResult> {
boolean setWindowedShardNumber;
Copy link
Contributor

Choose a reason for hiding this comment

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

Rename the variable to something more clear?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

made an enum

Writer<T> writer = writeOperation.createWriter();
if (windowedWrites) {
writer.openWindowed(UUID.randomUUID().toString(), window, c.pane(), c.element().getKey());
int shardNumber = setWindowedShardNumber ? c.element().getKey() : UNKNOWN_SHARDNUM;
Copy link
Contributor

Choose a reason for hiding this comment

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

And add a comment here

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It's more clear with an enum now, and a detailed comment was added elsewhere

"WriteBundles",
ParDo.of(windowedWrites ? new WriteWindowedBundles() : new WriteUnwindowedBundles()));
if (windowedWrites) {
TupleTag<FileResult> writtenRecordsTag = new TupleTag<FileResult>() {};
Copy link
Contributor

Choose a reason for hiding this comment

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

Note: these anonymous TupleTag's capture the enclosing class. Might want to use the TupleTag(String) constructor instead.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

writeTuple
.get(unwrittedRecordsTag)
.apply("GroupUnwritten", GroupByKey.<Integer, T>create())
.apply("WriteUnwritten", ParDo.of(new WriteShardedBundles(false)))
Copy link
Contributor

Choose a reason for hiding this comment

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

Instead of a boolean, maybe better to use an enum with eloquently named values

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

private SimpleSink makeSimpleSink(boolean windowedWrites) {
FilenamePolicy filenamePolicy = new PerWindowFiles("file", "simple");
return new SimpleSink(getBaseOutputDirectory(), filenamePolicy);
// return new SimpleSink(getBaseOutputDirectory(), "file", "-SS-of-NN", "simple");
Copy link
Contributor

Choose a reason for hiding this comment

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

Remove commented-out code

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

public void testEmptyWrite() throws IOException {
runWrite(Collections.<String>emptyList(), IDENTITY_MAP, getBaseOutputFilename());
runWrite(Collections.<String>emptyList(), IDENTITY_MAP, getBaseOutputFilename(),
Optional.<Integer>absent(), false);
Copy link
Contributor

Choose a reason for hiding this comment

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

false /* windowedWrites */

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I forget not everyone uses IntelliJ :)

}
private SimpleSink makeSimpleSink() {
return new SimpleSink(getBaseOutputDirectory(), "file", "-SS-of-NN", "simple");
private SimpleSink makeSimpleSink(boolean windowedWrites) {
Copy link
Contributor

Choose a reason for hiding this comment

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

This is not using windowedWrites.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Fixed

}

SimpleSink sink = makeSimpleSink();
SimpleSink sink = makeSimpleSink(windowedWrites);
Copy link
Contributor

Choose a reason for hiding this comment

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

I think it will be easier to read all the tests if, instead of passing the parameters such as numConfiguredShards, windowedWrites etc., you make runShardedWrite take a WriteFiles instance directly.

It's easier to read WriteFiles.to(makeSimpleSink(true) /* windowedWrites */).withMaxNumWritersPerBundle(42) than "..., true, Optional.of(42)".

@coveralls
Copy link

Coverage Status

Coverage increased (+0.5%) to 70.679% when pulling a5cdacf on reuvenlax:windowed_file_scalability into 529f251 on apache:master.

@reuvenlax
Copy link
Contributor Author

retest this please.

@coveralls
Copy link

Coverage Status

Coverage increased (+0.5%) to 70.712% when pulling a5cdacf on reuvenlax:windowed_file_scalability into 529f251 on apache:master.

@reuvenlax
Copy link
Contributor Author

retest this please.

@coveralls
Copy link

Coverage Status

Coverage increased (+0.4%) to 70.585% when pulling a5cdacf on reuvenlax:windowed_file_scalability into 529f251 on apache:master.

@coveralls
Copy link

Coverage Status

Coverage increased (+0.4%) to 70.628% when pulling 597f8a8 on reuvenlax:windowed_file_scalability into 529f251 on apache:master.

Copy link
Contributor

@jkff jkff left a comment

Choose a reason for hiding this comment

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

Thanks!

PTransformMatchers.classEqualTo(CreatePCollectionView.class),
new ViewOverrideFactory())) /* Uses pardos and GBKs */
TestPipelineOptions testOptions = options.as(TestPipelineOptions.class);
ImmutableList.Builder<PTransformOverride> builder =
Copy link
Contributor

Choose a reason for hiding this comment

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

Reindent

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done

*/
public WriteFiles<T> withNumShards(ValueProvider<Integer> numShardsProvider) {
return new WriteFiles<>(sink, null, numShardsProvider, windowedWrites);
return new WriteFiles<>(sink, null, numShardsProvider, windowedWrites,
Copy link
Contributor

Choose a reason for hiding this comment

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

might be time to convert this class to AutoValue (but feel free to ignore)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I think best left for another PR

windowedWriters.put(key, writer);
LOG.debug("Done opening writer");
} else {
c.output(unwrittedRecordsTag, KV.of(
Copy link
Contributor

Choose a reason for hiding this comment

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

nitpick: might make sense to put this (attaching a sharding key) into its own DoFn, and here output simply to a TupleTag<T>. That DoFn can be implemented more efficiently, without generating a random number for every value (you can generate a single number in startBundle or even setup(), and then keep incrementing it for every element - this has the same statistical properties).

Also feel free to ignore.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Think making it a separate DoFn is overkill, but the suggestion of caching the random number is a good one (and what we do elsewhere)

ParDo.of(windowedWrites ? new WriteWindowedBundles() : new WriteUnwindowedBundles()));
if (windowedWrites) {
TupleTag<FileResult> writtenRecordsTag =
new TupleTag<FileResult>("writtenRecordsTag") {};
Copy link
Contributor

Choose a reason for hiding this comment

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

These anonymous TupleTag classes capture the enclosing transform so they'll blow up the serialized graph unnecessarily. I think the code will work even if you remove {} - if it won't, you'll probably just need to specify some coders, because all the {} does is capture the type information.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done

@Internal
@Hidden
@org.apache.beam.sdk.options.Description(
"Indicates whether this is an automatically-run unit test.")
Copy link
Contributor

Choose a reason for hiding this comment

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

...as opposed to? (what else is TestPipelineOptions used for, if not unit tests?)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

"automatically run"

IDENTITY_MAP,
getBaseOutputFilename(),
Optional.of(20));
Optional.of(20),
Copy link
Contributor

Choose a reason for hiding this comment

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

I'm still not super happy about readability here due to how many parameters there are, and how easy they are to confuse. What do you think about my suggestion to pass WriteFiles directly, so that the call site contains a builder call and it's obvious what parameters are configured with what values?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done

@aaltay
Copy link
Member

aaltay commented Jun 8, 2017

Hey @reuvenlax any updates on this PR?

@reuvenlax reuvenlax force-pushed the windowed_file_scalability branch from 597f8a8 to bce2c12 Compare June 8, 2017 20:18
@reuvenlax
Copy link
Contributor Author

@jkff comments are addressed.

@coveralls
Copy link

Coverage Status

Coverage decreased (-0.09%) to 70.502% when pulling c854d34 on reuvenlax:windowed_file_scalability into 4aef938 on apache:master.

@coveralls
Copy link

Coverage Status

Coverage decreased (-0.08%) to 70.509% when pulling c854d34 on reuvenlax:windowed_file_scalability into 4aef938 on apache:master.

@jkff
Copy link
Contributor

jkff commented Jun 13, 2017

Run Dataflow ValidatesRunner

@reuvenlax
Copy link
Contributor Author

The dataflow test failure seems unrelated to this change.

@reuvenlax
Copy link
Contributor Author

Run Dataflow ValidatesRunner

@reuvenlax
Copy link
Contributor Author

Again fails in CombineTest.testSimpleCombineWithContext on Dataflow Runner. I see no way this could be related.

@reuvenlax
Copy link
Contributor Author

Run Dataflow ValidatesRunner

@asfgit asfgit closed this in 698b89e Jun 20, 2017
@reuvenlax reuvenlax deleted the windowed_file_scalability 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.

5 participants