@Override public BroadcastKey read(ObjectDataInput in) throws IOException { return broadcastKey(in.readObject()); } };
@Override public void restoreFromSnapshot(@Nonnull Object key, @Nonnull Object value) { assert ((BroadcastKey) key).key().equals(Keys.LAST_EMITTED_WM) : "Unexpected key: " + key; // we restart at the oldest WM any instance was at at the time of snapshot minRestoredWm = Math.min(minRestoredWm, (long) value); }
/** * Returns a given key as a broadcast key. * <p> * Note: Several processor instances can use the returned {@code BroadcastKey} * with the same {@code key} to store unique values and the values will not * overwrite each other. Upon a snapshot restore, each processor * will receive multiple key-value pairs with the given BroadcastKey */ @Nonnull public static <K> BroadcastKey<K> broadcastKey(@Nonnull K key) { return new BroadcastKey<>(key); }
@Override public void write(ObjectDataOutput out, BroadcastKey object) throws IOException { out.writeObject(object.key()); }
@Override public boolean saveToSnapshot() { return tryEmitToSnapshot(broadcastKey(Keys.LAST_EMITTED_WM), eventTimeMapper.getWatermark(0)); }
@Override protected void restoreFromSnapshot(@Nonnull Object key, @Nonnull Object value) { @SuppressWarnings("unchecked") int partitionId = ((BroadcastKey<Integer>) key).key(); int partitionIndex = arrayIndexOf(partitionId, partitionIds); long offset = ((long[]) value)[0]; long wm = ((long[]) value)[1]; if (partitionIndex >= 0) { readOffsets[partitionIndex] = offset; emitOffsets[partitionIndex] = offset; // Always use partition index of 0, treating all the partitions the // same for coalescing purposes. eventTimeMapper.restoreWatermark(0, wm); } }
@Override public boolean saveToSnapshot() { if (!emitFromTraverser(traverser)) { return false; } if (snapshotTraverser == null) { Stream<Entry<BroadcastKey<TopicPartition>, long[]>> snapshotStream = offsets.entrySet().stream() .flatMap(entry -> IntStream.range(0, entry.getValue().length) .filter(partition -> entry.getValue()[partition] >= 0) .mapToObj(partition -> { TopicPartition key = new TopicPartition(entry.getKey(), partition); long offset = entry.getValue()[partition]; long watermark = eventTimeMapper.getWatermark(currentAssignment.get(key)); return entry(broadcastKey(key), new long[]{offset, watermark}); })); snapshotTraverser = traverseStream(snapshotStream) .onFirstNull(() -> { snapshotTraverser = null; if (getLogger().isFineEnabled()) { getLogger().fine("Finished saving snapshot." + " Saved offsets: " + offsets() + ", Saved watermarks: " + watermarks()); } }); } return emitFromTraverserToSnapshot(snapshotTraverser); }
@Override @SuppressWarnings("unchecked") protected void restoreFromSnapshot(@Nonnull Object key, @Nonnull Object value) { if (key instanceof BroadcastKey) { BroadcastKey bcastKey = (BroadcastKey) key; if (!Keys.CURRENT_WATERMARK.equals(bcastKey.key())) { throw new JetException("Unexpected broadcast key: " + bcastKey.key()); } long newCurrentWatermark = (long) value; assert processingGuarantee != EXACTLY_ONCE || minRestoredCurrentWatermark == Long.MAX_VALUE || minRestoredCurrentWatermark == newCurrentWatermark : "different values for currentWatermark restored, before=" + minRestoredCurrentWatermark + ", new=" + newCurrentWatermark; minRestoredCurrentWatermark = Math.min(newCurrentWatermark, minRestoredCurrentWatermark); return; } if (keyToWindows.put((K) key, (Windows) value) != null) { throw new JetException("Duplicate key in snapshot: " + key); } }
@Override public boolean saveToSnapshot() { if (!emitFromTraverser(traverser)) { return false; } if (snapshotTraverser == null) { Stream<Entry<BroadcastKey<TopicPartition>, long[]>> snapshotStream = offsets.entrySet().stream() .flatMap(entry -> IntStream.range(0, entry.getValue().length) .filter(partition -> entry.getValue()[partition] >= 0) .mapToObj(partition -> { TopicPartition key = new TopicPartition(entry.getKey(), partition); long offset = entry.getValue()[partition]; long watermark = watermarkSourceUtil.getWatermark(currentAssignment.get(key)); return entry(broadcastKey(key), new long[]{offset, watermark}); })); snapshotTraverser = traverseStream(snapshotStream) .onFirstNull(() -> { snapshotTraverser = null; if (getLogger().isFineEnabled()) { getLogger().fine("Finished saving snapshot." + " Saved offsets: " + offsets() + ", Saved watermarks: " + watermarks()); } }); } return emitFromTraverserToSnapshot(snapshotTraverser); }
@Override @SuppressWarnings("unchecked") protected void restoreFromSnapshot(@Nonnull Object key, @Nonnull Object value) { if (key instanceof BroadcastKey) { assert ((BroadcastKey) key).key().equals(Keys.LAST_EMITTED_WM) : "Unexpected key: " + key; // we restart at the oldest WM any instance was at at the time of snapshot minRestoredWm = Math.min(minRestoredWm, (long) value); return; } Tuple2<T, Integer> value1 = (Tuple2<T, Integer>) value; // we can't apply backpressure here, we have to store the items and execute them later for (int i = 0; i < value1.f1(); i++) { restoredObjects.add(value1.f0()); LoggingUtil.logFinest(getLogger(), "Restored: %s", value1.f0()); } }
@Override public boolean saveToSnapshot() { assert restoredObjects.isEmpty() : "restoredObjects not empty"; if (!emitFromTraverser(currentTraverser)) { return false; } if (snapshotTraverser == null) { LoggingUtil.logFinest(getLogger(), "Saving to snapshot: %s, lastReceivedWm=%d", inFlightItems, lastReceivedWm); snapshotTraverser = traverseIterable(inFlightItems.entrySet()) .<Entry>map(en -> entry( extractKeyFn.apply(en.getKey()), tuple2(en.getKey(), en.getValue()))) .append(entry(broadcastKey(Keys.LAST_EMITTED_WM), lastReceivedWm)) .onFirstNull(() -> snapshotTraverser = null); } return emitFromTraverserToSnapshot(snapshotTraverser); }
@Override public boolean saveToSnapshot() { if (!emitFromTraverser(traverser)) { return false; } if (snapshotTraverser == null) { snapshotTraverser = traverseStream(IntStream.range(0, partitionIds.length) .mapToObj(pIdx -> entry( broadcastKey(partitionIds[pIdx]), // Always use partition index of 0, treating all the partitions the // same for coalescing purposes. new long[] {emitOffsets[pIdx], eventTimeMapper.getWatermark(0)}))); } boolean done = emitFromTraverserToSnapshot(snapshotTraverser); if (done) { logFinest(getLogger(), "Saved snapshot. partitions=%s, offsets=%s, watermark=%d", Arrays.toString(partitionIds), Arrays.toString(emitOffsets), eventTimeMapper.getWatermark(0)); snapshotTraverser = null; } return done; }
@Override public void restoreFromSnapshot(@Nonnull Object key, @Nonnull Object value) { TopicPartition topicPartition = ((BroadcastKey<TopicPartition>) key).key(); long[] value1 = (long[]) value; long offset = value1[0]; long watermark = value1[1]; long[] topicOffsets = offsets.get(topicPartition.topic()); if (topicOffsets == null) { getLogger().warning("Offset for topic '" + topicPartition.topic() + "' is present in snapshot, but the topic is not supposed to be read"); return; } if (topicPartition.partition() >= topicOffsets.length) { getLogger().warning("Offset for partition '" + topicPartition + "' is present in snapshot," + " but that topic currently has only " + topicOffsets.length + " partitions"); } Integer partitionIndex = currentAssignment.get(topicPartition); if (partitionIndex != null) { assert topicOffsets[topicPartition.partition()] < 0 : "duplicate offset for topicPartition '" + topicPartition + "' restored, offset1=" + topicOffsets[topicPartition.partition()] + ", offset2=" + offset; topicOffsets[topicPartition.partition()] = offset; consumer.seek(topicPartition, offset + 1); eventTimeMapper.restoreWatermark(partitionIndex, watermark); } }
@SuppressWarnings("unchecked") @Override public boolean saveToSnapshot() { if (inComplete) { // If we are in completing phase, we can have a half-emitted item. Instead of finishing it and // writing a snapshot, we finish the final items and save no state. return complete(); } if (snapshotTraverser == null) { snapshotTraverser = Traversers.<Object>traverseIterable(keyToWindows.entrySet()) .append(entry(broadcastKey(Keys.CURRENT_WATERMARK), currentWatermark)) .onFirstNull(() -> snapshotTraverser = null); } return emitFromTraverserToSnapshot(snapshotTraverser); }
@Override public void restoreFromSnapshot(@Nonnull Object key, @Nonnull Object value) { TopicPartition topicPartition = ((BroadcastKey<TopicPartition>) key).key(); long[] value1 = (long[]) value; long offset = value1[0]; long watermark = value1[1]; long[] topicOffsets = offsets.get(topicPartition.topic()); if (topicOffsets == null) { getLogger().severe("Offset for topic '" + topicPartition.topic() + "' is present in snapshot, but the topic is not supposed to be read"); return; } if (topicPartition.partition() >= topicOffsets.length) { getLogger().severe("Offset for partition '" + topicPartition + "' is present in snapshot," + " but that topic currently has only " + topicOffsets.length + " partitions"); } Integer partitionIndex = currentAssignment.get(topicPartition); if (partitionIndex != null) { assert topicOffsets[topicPartition.partition()] < 0 : "duplicate offset for topicPartition '" + topicPartition + "' restored, offset1=" + topicOffsets[topicPartition.partition()] + ", offset2=" + offset; topicOffsets[topicPartition.partition()] = offset; consumer.seek(topicPartition, offset + 1); watermarkSourceUtil.restoreWatermark(partitionIndex, watermark); } }
@Override public boolean saveToSnapshot() { if (!isLastStage || flushTraverser != null) { return flushBuffers(); } if (snapshotTraverser == null) { snapshotTraverser = traverseIterable(tsToKeyToAcc.entrySet()) .<Entry>flatMap(e -> traverseIterable(e.getValue().entrySet()) .map(e2 -> entry(new SnapshotKey(e.getKey(), e2.getKey()), e2.getValue())) ) .append(entry(broadcastKey(Keys.NEXT_WIN_TO_EMIT), nextWinToEmit)) .onFirstNull(() -> { logFine(getLogger(), "Saved nextWinToEmit: %s", nextWinToEmit); snapshotTraverser = null; }); } return emitFromTraverserToSnapshot(snapshotTraverser); }