private void applicableTo(PCollection<?> input) { WindowingStrategy<?, ?> outputStrategy = getOutputStrategyInternal(input.getWindowingStrategy()); // Make sure that the windowing strategy is complete & valid. if (outputStrategy.isTriggerSpecified() && !(outputStrategy.getTrigger() instanceof DefaultTrigger) && !(outputStrategy.getWindowFn() instanceof GlobalWindows) && !outputStrategy.isAllowedLatenessSpecified()) { throw new IllegalArgumentException( "Except when using GlobalWindows," + " calling .triggering() to specify a trigger requires that the allowed lateness" + " be specified using .withAllowedLateness() to set the upper bound on how late" + " data can arrive before being dropped. See Javadoc for more details."); } if (!outputStrategy.isModeSpecified() && canProduceMultiplePanes(outputStrategy)) { throw new IllegalArgumentException( "Calling .triggering() to specify a trigger or calling .withAllowedLateness() to" + " specify an allowed lateness greater than zero requires that the accumulation" + " mode be specified using .discardingFiredPanes() or .accumulatingFiredPanes()." + " See Javadoc for more details."); } }
@Test public void testWindowIntoAccumulatingLatenessNoTrigger() { FixedWindows fixed = FixedWindows.of(Duration.standardMinutes(10)); WindowingStrategy<?, ?> strategy = pipeline .apply(Create.of("hello", "world").withCoder(StringUtf8Coder.of())) .apply( "Lateness", Window.<String>into(fixed) .withAllowedLateness(Duration.standardDays(1)) .accumulatingFiredPanes()) .getWindowingStrategy(); assertThat(strategy.isTriggerSpecified(), is(false)); assertThat(strategy.isModeSpecified(), is(true)); assertThat(strategy.isAllowedLatenessSpecified(), is(true)); assertThat(strategy.getMode(), equalTo(AccumulationMode.ACCUMULATING_FIRED_PANES)); assertThat(strategy.getAllowedLateness(), equalTo(Duration.standardDays(1))); }