/** * Returns a new {@link Create.TimestampedValues} transform that produces a {@link PCollection} * containing the specified elements with the specified timestamps. * * <p>The arguments should not be modified after this is called. */ @SafeVarargs public static <T> TimestampedValues<T> timestamped( TimestampedValue<T> elem, @SuppressWarnings("unchecked") TimestampedValue<T>... elems) { return timestamped(ImmutableList.<TimestampedValue<T>>builder().add(elem).add(elems).build()); }
@Test public void testCreateGetName() { assertEquals("Create.Values", Create.of(1, 2, 3).getName()); assertEquals("Create.TimestampedValues", Create.timestamped(Collections.emptyList()).getName()); }
@Test @Category(NeedsRunner.class) public void testCreateTimestampedEmpty() { PCollection<String> output = p.apply( Create.timestamped(new ArrayList<TimestampedValue<String>>()) .withCoder(StringUtf8Coder.of())); PAssert.that(output).empty(); p.run(); }
@Test public void testCreateTimestampedEmptyUnspecifiedCoder() { p.enableAbandonedNodeEnforcement(false); thrown.expect(IllegalArgumentException.class); thrown.expectMessage("determine a default Coder"); thrown.expectMessage("Create.empty(Coder)"); thrown.expectMessage("Create.empty(TypeDescriptor)"); thrown.expectMessage("withCoder(Coder)"); thrown.expectMessage("withType(TypeDescriptor)"); p.apply(Create.timestamped(new ArrayList<>())); }
@Test public void testCreateTimestampedPolymorphicType() throws Exception { thrown.expect(RuntimeException.class); thrown.expectMessage(Matchers.containsString("Unable to infer a coder")); // Create won't infer a default coder in this case. PCollection<Record> c = p.apply( Create.timestamped( TimestampedValue.of(new Record(), new Instant(0)), TimestampedValue.of(new Record2(), new Instant(0)))); p.run(); throw new RuntimeException("Coder: " + c.getCoder()); }
@Test public void testApplyIsScopedToExactClass() throws IOException { DataflowPipelineOptions options = buildPipelineOptions(); Pipeline p = Pipeline.create(options); Create.TimestampedValues<String> transform = Create.timestamped(Arrays.asList(TimestampedValue.of("TestString", Instant.now()))); p.apply(transform); CompositeTransformRecorder recorder = new CompositeTransformRecorder(); p.traverseTopologically(recorder); // The recorder will also have seen a Create.Values composite as well, but we can't obtain that // transform. assertThat( "Expected to have seen CreateTimestamped composite transform.", recorder.getCompositeTransforms(), hasItem(transform)); assertThat( "Expected to have two composites, CreateTimestamped and Create.Values", recorder.getCompositeTransforms(), hasItem(Matchers.<PTransform<?, ?>>isA((Class) Create.Values.class))); }
@Test public void testCreateTimestampedDefaultOutputCoderUsingCoder() throws Exception { Coder<Record> coder = new RecordCoder(); Create.TimestampedValues<Record> values = Create.timestamped( TimestampedValue.of(new Record(), new Instant(0)), TimestampedValue.of(new Record2(), new Instant(0))) .withCoder(coder); assertThat(p.apply(values).getCoder(), equalTo(coder)); }
@Test @Category(ValidatesRunner.class) public void testMergingWindowing() { PCollection<String> input = p.apply( Create.timestamped( TimestampedValue.of("a", new Instant(1)), TimestampedValue.of("a", new Instant(5)), TimestampedValue.of("a", new Instant(20)))); PCollection<String> output = input.apply(new WindowedCount(Sessions.withGapDuration(new Duration(10)))); PAssert.that(output).containsInAnyOrder(output("a", 2, 1, 1, 15), output("a", 1, 20, 20, 30)); p.run(); }
@Test public void testCreateTimestampedDefaultOutputCoderUsingTypeDescriptor() throws Exception { Coder<Record> coder = new RecordCoder(); p.getCoderRegistry().registerCoderForClass(Record.class, coder); Create.TimestampedValues<Record> values = Create.timestamped( TimestampedValue.of(new Record(), new Instant(0)), TimestampedValue.of(new Record2(), new Instant(0))) .withType(new TypeDescriptor<Record>() {}); assertThat(p.apply(values).getCoder(), equalTo(coder)); }
@Test @Category(NeedsRunner.class) public void testCreateTimestamped() { List<TimestampedValue<String>> data = Arrays.asList( TimestampedValue.of("a", new Instant(1L)), TimestampedValue.of("b", new Instant(2L)), TimestampedValue.of("c", new Instant(3L))); PCollection<String> output = p.apply(Create.timestamped(data)).apply(ParDo.of(new PrintTimestamps())); PAssert.that(output).containsInAnyOrder("a:1", "b:2", "c:3"); p.run(); }
@Test @Category(NeedsRunner.class) public void testGloballyEventTimestamp() { PCollection<String> output = p.apply( Create.timestamped( TimestampedValue.of("foo", new Instant(100)), TimestampedValue.of("bar", new Instant(300)), TimestampedValue.of("baz", new Instant(200)))) .apply(Latest.globally()); PAssert.that(output).containsInAnyOrder("bar"); p.run(); }
@Test @Category(ValidatesRunner.class) public void testPartitioningWindowing() { PCollection<String> input = p.apply( Create.timestamped( TimestampedValue.of("a", new Instant(1)), TimestampedValue.of("b", new Instant(2)), TimestampedValue.of("b", new Instant(3)), TimestampedValue.of("c", new Instant(11)), TimestampedValue.of("d", new Instant(11)))); PCollection<String> output = input.apply(new WindowedCount(FixedWindows.of(new Duration(10)))); PAssert.that(output) .containsInAnyOrder( output("a", 1, 1, 0, 10), output("b", 2, 2, 0, 10), output("c", 1, 11, 10, 20), output("d", 1, 11, 10, 20)); p.run(); }
@Test @Category(NeedsRunner.class) public void timestampsSucceeds() { PCollection<String> timestamped = pipeline.apply( Create.timestamped( TimestampedValue.of("foo", new Instant(0L)), TimestampedValue.of("bar", new Instant(1L)))); PCollection<TimestampedValue<String>> reified = timestamped.apply(Reify.timestamps()); PAssert.that(reified) .containsInAnyOrder( TimestampedValue.of("foo", new Instant(0)), TimestampedValue.of("bar", new Instant(1))); pipeline.run(); }
@Test @Category(NeedsRunner.class) public void testPerKeyEventTimestamp() { PCollection<KV<String, String>> output = p.apply( Create.timestamped( TimestampedValue.of(KV.of("A", "foo"), new Instant(100)), TimestampedValue.of(KV.of("B", "bar"), new Instant(300)), TimestampedValue.of(KV.of("A", "baz"), new Instant(200)))) .apply(Latest.perKey()); PAssert.that(output).containsInAnyOrder(KV.of("B", "bar"), KV.of("A", "baz")); p.run(); }
@Test @Category(NeedsRunner.class) public void testSampleAny() { PCollection<Integer> input = pipeline .apply( Create.timestamped(ImmutableList.of(tv(0), tv(1), tv(2), tv(3), tv(4), tv(5))) .withCoder(BigEndianIntegerCoder.of())) .apply(Window.into(FixedWindows.of(Duration.standardSeconds(3)))); PCollection<Integer> output = input.apply(Sample.any(2)); PAssert.that(output) .inWindow(new IntervalWindow(new Instant(0), Duration.standardSeconds(3))) .satisfies(new VerifyCorrectSample<>(2, Arrays.asList(0, 1, 2))); PAssert.that(output) .inWindow(new IntervalWindow(new Instant(3000), Duration.standardSeconds(3))) .satisfies(new VerifyCorrectSample<>(2, Arrays.asList(3, 4, 5))); pipeline.run(); }
@Test @Category(ValidatesRunner.class) public void testTotalFlow() { PCollection<KV<String, Integer>> flow = pipeline .apply(Create.timestamped(TIME_STAMPED_INPUT)) .apply(ParDo.of(new ExtractFlowInfo())); PCollection<TableRow> totalFlow = flow.apply(Window.into(FixedWindows.of(Duration.standardMinutes(1)))) .apply(new TotalFlow("default")); PCollection<String> results = totalFlow.apply(ParDo.of(new FormatResults())); PAssert.that(results) .containsInAnyOrder(canonicalFormat(OUT_ROW_1), canonicalFormat(OUT_ROW_2)); pipeline.run().waitUntilFinish(); }
/** * Tests that when two elements are combined via a GroupByKey their output timestamp agrees with * the windowing function customized to use the latest value. */ @Test @Category(ValidatesRunner.class) public void testTimestampCombinerLatest() { p.apply( Create.timestamped( TimestampedValue.of(KV.of(0, "hello"), new Instant(0)), TimestampedValue.of(KV.of(0, "goodbye"), new Instant(10)))) .apply( Window.<KV<Integer, String>>into(FixedWindows.of(Duration.standardMinutes(10))) .withTimestampCombiner(TimestampCombiner.LATEST)) .apply(GroupByKey.create()) .apply(ParDo.of(new AssertTimestamp(new Instant(10)))); p.run(); }
/** * Tests that when two elements are combined via a GroupByKey their output timestamp agrees with * the windowing function customized to actually be the same as the default, the earlier of the * two values. */ @Test @Category(ValidatesRunner.class) public void testTimestampCombinerEarliest() { p.apply( Create.timestamped( TimestampedValue.of(KV.of(0, "hello"), new Instant(0)), TimestampedValue.of(KV.of(0, "goodbye"), new Instant(10)))) .apply( Window.<KV<Integer, String>>into(FixedWindows.of(Duration.standardMinutes(10))) .withTimestampCombiner(TimestampCombiner.EARLIEST)) .apply(GroupByKey.create()) .apply(ParDo.of(new AssertTimestamp(new Instant(0)))); p.run(); }
/** Basic test for {@code isEqualTo}. */ @Test @Category(ValidatesRunner.class) public void testWindowedIsEqualTo() throws Exception { PCollection<Integer> pcollection = pipeline .apply( Create.timestamped( TimestampedValue.of(43, new Instant(250L)), TimestampedValue.of(22, new Instant(-250L)))) .apply(Window.into(FixedWindows.of(Duration.millis(500L)))); PAssert.thatSingleton(pcollection) .inOnlyPane(new IntervalWindow(new Instant(0L), new Instant(500L))) .isEqualTo(43); PAssert.thatSingleton(pcollection) .inOnlyPane(new IntervalWindow(new Instant(-500L), new Instant(0L))) .isEqualTo(22); pipeline.run(); }
@Test @Category(NeedsRunner.class) public void testSampleAnyZero() { PCollection<Integer> input = pipeline.apply( Create.timestamped(ImmutableList.of(tv(0), tv(1), tv(2), tv(3), tv(4), tv(5))) .withCoder(BigEndianIntegerCoder.of())); PCollection<Integer> output = input .apply(Window.into(FixedWindows.of(Duration.standardSeconds(3)))) .apply(Sample.any(0)); PAssert.that(output) .inWindow(new IntervalWindow(new Instant(0), Duration.standardSeconds(3))) .satisfies(new VerifyCorrectSample<>(0, EMPTY)); PAssert.that(output) .inWindow(new IntervalWindow(new Instant(3000), Duration.standardSeconds(3))) .satisfies(new VerifyCorrectSample<>(0, EMPTY)); pipeline.run(); }