public InsertWatermarksP(EventTimePolicy<? super T> eventTimePolicy) { eventTimeMapper = new EventTimeMapper<>(eventTimePolicy); eventTimeMapper.increasePartitionCount(1); }
/** * Call this method when there is no event coming. It returns a traverser * with 0 or 1 object (the watermark). */ @Nonnull public Traverser<Object> flatMapIdle() { return flatMapEvent(System.nanoTime(), null, -1, NO_NATIVE_TIME); }
private Map<TopicPartition, Long> watermarks() { return currentAssignment.entrySet().stream() .collect(Collectors.toMap(Entry::getKey, e -> eventTimeMapper.getWatermark(e.getValue()))); }
@Override public boolean complete() { if (traverser == null) { fillBufferFn.accept(src, buffer); traverser = eventTimeMapper == null ? buffer.traverse() : buffer.isEmpty() ? eventTimeMapper.flatMapIdle() : buffer.traverse().flatMap(t -> { // if eventTimeMapper is not null, we know that T is TimestampedItem<?> TimestampedItem<T> t1 = (TimestampedItem<T>) t; return eventTimeMapper.flatMapEvent(t1.item(), 0, t1.timestamp()); }); } boolean bufferEmpty = emitFromTraverser(traverser); if (bufferEmpty) { traverser = null; } return bufferEmpty && buffer.isClosed(); }
@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); } }
/** * Changes the partition count. The new partition count must be higher or * equal to the current count. * <p> * You can call this method at any moment. Added partitions will be * considered <em>active</em> initially. * * @param newPartitionCount partition count, must be higher than the * current count */ public void increasePartitionCount(int newPartitionCount) { increasePartitionCount(System.nanoTime(), newPartitionCount); }
StreamKafkaP( @Nonnull Properties properties, @Nonnull List<String> topics, @Nonnull DistributedFunction<? super ConsumerRecord<K, V>, ? extends T> projectionFn, @Nonnull EventTimePolicy<? super T> eventTimePolicy ) { this.properties = properties; this.topics = topics; this.projectionFn = projectionFn; eventTimeMapper = new EventTimeMapper<>(eventTimePolicy); partitionCounts = new int[topics.size()]; }
Traverser<Object> flatMapEvent(long now, @Nullable T event, int partitionIndex, long nativeEventTime) { assert traverser.isEmpty() : "the traverser returned previously not yet drained: remove all " + "items from the traverser before you call this method again."; if (event != null) { if (timestampFn == null && nativeEventTime == NO_NATIVE_TIME) { throw new JetException("Neither timestampFn nor nativeEventTime specified"); } long eventTime = timestampFn == null ? nativeEventTime : timestampFn.applyAsLong(event); handleEventInt(now, partitionIndex, eventTime); traverser.append(wrapFn.apply(event, eventTime)); } else { handleNoEventInt(now); } return traverser; }
private void tryGetNextResultSet() { while (resultSet == null && ++currentPartitionIndex < partitionIds.length) { ICompletableFuture<ReadResultSet<T>> future = readFutures[currentPartitionIndex]; if (!future.isDone()) { continue; } resultSet = toResultSet(future); int partitionId = partitionIds[currentPartitionIndex]; if (resultSet != null) { assert resultSet.size() > 0 : "empty resultSet"; long prevSequence = readOffsets[currentPartitionIndex]; long lostCount = resultSet.getNextSequenceToReadFrom() - resultSet.readCount() - prevSequence; if (lostCount > 0) { getLogger().warning(lostCount + " events lost for partition " + partitionId + " due to journal overflow when reading from event journal." + " Increase journal size to avoid this error. nextSequenceToReadFrom=" + resultSet.getNextSequenceToReadFrom() + ", readCount=" + resultSet.readCount() + ", prevSeq=" + prevSequence); } readOffsets[currentPartitionIndex] = resultSet.getNextSequenceToReadFrom(); } // make another read on the same partition readFutures[currentPartitionIndex] = readFromJournal(partitionId, readOffsets[currentPartitionIndex]); } if (currentPartitionIndex == partitionIds.length) { currentPartitionIndex = -1; traverser = eventTimeMapper.flatMapIdle(); } }
private void handleEventInt(long now, int partitionIndex, long eventTime) { wmPolicies[partitionIndex].reportEvent(eventTime); markIdleAt[partitionIndex] = now + idleTimeoutNanos; allAreIdle = false; handleNoEventInt(now); }
? eventTimeMapper.flatMapIdle() : traverseIterable(records).flatMap(record -> { offsets.get(record.topic())[record.partition()] = record.offset(); return eventTimeMapper.flatMapEvent(projectedRecord, currentAssignment.get(topicPartition), record.timestamp()); });
@Override public boolean finishSnapshotRestore() { eventTimeMapper.restoreWatermark(0, minRestoredWm); logFine(getLogger(), "restored lastEmittedWm=%s", minRestoredWm); return true; }
currentAssignment.put(tp, currentAssignment.size()); eventTimeMapper.increasePartitionCount(currentAssignment.size()); consumer.assign(currentAssignment.keySet()); if (seekToBeginning) {
StreamJmsP(Connection connection, DistributedFunction<? super Connection, ? extends Session> sessionFn, DistributedFunction<? super Session, ? extends MessageConsumer> consumerFn, DistributedConsumer<? super Session> flushFn, DistributedFunction<? super Message, ? extends T> projectionFn, EventTimePolicy<? super T> eventTimePolicy ) { this.connection = connection; this.sessionFn = sessionFn; this.consumerFn = consumerFn; this.flushFn = flushFn; this.projectionFn = projectionFn; eventTimeMapper = new EventTimeMapper<>(eventTimePolicy); eventTimeMapper.increasePartitionCount(1); }
/** * Flat-maps the given {@code event} by (possibly) prepending it with a * watermark. Designed to use when emitting from traverser: * <pre>{@code * Traverser t = traverserIterable(...) * .flatMap(event -> eventTimeMapper.flatMapEvent( * event, event.getPartition(), nativeEventTime)); * }</pre> * * @param event the event * @param partitionIndex the source partition index the event came from * @param nativeEventTime native event time in case no {@code timestampFn} was supplied or * {@link #NO_NATIVE_TIME} if the event has no native timestamp */ @Nonnull public Traverser<Object> flatMapEvent(T event, int partitionIndex, long nativeEventTime) { return flatMapEvent(System.nanoTime(), event, partitionIndex, nativeEventTime); }
@Override public boolean saveToSnapshot() { return tryEmitToSnapshot(broadcastKey(Keys.LAST_EMITTED_WM), eventTimeMapper.getWatermark(0)); }
@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); } }
public ConvenientSourceP( @Nonnull Function<? super Context, ? extends S> createFn, @Nonnull BiConsumer<? super S, ? super SourceBufferConsumerSide<?>> fillBufferFn, @Nonnull Consumer<? super S> destroyFn, @Nonnull SourceBufferConsumerSide<?> buffer, @Nullable EventTimePolicy<? super T> eventTimePolicy ) { this.createFn = createFn; this.fillBufferFn = fillBufferFn; this.destroyFn = destroyFn; this.buffer = buffer; if (eventTimePolicy != null) { eventTimeMapper = new EventTimeMapper<>(eventTimePolicy); eventTimeMapper.increasePartitionCount(1); } else { eventTimeMapper = null; } }
@SuppressWarnings("unchecked") private boolean tryProcessInternal(@Nullable Object item) { if (traverser == null) { traverser = eventTimeMapper.flatMapEvent((T) item, 0, EventTimeMapper.NO_NATIVE_TIME); } if (emitFromTraverser(traverser)) { traverser = null; return true; } return false; }
@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; }