protected void checkTimestamp(WindowedValue<IN> ref, Instant timestamp) { if (timestamp.isBefore(ref.getTimestamp().minus(doFn.getAllowedTimestampSkew()))) { throw new IllegalArgumentException( String.format( "Cannot output with timestamp %s. Output timestamps must be no earlier than the " + "timestamp of the current input (%s) minus the allowed skew (%s). See the " + "DoFn#getAllowedTimestmapSkew() Javadoc for details on changing the allowed skew.", timestamp, ref.getTimestamp(), PeriodFormat.getDefault().print(doFn.getAllowedTimestampSkew().toPeriod()))); } }
@Override public void flatMap(WindowedValue<IN> value, Collector<WindowedValue<OUTFL>> out) throws Exception { this.initContext(doFn, out); // for each window the element belongs to, create a new copy here. Collection<? extends BoundedWindow> windows = value.getWindows(); if (windows.size() <= 1) { processElement(value); } else { for (BoundedWindow window : windows) { processElement( WindowedValue.of(value.getValue(), value.getTimestamp(), window, value.getPane())); } } }
public void injectElement(InputT value, Instant timestamp) throws Exception { Collection<W> windows = windowFn.assignWindows( new TriggerTester.StubAssignContext<W>( windowFn, value, timestamp, Arrays.asList(GlobalWindow.INSTANCE))); logInteraction( "Element %s at time %d put in windows %s", value, timestamp.getMillis(), windows); runner.processElement(WindowedValue.of(value, timestamp, windows, PaneInfo.NO_FIRING)); }
@Override public void outputWindowedValue( KV<String, OutputT> output, Instant timestamp, Collection<? extends BoundedWindow> windows, PaneInfo pane) { // Copy the output value (using coders) before capturing it. KV<String, OutputT> copy = SerializableUtils.<KV<String, OutputT>>ensureSerializableByCoder( KvCoder.of(StringUtf8Coder.of(), outputCoder), output, "outputForWindow"); WindowedValue<KV<String, OutputT>> value = WindowedValue.of(copy, timestamp, windows, pane); logInteraction("Outputting: %s", value); outputs.add(value); }
protected <T> WindowedValue<T> makeWindowedValue( T output, Instant timestamp, Collection<? extends BoundedWindow> windows, PaneInfo pane) { final Instant inputTimestamp = timestamp; final WindowFn windowFn = windowingStrategy.getWindowFn(); if (timestamp == null) { timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE; } if (windows == null) { try { windows = windowFn.assignWindows( windowFn.new AssignContext() { @Override public Object element() { throw new UnsupportedOperationException( "WindowFn attempted to access input element when none was available"); // TODO: 12/16/15 aljoscha's comment in slack } @Override public Instant timestamp() { if (inputTimestamp == null) { throw new UnsupportedOperationException( "WindowFn attempted to access input timestamp when none was available"); } return inputTimestamp; } @Override public Collection<? extends BoundedWindow> windows() { throw new UnsupportedOperationException( "WindowFn attempted to access input windows when none were available"); } }); } catch (Exception e) { Throwables.propagateIfInstanceOf(e, UserCodeException.class); throw new UserCodeException(e); } } return WindowedValue.of(output, timestamp, windows, pane); }
@Override public void processElement(ProcessContext c) throws Exception { final K key = c.element().getKey(); Iterator<WindowedValue<InputT>> iterator = c.element().getValue().iterator(); final PriorityQueue<W> liveWindows = new PriorityQueue<>( 11, new Comparator<BoundedWindow>() { @Override public int compare(BoundedWindow w1, BoundedWindow w2) { return Long.signum(w1.maxTimestamp().getMillis() - w2.maxTimestamp().getMillis()); } }); final Map<W, AccumT> accumulators = Maps.newHashMap(); final Map<W, Instant> minTimestamps = Maps.newHashMap(); WindowFn<Object, W>.MergeContext mergeContext = new CombiningMergeContext() { @Override public Collection<W> windows() { return liveWindows; } @Override public void merge(Collection<W> toBeMerged, W mergeResult) throws Exception { List<AccumT> accumsToBeMerged = new ArrayList<>(toBeMerged.size()); Instant minTimestamp = null; for (W window : toBeMerged) { accumsToBeMerged.add(accumulators.remove(window)); Instant timestampToBeMerged = minTimestamps.remove(window); if (minTimestamp == null || (timestampToBeMerged != null && timestampToBeMerged.isBefore(minTimestamp))) { minTimestamp = timestampToBeMerged; } } liveWindows.removeAll(toBeMerged); minTimestamps.put(mergeResult, minTimestamp); liveWindows.add(mergeResult); accumulators.put(mergeResult, combineFn.mergeAccumulators(key, accumsToBeMerged)); } }; while (iterator.hasNext()) { WindowedValue<InputT> e = iterator.next(); @SuppressWarnings("unchecked") Collection<W> windows = (Collection<W>) e.getWindows(); for (W w : windows) { Instant timestamp = minTimestamps.get(w); if (timestamp == null || timestamp.compareTo(e.getTimestamp()) > 0) { minTimestamps.put(w, e.getTimestamp()); } else { minTimestamps.put(w, timestamp); } AccumT accum = accumulators.get(w); checkState((timestamp == null && accum == null) || (timestamp != null && accum != null)); if (accum == null) { accum = combineFn.createAccumulator(key); liveWindows.add(w); } accum = combineFn.addInput(key, accum, e.getValue()); accumulators.put(w, accum); } windowFn.mergeWindows(mergeContext); while (!liveWindows.isEmpty() && liveWindows.peek().maxTimestamp().isBefore(e.getTimestamp())) { closeWindow(key, liveWindows.poll(), accumulators, minTimestamps, c); } } // To have gotten here, we've either not had any elements added, or we've only run merge // and then closed windows. We don't need to retry merging. while (!liveWindows.isEmpty()) { closeWindow(key, liveWindows.poll(), accumulators, minTimestamps, c); } }