/** * Tests that the given {@link GroupAlsoByWindowsDoFn} implementation combines elements per * session window correctly according to the provided {@link CombineFn}. */ public static void combinesElementsPerSessionWithEndOfWindowTimestamp( GroupAlsoByWindowsDoFnFactory<String, Long, Long> gabwFactory, CombineFn<Long, ?, Long> combineFn) throws Exception { WindowingStrategy<?, IntervalWindow> windowingStrategy = WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10))) .withOutputTimeFn(OutputTimeFns.outputAtEndOfWindow()); BoundedWindow secondWindow = window(15, 25); List<WindowedValue<KV<String, Long>>> result = runGABW( gabwFactory, windowingStrategy, "k", WindowedValue.of(1L, new Instant(0), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), WindowedValue.of(2L, new Instant(5), Arrays.asList(window(5, 15)), PaneInfo.NO_FIRING), WindowedValue.of(4L, new Instant(15), Arrays.asList(secondWindow), PaneInfo.NO_FIRING)); assertThat(result, hasSize(2)); BoundedWindow firstResultWindow = window(0, 15); TimestampedValue<KV<String, Long>> item0 = getOnlyElementInWindow(result, firstResultWindow); assertThat(item0.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(1L, 2L)))); assertThat(item0.getTimestamp(), equalTo(firstResultWindow.maxTimestamp())); TimestampedValue<KV<String, Long>> item1 = getOnlyElementInWindow(result, secondWindow); assertThat(item1.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(4L)))); assertThat(item1.getTimestamp(), equalTo(secondWindow.maxTimestamp())); }
/** * Tests that the given GABW implementation correctly groups elements into merged sessions with * output timestamps at the end of the merged window. */ public static void groupsElementsInMergedSessionsWithLatestTimestamp( GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory) throws Exception { WindowingStrategy<?, IntervalWindow> windowingStrategy = WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10))) .withOutputTimeFn(OutputTimeFns.outputAtLatestInputTimestamp()); BoundedWindow unmergedWindow = window(15, 25); List<WindowedValue<KV<String, Iterable<String>>>> result = runGABW( gabwFactory, windowingStrategy, "k", WindowedValue.of( "v1", new Instant(0), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), WindowedValue.of( "v2", new Instant(5), Arrays.asList(window(5, 15)), PaneInfo.NO_FIRING), WindowedValue.of( "v3", new Instant(15), Arrays.asList(unmergedWindow), PaneInfo.NO_FIRING)); assertThat(result, hasSize(2)); BoundedWindow mergedWindow = window(0, 15); TimestampedValue<KV<String, Iterable<String>>> item0 = getOnlyElementInWindow(result, mergedWindow); assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2")); assertThat(item0.getTimestamp(), equalTo(new Instant(5))); TimestampedValue<KV<String, Iterable<String>>> item1 = getOnlyElementInWindow(result, unmergedWindow); assertThat(item1.getValue().getValue(), contains("v3")); assertThat(item1.getTimestamp(), equalTo(new Instant(15))); }
/** * Tests that for a simple sequence of elements on the same key, the given GABW implementation * correctly groups them according to fixed windows and also sets the output timestamp according * to the policy {@link OutputTimeFns#outputAtEndOfWindow()}. */ public static void groupsElementsIntoFixedWindowsWithEndOfWindowTimestamp( GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory) throws Exception { WindowingStrategy<?, IntervalWindow> windowingStrategy = WindowingStrategy.of(FixedWindows.of(Duration.millis(10))) .withOutputTimeFn(OutputTimeFns.outputAtEndOfWindow()); List<WindowedValue<KV<String, Iterable<String>>>> result = runGABW( gabwFactory, windowingStrategy, "key", WindowedValue.of( "v1", new Instant(1), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), WindowedValue.of( "v2", new Instant(2), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), WindowedValue.of( "v3", new Instant(13), Arrays.asList(window(10, 20)), PaneInfo.NO_FIRING)); assertThat(result, hasSize(2)); TimestampedValue<KV<String, Iterable<String>>> item0 = getOnlyElementInWindow(result, window(0, 10)); assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2")); assertThat(item0.getTimestamp(), equalTo(window(0, 10).maxTimestamp())); TimestampedValue<KV<String, Iterable<String>>> item1 = getOnlyElementInWindow(result, window(10, 20)); assertThat(item1.getValue().getValue(), contains("v3")); assertThat(item1.getTimestamp(), equalTo(window(10, 20).maxTimestamp())); }
/** * Tests that for a simple sequence of elements on the same key, the given GABW implementation * correctly groups and combines them according to sliding windows. * * <p>In the input here, each element occurs in multiple windows. */ public static void combinesElementsInSlidingWindows( GroupAlsoByWindowsDoFnFactory<String, Long, Long> gabwFactory, CombineFn<Long, ?, Long> combineFn) throws Exception { WindowingStrategy<?, IntervalWindow> windowingStrategy = WindowingStrategy.of(SlidingWindows.of(Duration.millis(20)).every(Duration.millis(10))) .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp()); List<WindowedValue<KV<String, Long>>> result = runGABW( gabwFactory, windowingStrategy, "k", WindowedValue.of( 1L, new Instant(5), Arrays.asList(window(-10, 10), window(0, 20)), PaneInfo.NO_FIRING), WindowedValue.of( 2L, new Instant(15), Arrays.asList(window(0, 20), window(10, 30)), PaneInfo.NO_FIRING), WindowedValue.of( 4L, new Instant(18), Arrays.asList(window(0, 20), window(10, 30)), PaneInfo.NO_FIRING)); assertThat(result, hasSize(3)); TimestampedValue<KV<String, Long>> item0 = getOnlyElementInWindow(result, window(-10, 10)); assertThat(item0.getValue().getKey(), equalTo("k")); assertThat(item0.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(1L)))); assertThat(item0.getTimestamp(), equalTo(new Instant(5L))); TimestampedValue<KV<String, Long>> item1 = getOnlyElementInWindow(result, window(0, 20)); assertThat(item1.getValue().getKey(), equalTo("k")); assertThat(item1.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(1L, 2L, 4L)))); // Timestamp adjusted by WindowFn to exceed the end of the prior sliding window assertThat(item1.getTimestamp(), equalTo(new Instant(10L))); TimestampedValue<KV<String, Long>> item2 = getOnlyElementInWindow(result, window(10, 30)); assertThat(item2.getValue().getKey(), equalTo("k")); assertThat(item2.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(2L, 4L)))); // Timestamp adjusted by WindowFn to exceed the end of the prior sliding window assertThat(item2.getTimestamp(), equalTo(new Instant(20L))); }
@SuppressWarnings("ConstantConditions") JavaDStream<WindowedValue<T>> getDStream() { if (dStream == null) { WindowedValue.ValueOnlyWindowedValueCoder<T> windowCoder = WindowedValue.getValueOnlyCoder(coder); // create the DStream from queue Queue<JavaRDD<WindowedValue<T>>> rddQueue = new LinkedBlockingQueue<>(); JavaRDD<WindowedValue<T>> lastRDD = null; for (Iterable<T> v : values) { Iterable<WindowedValue<T>> windowedValues = Iterables.transform(v, WindowingHelpers.<T>windowValueFunction()); JavaRDD<WindowedValue<T>> rdd = jssc.sc() .parallelize(CoderHelpers.toByteArrays(windowedValues, windowCoder)) .map(CoderHelpers.fromByteFunction(windowCoder)); rddQueue.offer(rdd); lastRDD = rdd; } // create DStream from queue, one at a time, // with last as default in case batches repeat (graceful stops for example). // if the stream is empty, avoid creating a default empty RDD. // mainly for unit test so no reason to have this configurable. dStream = lastRDD != null ? jssc.queueStream(rddQueue, true, lastRDD) : jssc.queueStream(rddQueue, true); } return dStream; }
@Override @SuppressWarnings("unchecked") public void translateNode( Flatten.FlattenPCollectionList<T> transform, FlinkBatchTranslationContext context) { List<TaggedPValue> allInputs = context.getInputs(transform); DataSet<WindowedValue<T>> result = null; if (allInputs.isEmpty()) { // create an empty dummy source to satisfy downstream operations // we cannot create an empty source in Flink, therefore we have to // add the flatMap that simply never forwards the single element DataSource<String> dummySource = context.getExecutionEnvironment().fromElements("dummy"); result = dummySource .flatMap( new FlatMapFunction<String, WindowedValue<T>>() { @Override public void flatMap(String s, Collector<WindowedValue<T>> collector) throws Exception { // never return anything } }) .returns( new CoderTypeInformation<>( WindowedValue.getFullCoder( (Coder<T>) VoidCoder.of(), GlobalWindow.Coder.INSTANCE))); } else { for (TaggedPValue taggedPc : allInputs) { checkArgument( taggedPc.getValue() instanceof PCollection, "Got non-PCollection input to flatten: %s of type %s", taggedPc.getValue(), taggedPc.getValue().getClass().getSimpleName()); PCollection<T> collection = (PCollection<T>) taggedPc.getValue(); DataSet<WindowedValue<T>> current = context.getInputDataSet(collection); if (result == null) { result = current; } else { result = result.union(current); } } } // insert a dummy filter, there seems to be a bug in Flink // that produces duplicate elements after the union in some cases // if we don't result = result .filter( new FilterFunction<WindowedValue<T>>() { @Override public boolean filter(WindowedValue<T> tWindowedValue) throws Exception { return true; } }) .name("UnionFixFilter"); context.setOutputDataSet(context.getOutput(transform), result); }
/** Fires any available timers. */ private void fireTimers() throws Exception { try { for (FiredTimers transformTimers : evaluationContext.extractFiredTimers()) { Collection<TimerData> delivery = transformTimers.getTimers(); KeyedWorkItem<?, Object> work = KeyedWorkItems.timersWorkItem(transformTimers.getKey().getKey(), delivery); @SuppressWarnings({"unchecked", "rawtypes"}) CommittedBundle<?> bundle = evaluationContext .createKeyedBundle( transformTimers.getKey(), (PCollection) transformTimers.getTransform().getInput()) .add(WindowedValue.valueInGlobalWindow(work)) .commit(evaluationContext.now()); scheduleConsumption( transformTimers.getTransform(), bundle, new TimerIterableCompletionCallback(delivery)); state.set(ExecutorState.ACTIVE); } } catch (Exception e) { LOG.error("Internal Error while delivering timers", e); throw e; } }
@Override protected void outputWithTimestampAndWindow( OutputT value, Instant timestamp, Collection<? extends BoundedWindow> windows, PaneInfo pane) { collector.collect(WindowedValue.of(value, timestamp, windows, pane)); }
@Override public void outputWindowedValue( KV<K, Iterable<V>> output, Instant timestamp, Collection<? extends BoundedWindow> windows, PaneInfo pane) { bundle.add(WindowedValue.of(output, timestamp, windows, pane)); }
/** * Tests that for a simple sequence of elements on the same key, the given GABW implementation * correctly groups them into sliding windows. * * <p>In the input here, each element occurs in multiple windows. */ public static void groupsElementsIntoSlidingWindowsWithMinTimestamp( GroupAlsoByWindowsDoFnFactory<String, String, Iterable<String>> gabwFactory) throws Exception { WindowingStrategy<?, IntervalWindow> windowingStrategy = WindowingStrategy.of(SlidingWindows.of(Duration.millis(20)).every(Duration.millis(10))) .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp()); List<WindowedValue<KV<String, Iterable<String>>>> result = runGABW( gabwFactory, windowingStrategy, "key", WindowedValue.of( "v1", new Instant(5), Arrays.asList(window(-10, 10), window(0, 20)), PaneInfo.NO_FIRING), WindowedValue.of( "v2", new Instant(15), Arrays.asList(window(0, 20), window(10, 30)), PaneInfo.NO_FIRING)); assertThat(result, hasSize(3)); TimestampedValue<KV<String, Iterable<String>>> item0 = getOnlyElementInWindow(result, window(-10, 10)); assertThat(item0.getValue().getValue(), contains("v1")); assertThat(item0.getTimestamp(), equalTo(new Instant(5))); TimestampedValue<KV<String, Iterable<String>>> item1 = getOnlyElementInWindow(result, window(0, 20)); assertThat(item1.getValue().getValue(), containsInAnyOrder("v1", "v2")); // Timestamp adjusted by WindowFn to exceed the end of the prior sliding window assertThat(item1.getTimestamp(), equalTo(new Instant(10))); TimestampedValue<KV<String, Iterable<String>>> item2 = getOnlyElementInWindow(result, window(10, 30)); assertThat(item2.getValue().getValue(), contains("v2")); // Timestamp adjusted by WindowFn to exceed the end of the prior sliding window assertThat(item2.getTimestamp(), equalTo(new Instant(20))); }
@SuppressWarnings("ConstantConditions") public JavaRDD<WindowedValue<T>> getRDD() { if (rdd == null) { WindowedValue.ValueOnlyWindowedValueCoder<T> windowCoder = WindowedValue.getValueOnlyCoder(coder); rdd = jsc.parallelize(CoderHelpers.toByteArrays(windowedValues, windowCoder)) .map(CoderHelpers.fromByteFunction(windowCoder)); } return rdd; }
@Override public void translateNode( ParDo.BoundMulti<InputT, OutputT> transform, FlinkBatchTranslationContext context) { DoFn<InputT, OutputT> doFn = transform.getFn(); rejectStateAndTimers(doFn); DataSet<WindowedValue<InputT>> inputDataSet = context.getInputDataSet(context.getInput(transform)); List<TaggedPValue> outputs = context.getOutputs(transform); Map<TupleTag<?>, Integer> outputMap = Maps.newHashMap(); // put the main output at index 0, FlinkMultiOutputDoFnFunction expects this outputMap.put(transform.getMainOutputTag(), 0); int count = 1; for (TaggedPValue taggedValue : outputs) { if (!outputMap.containsKey(taggedValue.getTag())) { outputMap.put(taggedValue.getTag(), count++); } } // assume that the windowing strategy is the same for all outputs WindowingStrategy<?, ?> windowingStrategy = null; // collect all output Coders and create a UnionCoder for our tagged outputs List<Coder<?>> outputCoders = Lists.newArrayList(); for (TaggedPValue taggedValue : outputs) { checkState( taggedValue.getValue() instanceof PCollection, "Within ParDo, got a non-PCollection output %s of type %s", taggedValue.getValue(), taggedValue.getValue().getClass().getSimpleName()); PCollection<?> coll = (PCollection<?>) taggedValue.getValue(); outputCoders.add(coll.getCoder()); windowingStrategy = coll.getWindowingStrategy(); } if (windowingStrategy == null) { throw new IllegalStateException("No outputs defined."); } UnionCoder unionCoder = UnionCoder.of(outputCoders); TypeInformation<WindowedValue<RawUnionValue>> typeInformation = new CoderTypeInformation<>( WindowedValue.getFullCoder( unionCoder, windowingStrategy.getWindowFn().windowCoder())); List<PCollectionView<?>> sideInputs = transform.getSideInputs(); // construct a map from side input to WindowingStrategy so that // the OldDoFn runner can map main-input windows to side input windows Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputStrategies = new HashMap<>(); for (PCollectionView<?> sideInput : sideInputs) { sideInputStrategies.put(sideInput, sideInput.getWindowingStrategyInternal()); } @SuppressWarnings("unchecked") FlinkMultiOutputDoFnFunction<InputT, OutputT> doFnWrapper = new FlinkMultiOutputDoFnFunction( doFn, windowingStrategy, sideInputStrategies, context.getPipelineOptions(), outputMap); MapPartitionOperator<WindowedValue<InputT>, WindowedValue<RawUnionValue>> taggedDataSet = new MapPartitionOperator<>( inputDataSet, typeInformation, doFnWrapper, transform.getName()); transformSideInputs(sideInputs, taggedDataSet, context); for (TaggedPValue output : outputs) { pruneOutput( taggedDataSet, context, outputMap.get(output.getTag()), (PCollection) output.getValue()); } }
@Override public void translateNode( GroupByKey<K, InputT> transform, FlinkBatchTranslationContext context) { // for now, this is copied from the Combine.PerKey translater. Once we have the new runner API // we can replace GroupByKey by a Combine.PerKey with the Concatenate CombineFn DataSet<WindowedValue<KV<K, InputT>>> inputDataSet = context.getInputDataSet(context.getInput(transform)); Combine.KeyedCombineFn<K, InputT, List<InputT>, List<InputT>> combineFn = new Concatenate<InputT>().asKeyedFn(); KvCoder<K, InputT> inputCoder = (KvCoder<K, InputT>) context.getInput(transform).getCoder(); Coder<List<InputT>> accumulatorCoder; try { accumulatorCoder = combineFn.getAccumulatorCoder( context.getInput(transform).getPipeline().getCoderRegistry(), inputCoder.getKeyCoder(), inputCoder.getValueCoder()); } catch (CannotProvideCoderException e) { throw new RuntimeException(e); } WindowingStrategy<?, ?> windowingStrategy = context.getInput(transform).getWindowingStrategy(); TypeInformation<WindowedValue<KV<K, List<InputT>>>> partialReduceTypeInfo = new CoderTypeInformation<>( WindowedValue.getFullCoder( KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder), windowingStrategy.getWindowFn().windowCoder())); Grouping<WindowedValue<KV<K, InputT>>> inputGrouping = inputDataSet.groupBy(new KvKeySelector<InputT, K>(inputCoder.getKeyCoder())); FlinkPartialReduceFunction<K, InputT, List<InputT>, ?> partialReduceFunction; FlinkReduceFunction<K, List<InputT>, List<InputT>, ?> reduceFunction; if (windowingStrategy.getWindowFn().isNonMerging()) { @SuppressWarnings("unchecked") WindowingStrategy<?, BoundedWindow> boundedStrategy = (WindowingStrategy<?, BoundedWindow>) windowingStrategy; partialReduceFunction = new FlinkPartialReduceFunction<>( combineFn, boundedStrategy, Collections.<PCollectionView<?>, WindowingStrategy<?, ?>>emptyMap(), context.getPipelineOptions()); reduceFunction = new FlinkReduceFunction<>( combineFn, boundedStrategy, Collections.<PCollectionView<?>, WindowingStrategy<?, ?>>emptyMap(), context.getPipelineOptions()); } else { if (!windowingStrategy.getWindowFn().windowCoder().equals(IntervalWindow.getCoder())) { throw new UnsupportedOperationException( "Merging WindowFn with windows other than IntervalWindow are not supported."); } @SuppressWarnings("unchecked") WindowingStrategy<?, IntervalWindow> intervalStrategy = (WindowingStrategy<?, IntervalWindow>) windowingStrategy; partialReduceFunction = new FlinkMergingPartialReduceFunction<>( combineFn, intervalStrategy, Collections.<PCollectionView<?>, WindowingStrategy<?, ?>>emptyMap(), context.getPipelineOptions()); reduceFunction = new FlinkMergingReduceFunction<>( combineFn, intervalStrategy, Collections.<PCollectionView<?>, WindowingStrategy<?, ?>>emptyMap(), context.getPipelineOptions()); } // Partially GroupReduce the values into the intermediate format AccumT (combine) GroupCombineOperator<WindowedValue<KV<K, InputT>>, WindowedValue<KV<K, List<InputT>>>> groupCombine = new GroupCombineOperator<>( inputGrouping, partialReduceTypeInfo, partialReduceFunction, "GroupCombine: " + transform.getName()); Grouping<WindowedValue<KV<K, List<InputT>>>> intermediateGrouping = groupCombine.groupBy(new KvKeySelector<List<InputT>, K>(inputCoder.getKeyCoder())); // Fully reduce the values and create output format VO GroupReduceOperator<WindowedValue<KV<K, List<InputT>>>, WindowedValue<KV<K, List<InputT>>>> outputDataSet = new GroupReduceOperator<>( intermediateGrouping, partialReduceTypeInfo, reduceFunction, transform.getName()); context.setOutputDataSet(context.getOutput(transform), outputDataSet); }