Skip to content
Merged
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -29,56 +29,53 @@
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;

@RunWith(JUnit4.class)
public class MetadataPropagationTest {

@RunWith(JUnit4.class)
public static class MiscTest {
/** Tests for metadata propagation. */
@Rule public final transient TestPipeline pipeline = TestPipeline.create();

/** Tests for metadata propagation. */
@Rule public final transient TestPipeline pipeline = TestPipeline.create();

static class CausedByDrainSettingDoFn extends DoFn<Integer, String> {
@ProcessElement
public void process(OutputReceiver<String> r) {
r.builder("value").setCausedByDrain(CausedByDrain.CAUSED_BY_DRAIN).output();
}
static class CausedByDrainSettingDoFn extends DoFn<Integer, String> {
@ProcessElement
public void process(OutputReceiver<String> r) {
r.builder("value").setCausedByDrain(CausedByDrain.CAUSED_BY_DRAIN).output();
}
}

static class CausedByDrainExtractingDoFn extends DoFn<String, String> {
@ProcessElement
public void process(ProcessContext pc, OutputReceiver<String> r) {
r.output(pc.causedByDrain().toString());
}
static class CausedByDrainExtractingDoFn extends DoFn<String, String> {
@ProcessElement
public void process(ProcessContext pc, OutputReceiver<String> r) {
r.output(pc.causedByDrain().toString());
}
}

@Test
@Category(NeedsRunner.class)
public void testMetadataPropagationAcrossShuffleParameter() {
WindowedValues.WindowedValueCoder.setMetadataSupported();
PCollection<String> results =
pipeline
.apply(Create.of(1))
.apply(ParDo.of(new CausedByDrainSettingDoFn()))
.apply(Redistribute.arbitrarily())
.apply(ParDo.of(new CausedByDrainExtractingDoFn()));
@Test
@Category(NeedsRunner.class)
public void testMetadataPropagationAcrossShuffleParameter() {
WindowedValues.WindowedValueCoder.setMetadataSupported();
PCollection<String> results =
pipeline
.apply(Create.of(1))
.apply(ParDo.of(new CausedByDrainSettingDoFn()))
.apply(Redistribute.arbitrarily())
.apply(ParDo.of(new CausedByDrainExtractingDoFn()));

PAssert.that(results).containsInAnyOrder("CAUSED_BY_DRAIN");
PAssert.that(results).containsInAnyOrder("CAUSED_BY_DRAIN");

pipeline.run();
}
pipeline.run();
}

@Test
@Category(NeedsRunner.class)
public void testMetadataPropagationParameter() {
PCollection<String> results =
pipeline
.apply(Create.of(1))
.apply(ParDo.of(new CausedByDrainSettingDoFn()))
.apply(ParDo.of(new CausedByDrainExtractingDoFn()));
@Test
@Category(NeedsRunner.class)
public void testMetadataPropagationParameter() {
PCollection<String> results =
pipeline
.apply(Create.of(1))
.apply(ParDo.of(new CausedByDrainSettingDoFn()))
.apply(ParDo.of(new CausedByDrainExtractingDoFn()));

PAssert.that(results).containsInAnyOrder("CAUSED_BY_DRAIN");
PAssert.that(results).containsInAnyOrder("CAUSED_BY_DRAIN");

pipeline.run();
}
pipeline.run();
}
}
Loading