private boolean isDuplicate(final E eventId) { long eventTime = context.timestamp(); WindowStoreIterator<Long> timeIterator = eventIdStore.fetch( eventId, eventTime - leftDurationMs, eventTime + rightDurationMs); boolean isDuplicate = timeIterator.hasNext(); timeIterator.close(); return isDuplicate; }
private void updateTimestampOfExistingEventToPreventExpiry(final E eventId, long newTimestamp) { eventIdStore.put(eventId, newTimestamp, newTimestamp); }
@Override public KeyValueIterator<Windowed<Bytes>, byte[]> all() { return bytesStore.all(); }
@Override public void init(final ProcessorContext context, final StateStore root) { this.context = context; bytesStore.init(context, root); final String topic = ProcessorStateManager.storeChangelogTopic(context.applicationId(), bytesStore.name()); changeLogger = new StoreChangeLogger<>( name(), context, new StateSerdes<>(topic, Serdes.Bytes(), Serdes.ByteArray())); }
@Override public KeyValueIterator<Windowed<Bytes>, byte[]> fetchAll(final long timeFrom, final long timeTo) { return bytesStore.fetchAll(timeFrom, timeTo); }
@Override public synchronized void flush() { cache.flush(name); underlying.flush(); }
@Override public void init(final ProcessorContext context, final StateStore root) { initInternal(context); underlying.init(context, root); keySchema.init(context.applicationId()); }
@Override public void close() { flush(); cache.close(name); underlying.close(); }
@SuppressWarnings("unchecked") private void initInternal(final ProcessorContext context) { this.context = (InternalProcessorContext) context; final String topic = ProcessorStateManager.storeChangelogTopic(context.applicationId(), underlying.name()); serdes = new StateSerdes<>(topic, keySerde == null ? (Serde<K>) context.keySerde() : keySerde, valueSerde == null ? (Serde<V>) context.valueSerde() : valueSerde); bytesSerdes = new StateSerdes<>(topic, Serdes.Bytes(), Serdes.ByteArray()); name = context.taskId() + "-" + underlying.name(); cache = this.context.getCache(); cache.addDirtyEntryFlushListener(name, new ThreadCache.DirtyEntryFlushListener() { @Override public void apply(final List<ThreadCache.DirtyEntry> entries) { for (final ThreadCache.DirtyEntry entry : entries) { final byte[] binaryWindowKey = cacheFunction.key(entry.key()).get(); final long timestamp = WindowKeySchema.extractStoreTimestamp(binaryWindowKey); final Windowed<K> windowedKey = WindowKeySchema.fromStoreKey(binaryWindowKey, windowSize, serdes); final Bytes key = Bytes.wrap(WindowKeySchema.extractStoreKeyBytes(binaryWindowKey)); maybeForward(entry, key, windowedKey, (InternalProcessorContext) context); underlying.put(key, entry.newValue(), timestamp); } } }); }
@Override public KeyValueIterator<Windowed<K>, V> fetchAll(final long timeFrom, final long timeTo) { return new MeteredWindowedKeyValueIterator<>(inner.fetchAll(timeFrom, timeTo), fetchTime, metrics, serdes, time); }
@Override public void flush() { final long startNs = time.nanoseconds(); try { inner.flush(); } finally { metrics.recordLatency(flushTime, startNs, time.nanoseconds()); } }
@SuppressWarnings("unchecked") @Override public void init(final ProcessorContext context, final StateStore root) { this.context = context; this.serdes = new StateSerdes<>(ProcessorStateManager.storeChangelogTopic(context.applicationId(), name()), keySerde == null ? (Serde<K>) context.keySerde() : keySerde, valueSerde == null ? (Serde<V>) context.valueSerde() : valueSerde); this.metrics = (StreamsMetricsImpl) context.metrics(); taskName = context.taskId().toString(); final String metricsGroup = "stream-" + metricScope + "-metrics"; final Map<String, String> taskTags = metrics.tagMap("task-id", taskName, metricScope + "-id", "all"); final Map<String, String> storeTags = metrics.tagMap("task-id", taskName, metricScope + "-id", name()); putTime = createTaskAndStoreLatencyAndThroughputSensors(DEBUG, "put", metrics, metricsGroup, taskName, name(), taskTags, storeTags); fetchTime = createTaskAndStoreLatencyAndThroughputSensors(DEBUG, "fetch", metrics, metricsGroup, taskName, name(), taskTags, storeTags); flushTime = createTaskAndStoreLatencyAndThroughputSensors(DEBUG, "flush", metrics, metricsGroup, taskName, name(), taskTags, storeTags); final Sensor restoreTime = createTaskAndStoreLatencyAndThroughputSensors(DEBUG, "restore", metrics, metricsGroup, taskName, name(), taskTags, storeTags); // register and possibly restore the state from the logs final long startNs = time.nanoseconds(); try { inner.init(context, root); } finally { this.metrics.recordLatency( restoreTime, startNs, time.nanoseconds() ); } }
@Override public byte[] fetch(final Bytes key, final long timestamp) { return bytesStore.fetch(key, timestamp); }
private void rememberNewEvent(final E eventId, long timestamp) { eventIdStore.put(eventId, timestamp, timestamp); }
@Override default KeyValueIterator<Windowed<K>, V> fetchAll(final Instant from, final Instant to) { ApiUtils.validateMillisecondInstant(from, "from"); ApiUtils.validateMillisecondInstant(to, "to"); return fetchAll(from.toEpochMilli(), to.toEpochMilli()); } }
@Override public KeyValueIterator<Windowed<K>, V> all() { return new MeteredWindowedKeyValueIterator<>(inner.all(), fetchTime, metrics, serdes, time); }
@Override public WindowStoreIterator<byte[]> fetch(final Bytes key, final long from, final long to) { return bytesStore.fetch(key, from, to); }
@Override public void put(final Bytes key, final byte[] value, final long windowStartTimestamp) { bytesStore.put(key, value, windowStartTimestamp); changeLogger.logChange(WindowKeySchema.toStoreKeyBinary(key, windowStartTimestamp, maybeUpdateSeqnumForDups()), value); }
@Override public KeyValueIterator<Windowed<Bytes>, byte[]> fetchAll(final long timeFrom, final long timeTo) { validateStoreOpen(); final KeyValueIterator<Windowed<Bytes>, byte[]> underlyingIterator = underlying.fetchAll(timeFrom, timeTo); final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.all(name); final HasNextCondition hasNextCondition = keySchema.hasNextCondition(null, null, timeFrom, timeTo); final PeekingKeyValueIterator<Bytes, LRUCacheEntry> filteredCacheIterator = new FilteredCacheIterator(cacheIterator, hasNextCondition, cacheFunction); return new MergedSortedCacheWindowStoreKeyValueIterator( filteredCacheIterator, underlyingIterator, bytesSerdes, windowSize, cacheFunction ); } }
@Override public KeyValueIterator<Windowed<Bytes>, byte[]> all() { validateStoreOpen(); final KeyValueIterator<Windowed<Bytes>, byte[]> underlyingIterator = underlying.all(); final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.all(name); return new MergedSortedCacheWindowStoreKeyValueIterator( cacheIterator, underlyingIterator, bytesSerdes, windowSize, cacheFunction ); }