From 04b9fc7c9d89eccee78bf67a00b1651007f0077f Mon Sep 17 00:00:00 2001 From: Gabriella Fu <2234862823@qq.com> Date: Mon, 30 Mar 2026 17:09:23 -0400 Subject: [PATCH 01/23] restore window code --- .../internals/ProcessorStateManager.java | 86 ++++--- .../internals/StoreChangelogReader.java | 219 +++++++++-------- ...tDualSchemaRocksDBSegmentedBytesStore.java | 5 + .../AbstractRocksDBSegmentedBytesStore.java | 5 + .../state/internals/InMemorySessionStore.java | 135 +++++------ .../state/internals/InMemoryWindowStore.java | 167 ++++++------- .../state/internals/WithRetentionPeriod.java | 21 ++ .../internals/StoreChangelogReaderTest.java | 220 +++++++++++++----- 8 files changed, 531 insertions(+), 327 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/state/internals/WithRetentionPeriod.java diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java index 0992d33f70095..a67ba230d3775 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java @@ -36,6 +36,8 @@ import org.apache.kafka.streams.state.internals.LegacyCheckpointingStateStore; import org.apache.kafka.streams.state.internals.RecordConverter; import org.apache.kafka.streams.state.internals.TimeOrderedKeyValueBuffer; +import org.apache.kafka.streams.state.internals.WithRetentionPeriod; +import org.apache.kafka.streams.state.internals.WrappedStateStore; import org.slf4j.Logger; @@ -103,6 +105,8 @@ public static class StateStoreMetadata { // corrupted state store should not be included in checkpointing private boolean corrupted; + private final long retentionPeriod; + private StateStoreMetadata(final StateStore stateStore, final CommitCallback commitCallback) { @@ -113,6 +117,7 @@ private StateStoreMetadata(final StateStore stateStore, this.changelogPartition = null; this.corrupted = false; this.offset = null; + this.retentionPeriod = -1L; } private StateStoreMetadata(final StateStore stateStore, @@ -130,12 +135,24 @@ private StateStoreMetadata(final StateStore stateStore, this.commitCallback = commitCallback; this.recordConverter = recordConverter; this.offset = null; + this.retentionPeriod = extractRetentionPeriod(stateStore); } private void setOffset(final Long offset) { this.offset = offset; } + private static long extractRetentionPeriod(final StateStore stateStore) { + StateStore current = stateStore; + while (current instanceof WrappedStateStore) { + current = ((WrappedStateStore) current).wrapped(); + } + if (current instanceof WithRetentionPeriod) { + return ((WithRetentionPeriod) current).retentionPeriod(); + } + return -1L; + } + // the offset is exposed to the changelog reader to determine if restoration is completed Long offset() { return this.offset; @@ -145,6 +162,11 @@ Long endOffset() { return this.endOffset; } + // the retentionPeriod is exposed to the changelog reader for window restoration + long retentionPeriod() { + return retentionPeriod; + } + public void setEndOffset(final Long endOffset) { this.endOffset = endOffset; } @@ -335,7 +357,7 @@ void initializeStoreOffsets(final boolean storeDirIsEmpty) { stateDirectory.updateTaskOffsets(taskId, changelogOffsets()); } catch (final RuntimeException e) { throw new ProcessorStateException(format("%sError updating state directory offsets when creating the state manager", - logPrefix), e); + logPrefix), e); } } @@ -355,7 +377,7 @@ public void registerStore(final StateStore store, if (LegacyCheckpointingStateStore.CHECKPOINT_FILE_NAME.startsWith(storeName)) { store.close(); throw new IllegalArgumentException(format("%sIllegal store name: %s, which collides with the pre-defined " + - "checkpoint file name", logPrefix, storeName)); + "checkpoint file name", logPrefix, storeName)); } if (stores.containsKey(storeName)) { @@ -369,13 +391,13 @@ public void registerStore(final StateStore store, } final StateStoreMetadata storeMetadata = isLoggingEnabled(storeName) ? - new StateStoreMetadata( - store, - getStorePartition(storeName), - stateRestoreCallback, - commitCallback, - converterForStore(store)) : - new StateStoreMetadata(store, commitCallback); + new StateStoreMetadata( + store, + getStorePartition(storeName), + stateRestoreCallback, + commitCallback, + converterForStore(store)) : + new StateStoreMetadata(store, commitCallback); // register the store first, so that if later an exception is thrown then eventually while we call `close` // on the state manager this state store would be closed as well @@ -408,7 +430,7 @@ void markChangelogAsCorrupted(final Collection partitions) { if (!partitionsToMarkAsCorrupted.isEmpty()) { throw new IllegalStateException("Some partitions " + partitionsToMarkAsCorrupted + " are not contained in " + - "the store list of task " + taskId + " marking as corrupted, this is not expected"); + "the store list of task " + taskId + " marking as corrupted, this is not expected"); } } @@ -421,8 +443,8 @@ public Map changelogOffsets() { // for changelog whose offset is unknown, use 0L indicating earliest offset // otherwise return the current offset + 1 as the next offset to fetch changelogOffsets.put( - storeMetadata.changelogPartition, - storeMetadata.offset == null ? 0L : storeMetadata.offset + 1L); + storeMetadata.changelogPartition, + storeMetadata.offset == null ? 0L : storeMetadata.offset + 1L); } } return changelogOffsets; @@ -464,7 +486,7 @@ StateStoreMetadata storeMetadata(final TopicPartition partition) { void restore(final StateStoreMetadata storeMetadata, final List> restoreRecords, final OptionalLong optionalLag) { if (!stores.containsValue(storeMetadata)) { throw new IllegalStateException("Restoring " + storeMetadata + " which is not registered in this state manager, " + - "this should not happen."); + "this should not happen."); } if (!restoreRecords.isEmpty()) { @@ -472,15 +494,15 @@ void restore(final StateStoreMetadata storeMetadata, final List> convertedRecords = restoreRecords.stream() - .map(storeMetadata.recordConverter::convert) - .collect(Collectors.toList()); + .map(storeMetadata.recordConverter::convert) + .collect(Collectors.toList()); try { restoreCallback.restoreBatch(convertedRecords); } catch (final RuntimeException e) { throw new ProcessorStateException( - format("%sException caught while trying to restore state from %s", logPrefix, storeMetadata.changelogPartition), - e + format("%sException caught while trying to restore state from %s", logPrefix, storeMetadata.changelogPartition), + e ); } @@ -576,14 +598,14 @@ public void flushCache() { // In case of FailedProcessingException Do not keep the failed processing exception in the stack trace if (exception instanceof FailedProcessingException) { firstException = new ProcessorStateException( - format("%sFailed to flush cache of store %s", logPrefix, store.name()), - exception.getCause()); + format("%sFailed to flush cache of store %s", logPrefix, store.name()), + exception.getCause()); } else if (exception instanceof StreamsException) { firstException = exception; } else { firstException = new ProcessorStateException( - format("%sFailed to flush cache of store %s", logPrefix, store.name()), - exception + format("%sFailed to flush cache of store %s", logPrefix, store.name()), + exception ); } log.error("Failed to flush cache of store {}: ", store.name(), firstException); @@ -626,13 +648,13 @@ public void close() throws ProcessorStateException { // In case of FailedProcessingException Do not keep the failed processing exception in the stack trace if (exception instanceof FailedProcessingException) firstException = new ProcessorStateException( - format("%sFailed to close state store %s", logPrefix, store.name()), - exception.getCause()); + format("%sFailed to close state store %s", logPrefix, store.name()), + exception.getCause()); else if (exception instanceof StreamsException) firstException = exception; else firstException = new ProcessorStateException( - format("%sFailed to close state store %s", logPrefix, store.name()), exception); + format("%sFailed to close state store %s", logPrefix, store.name()), exception); log.error("Failed to close state store {}: ", store.name(), firstException); } else { log.error("Failed to close state store {}: ", store.name(), exception); @@ -728,12 +750,12 @@ private boolean isLoggingEnabled(final String storeName) { private StateStoreMetadata findStore(final TopicPartition changelogPartition) { final List found = stores.values().stream() - .filter(metadata -> changelogPartition.equals(metadata.changelogPartition)) - .collect(Collectors.toList()); + .filter(metadata -> changelogPartition.equals(metadata.changelogPartition)) + .collect(Collectors.toList()); if (found.size() > 1) { throw new IllegalStateException("Multiple state stores are found for changelog partition " + changelogPartition + - ", this should never happen: " + found); + ", this should never happen: " + found); } return found.isEmpty() ? null : found.get(0); @@ -743,14 +765,14 @@ public TopicPartition registeredChangelogPartitionFor(final String storeName) { final StateStoreMetadata storeMetadata = stores.get(storeName); if (storeMetadata == null) { throw new IllegalStateException("State store " + storeName - + " for which the registered changelog partition should be" - + " retrieved has not been registered" + + " for which the registered changelog partition should be" + + " retrieved has not been registered" ); } if (storeMetadata.changelogPartition == null) { throw new IllegalStateException("Registered state store " + storeName - + " does not have a registered changelog partition." - + " This may happen if logging is disabled for the state store." + + " does not have a registered changelog partition." + + " This may happen if logging is disabled for the state store." ); } return storeMetadata.changelogPartition; @@ -759,4 +781,4 @@ public TopicPartition registeredChangelogPartitionFor(final String storeName) { public String changelogFor(final String storeName) { return storeToChangelogTopic.get(storeName); } -} +} \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java index eab7da800d882..7b43cee270ff4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java @@ -25,6 +25,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.InvalidOffsetException; +import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.common.IsolationLevel; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; @@ -167,7 +168,7 @@ private void transitTo(final ChangelogState newState) { public String toString() { final Long currentOffset = storeMetadata.offset(); return changelogState + " " + stateManager.taskType() + - " (currentOffset " + currentOffset + ", endOffset " + restoreEndOffset + ")"; + " (currentOffset " + currentOffset + ", endOffset " + restoreEndOffset + ")"; } // for testing only below @@ -243,7 +244,7 @@ public StoreChangelogReader(final Time time, this.groupId = config.getString(StreamsConfig.APPLICATION_ID_CONFIG); this.pollTime = Duration.ofMillis(config.getLong(StreamsConfig.POLL_MS_CONFIG)); this.updateOffsetIntervalMs = config.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG) == Long.MAX_VALUE ? - DEFAULT_OFFSET_UPDATE_MS : config.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG); + DEFAULT_OFFSET_UPDATE_MS : config.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG); this.lastUpdateOffsetTime = 0L; this.changelogs = new HashMap<>(); @@ -259,7 +260,7 @@ private boolean hasRestoredToEnd(final ChangelogMetadata metadata) { // end offset is not initialized meaning that it is from a standby task, // this should never happen since we only call this function for active task in restoring phase throw new IllegalStateException("End offset for changelog " + metadata + " is unknown when deciding " + - "if it has completed restoration, this should never happen."); + "if it has completed restoration, this should never happen."); } else if (endOffset == 0) { // this is a special case, meaning there's nothing to be restored since the changelog has no data // OR the changelog is a source topic and there's no committed offset @@ -287,7 +288,7 @@ private boolean hasRestoredToEnd(final ChangelogMetadata metadata) { // this also includes InvalidOffsetException, which should not happen under normal // execution, hence it is also okay to wrap it as fatal StreamsException throw new StreamsException("Restore consumer get unexpected error trying to get the position " + - " of " + partition, e); + " of " + partition, e); } } else { return metadata.bufferedRecords.get(0).offset() >= endOffset; @@ -324,8 +325,8 @@ public void enforceRestoreActive() { public void transitToUpdateStandby() { if (state != ChangelogReaderState.ACTIVE_RESTORING) { throw new IllegalStateException( - "The changelog reader is not restoring active tasks (is " + state + ") while trying to " + - "transit to update standby tasks: " + changelogs + "The changelog reader is not restoring active tasks (is " + state + ") while trying to " + + "transit to update standby tasks: " + changelogs ); } @@ -351,7 +352,7 @@ public void register(final TopicPartition partition, final ProcessorStateManager final StateStoreMetadata storeMetadata = stateManager.storeMetadata(partition); if (storeMetadata == null) { throw new IllegalStateException("Cannot find the corresponding state store metadata for changelog " + - partition); + partition); } final ChangelogMetadata changelogMetadata = new ChangelogMetadata(storeMetadata, stateManager); @@ -363,7 +364,7 @@ public void register(final TopicPartition partition, final ProcessorStateManager if (changelogs.putIfAbsent(partition, changelogMetadata) != null) { throw new IllegalStateException("There is already a changelog registered for " + partition + - ", this should not happen: " + changelogs); + ", this should not happen: " + changelogs); } } @@ -378,11 +379,11 @@ private ChangelogMetadata restoringChangelogByPartition(final TopicPartition par final ChangelogMetadata changelogMetadata = changelogs.get(partition); if (changelogMetadata == null) { throw new IllegalStateException("The corresponding changelog restorer for " + partition + - " does not exist, this should not happen."); + " does not exist, this should not happen."); } if (changelogMetadata.changelogState != ChangelogState.RESTORING) { throw new IllegalStateException("The corresponding changelog restorer for " + partition + - " has already transited to completed state, this should not happen."); + " has already transited to completed state, this should not happen."); } return changelogMetadata; @@ -390,45 +391,45 @@ private ChangelogMetadata restoringChangelogByPartition(final TopicPartition par private Set registeredChangelogs() { return changelogs.values().stream() - .filter(metadata -> metadata.changelogState == ChangelogState.REGISTERED) - .collect(Collectors.toSet()); + .filter(metadata -> metadata.changelogState == ChangelogState.REGISTERED) + .collect(Collectors.toSet()); } private Set restoringChangelogs() { return changelogs.values().stream() - .filter(metadata -> metadata.changelogState == ChangelogState.RESTORING) - .map(metadata -> metadata.storeMetadata.changelogPartition()) - .collect(Collectors.toSet()); + .filter(metadata -> metadata.changelogState == ChangelogState.RESTORING) + .map(metadata -> metadata.storeMetadata.changelogPartition()) + .collect(Collectors.toSet()); } private Set activeRestoringChangelogs() { return changelogs.values().stream() - .filter(metadata -> metadata.changelogState == ChangelogState.RESTORING && - metadata.stateManager.taskType() == Task.TaskType.ACTIVE) - .map(metadata -> metadata.storeMetadata.changelogPartition()) - .collect(Collectors.toSet()); + .filter(metadata -> metadata.changelogState == ChangelogState.RESTORING && + metadata.stateManager.taskType() == Task.TaskType.ACTIVE) + .map(metadata -> metadata.storeMetadata.changelogPartition()) + .collect(Collectors.toSet()); } private Set standbyRestoringChangelogs() { return changelogs.values().stream() - .filter(metadata -> metadata.changelogState == ChangelogState.RESTORING && - metadata.stateManager.taskType() == Task.TaskType.STANDBY) - .map(metadata -> metadata.storeMetadata.changelogPartition()) - .collect(Collectors.toSet()); + .filter(metadata -> metadata.changelogState == ChangelogState.RESTORING && + metadata.stateManager.taskType() == Task.TaskType.STANDBY) + .map(metadata -> metadata.storeMetadata.changelogPartition()) + .collect(Collectors.toSet()); } @Override public boolean allChangelogsCompleted() { return changelogs.values().stream() - .allMatch(metadata -> metadata.changelogState == ChangelogState.COMPLETED); + .allMatch(metadata -> metadata.changelogState == ChangelogState.COMPLETED); } @Override public Set completedChangelogs() { return changelogs.values().stream() - .filter(metadata -> metadata.changelogState == ChangelogState.COMPLETED) - .map(metadata -> metadata.storeMetadata.changelogPartition()) - .collect(Collectors.toSet()); + .filter(metadata -> metadata.changelogState == ChangelogState.COMPLETED) + .map(metadata -> metadata.storeMetadata.changelogPartition()) + .collect(Collectors.toSet()); } // 1. if there are any registered changelogs that needs initialization, try to initialize them first; @@ -470,8 +471,8 @@ public long restore(final Map tasks) { totalRestored += restoreChangelog(task, changelogMetadata); } catch (final TimeoutException timeoutException) { tasks.get(taskId).maybeInitTaskTimeoutOrThrow( - time.milliseconds(), - timeoutException + time.milliseconds(), + timeoutException ); } } @@ -497,10 +498,10 @@ private ConsumerRecords pollRecordsFromRestoreConsumer(final Map // TODO (?) If we cannot fetch records for standby task, should we trigger `task.timeout.ms` ? } catch (final InvalidOffsetException e) { log.warn("Encountered " + e.getClass().getName() + - " fetching records from restore consumer for partitions " + e.partitions() + ", it is likely that " + - "the consumer's position has fallen out of the topic partition offset range because the topic was " + - "truncated or compacted on the broker, marking the corresponding tasks as corrupted and re-initializing " + - "it later.", e); + " fetching records from restore consumer for partitions " + e.partitions() + ", it is likely that " + + "the consumer's position has fallen out of the topic partition offset range because the topic was " + + "truncated or compacted on the broker, marking the corresponding tasks as corrupted and re-initializing " + + "it later.", e); final Set corruptedTasks = new HashSet<>(); e.partitions().forEach(partition -> corruptedTasks.add(changelogs.get(partition).stateManager.taskId())); @@ -528,9 +529,9 @@ private void updatePartitionsByType(final Map tasks, final Set restoringChangelogs, final TaskType taskType) { final Collection toResume = - restoringChangelogs.stream().filter(t -> shouldResume(tasks, t, taskType)).collect(Collectors.toList()); + restoringChangelogs.stream().filter(t -> shouldResume(tasks, t, taskType)).collect(Collectors.toList()); final Collection toPause = - restoringChangelogs.stream().filter(t -> shouldPause(tasks, t, taskType)).collect(Collectors.toList()); + restoringChangelogs.stream().filter(t -> shouldPause(tasks, t, taskType)).collect(Collectors.toList()); restoreConsumer.resume(toResume); restoreConsumer.pause(toPause); } @@ -561,20 +562,20 @@ private void maybeLogRestorationProgress() { final Set topicPartitions = activeRestoringChangelogs(); if (!topicPartitions.isEmpty()) { final StringBuilder builder = new StringBuilder().append("Restoration in progress for ") - .append(topicPartitions.size()) - .append(" partitions."); + .append(topicPartitions.size()) + .append(" partitions."); for (final TopicPartition partition : topicPartitions) { final ChangelogMetadata changelogMetadata = restoringChangelogByPartition(partition); builder.append(" {") - .append(partition) - .append(": ") - .append("position=") - .append(getPositionString(partition, changelogMetadata)) - .append(", end=") - .append(changelogMetadata.restoreEndOffset) - .append(", totalRestored=") - .append(changelogMetadata.totalRestored) - .append("}"); + .append(partition) + .append(": ") + .append("position=") + .append(getPositionString(partition, changelogMetadata)) + .append(", end=") + .append(changelogMetadata.restoreEndOffset) + .append(", totalRestored=") + .append(changelogMetadata.totalRestored) + .append("}"); } log.info(builder.toString()); lastRestoreLogTime = time.milliseconds(); @@ -595,18 +596,18 @@ private static String getPositionString(final TopicPartition partition, private void maybeUpdateLimitOffsetsForStandbyChangelogs(final Map tasks) { // we only consider updating the limit offset for standbys if we are not restoring active tasks if (state == ChangelogReaderState.STANDBY_UPDATING && - updateOffsetIntervalMs < time.milliseconds() - lastUpdateOffsetTime) { + updateOffsetIntervalMs < time.milliseconds() - lastUpdateOffsetTime) { // when the interval has elapsed we should try to update the limit offset for standbys reading from - // a source changelog with the new committed offset, unless there are no buffered records since + // a source changelog with the new committed offset, unless there are no buffered records since // we only need the limit when processing new records // for other changelog partitions we do not need to update limit offset at all since we never need to // check when it completes based on limit offset anyways: the end offset would keep increasing and the // standby never need to stop final Set changelogsWithLimitOffsets = changelogs.entrySet().stream() - .filter(entry -> entry.getValue().stateManager.taskType() == Task.TaskType.STANDBY && - entry.getValue().stateManager.changelogAsSource(entry.getKey())) - .map(Map.Entry::getKey).collect(Collectors.toSet()); + .filter(entry -> entry.getValue().stateManager.taskType() == Task.TaskType.STANDBY && + entry.getValue().stateManager.changelogAsSource(entry.getKey())) + .map(Map.Entry::getKey).collect(Collectors.toSet()); for (final TopicPartition partition : changelogsWithLimitOffsets) { if (!changelogs.get(partition).bufferedRecords().isEmpty()) { @@ -624,7 +625,7 @@ private void bufferChangelogRecords(final ChangelogMetadata changelogMetadata, f // filter polled records for null-keys and also possibly update buffer limit index if (record.key() == null) { log.warn("Read changelog record with null key from changelog {} at offset {}, " + - "skipping it for restoration", changelogMetadata.storeMetadata.changelogPartition(), record.offset()); + "skipping it for restoration", changelogMetadata.storeMetadata.changelogPartition(), record.offset()); } else { changelogMetadata.bufferedRecords.add(record); final long offset = record.offset(); @@ -666,7 +667,7 @@ private int restoreChangelog(final Task task, final ChangelogMetadata changelogM final Long currentOffset = storeMetadata.offset(); log.trace("Restored {} records from changelog {} to store {}, end offset is {}, current offset is {}", - numRecords, partition, storeName, recordEndOffset(changelogMetadata.restoreEndOffset), currentOffset); + numRecords, partition, storeName, recordEndOffset(changelogMetadata.restoreEndOffset), currentOffset); changelogMetadata.bufferedLimitIndex = 0; changelogMetadata.totalRestored += numRecords; @@ -690,7 +691,7 @@ private int restoreChangelog(final Task task, final ChangelogMetadata changelogM // we should check even if there's nothing restored, but do not check completed if we are processing standby tasks if (changelogMetadata.stateManager.taskType() == Task.TaskType.ACTIVE && hasRestoredToEnd(changelogMetadata)) { log.info("Finished restoring changelog {} to store {} with a total number of {} records", - partition, storeName, changelogMetadata.totalRestored); + partition, storeName, changelogMetadata.totalRestored); changelogMetadata.transitTo(ChangelogState.COMPLETED); pauseChangelogsFromRestoreConsumer(Collections.singleton(partition)); @@ -743,23 +744,23 @@ private Map committedOffsetForChangelogs(final Map(partitions)); + .topicPartitions(new ArrayList<>(partitions)); final Map committedOffsets = - adminClient.listConsumerGroupOffsets( - Collections.singletonMap(groupId, spec), - options - ) - .partitionsToOffsetAndMetadata(groupId).get().entrySet() - .stream() - .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue() == null ? 0L : e.getValue().offset())); + adminClient.listConsumerGroupOffsets( + Collections.singletonMap(groupId, spec), + options + ) + .partitionsToOffsetAndMetadata(groupId).get().entrySet() + .stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue() == null ? 0L : e.getValue().offset())); clearTaskTimeout(getTasksFromPartitions(tasks, partitions)); return committedOffsets; } catch (final TimeoutException | InterruptedException | ExecutionException retriableException) { log.debug("Could not retrieve the committed offsets for partitions {} due to {}, will retry in the next run loop", - partitions, retriableException.toString()); + partitions, retriableException.toString()); maybeInitTaskTimeoutOrThrow(getTasksFromPartitions(tasks, partitions), retriableException); return Collections.emptyMap(); } catch (final KafkaException e) { @@ -769,7 +770,7 @@ private Map committedOffsetForChangelogs(final Map tasks, final Set newPartitionsToRestore) { newPartitionsToRestore.removeIf(changelogMetadata -> - !tasks.containsKey(changelogs.get(changelogMetadata.storeMetadata.changelogPartition()).stateManager.taskId())); + !tasks.containsKey(changelogs.get(changelogMetadata.storeMetadata.changelogPartition()).stateManager.taskId())); } private Map endOffsetForChangelogs(final Map tasks, final Set partitions) { @@ -782,16 +783,16 @@ private Map endOffsetForChangelogs(final Map // see KAFKA-10167 for more details final ListOffsetsOptions options = new ListOffsetsOptions(IsolationLevel.READ_UNCOMMITTED); final Map offsetSpecs = - partitions.stream().collect(Collectors.toMap(Function.identity(), tp -> OffsetSpec.latest())); + partitions.stream().collect(Collectors.toMap(Function.identity(), tp -> OffsetSpec.latest())); final Map logEndOffsets = adminClient.listOffsets(offsetSpecs, options) - .all().get().entrySet() - .stream().collect(Collectors.toMap(Map.Entry::getKey, entry -> entry.getValue().offset())); + .all().get().entrySet() + .stream().collect(Collectors.toMap(Map.Entry::getKey, entry -> entry.getValue().offset())); clearTaskTimeout(getTasksFromPartitions(tasks, partitions)); return logEndOffsets; } catch (final TimeoutException | InterruptedException | ExecutionException retriableException) { log.debug("Could not fetch all end offsets for {} due to {}, will retry in the next run loop", - partitions, retriableException.toString()); + partitions, retriableException.toString()); maybeInitTaskTimeoutOrThrow(getTasksFromPartitions(tasks, partitions), retriableException); return Collections.emptyMap(); } catch (final KafkaException e) { @@ -803,22 +804,22 @@ private void updateLimitOffsetsForStandbyChangelogs(final Map newLimit) { throw new IllegalStateException("Offset limit should monotonically increase, but was reduced for partition " + - partition + ". New limit: " + newLimit + ". Previous limit: " + previousLimit); + partition + ". New limit: " + newLimit + ". Previous limit: " + previousLimit); } metadata.restoreEndOffset = newLimit; // update the limit index for buffered records while (metadata.bufferedLimitIndex < metadata.bufferedRecords.size() && - metadata.bufferedRecords.get(metadata.bufferedLimitIndex).offset() < metadata.restoreEndOffset) + metadata.bufferedRecords.get(metadata.bufferedLimitIndex).offset() < metadata.restoreEndOffset) metadata.bufferedLimitIndex++; } } @@ -864,13 +865,13 @@ private void initializeChangelogs(final Map tasks, final ChangelogMetadata changelogMetadata = changelogs.get(partition); final Long endOffset = endOffsets.get(partition); final Long committedOffset = newPartitionsToFindCommittedOffset.contains(partition) ? - committedOffsets.get(partition) : Long.valueOf(Long.MAX_VALUE); + committedOffsets.get(partition) : Long.valueOf(Long.MAX_VALUE); if (endOffset != null && committedOffset != null) { if (changelogMetadata.restoreEndOffset != null) { throw new IllegalStateException("End offset for " + partition + - " should only be initialized once. Existing value: " + changelogMetadata.restoreEndOffset + - ", new value: (" + endOffset + ", " + committedOffset + ")"); + " should only be initialized once. Existing value: " + changelogMetadata.restoreEndOffset + + ", new value: (" + endOffset + ", " + committedOffset + ")"); } changelogMetadata.restoreEndOffset = Math.min(endOffset, committedOffset); @@ -879,7 +880,7 @@ private void initializeChangelogs(final Map tasks, } else { if (!newPartitionsToRestore.remove(changelogMetadata)) { throw new IllegalStateException("New changelogs to restore " + newPartitionsToRestore + - " does not contain the one looking for end offset: " + partition + ", this should not happen."); + " does not contain the one looking for end offset: " + partition + ", this should not happen."); } log.info("End offset for changelog {} cannot be found; will retry in the next time.", partition); @@ -893,7 +894,7 @@ private void initializeChangelogs(final Map tasks, // add new partitions to the restore consumer and transit them to restoring state addChangelogsToRestoreConsumer(newPartitionsToRestore.stream().map(metadata -> metadata.storeMetadata.changelogPartition()) - .collect(Collectors.toSet())); + .collect(Collectors.toSet())); newPartitionsToRestore.forEach(metadata -> metadata.transitTo(ChangelogState.RESTORING)); @@ -916,7 +917,7 @@ private void addChangelogsToRestoreConsumer(final Set partitions // the current assignment should not contain any of the new partitions if (assignment.removeAll(partitions)) { throw new IllegalStateException("The current assignment " + restoreConsumer.assignment() + " " + - "already contains some of the new partitions " + partitions); + "already contains some of the new partitions " + partitions); } assignment.addAll(partitions); restoreConsumer.assign(assignment); @@ -930,7 +931,7 @@ private void pauseChangelogsFromRestoreConsumer(final Collection // the current assignment should contain all the partitions to pause if (!assignment.containsAll(partitions)) { throw new IllegalStateException("The current assignment " + assignment + " " + - "does not contain some of the partitions " + partitions + " for pausing."); + "does not contain some of the partitions " + partitions + " for pausing."); } restoreConsumer.pause(partitions); @@ -947,7 +948,7 @@ private void removeChangelogsFromRestoreConsumer(final Collection tasks, final Set newPartitionsToRestore) { // separate those who do not have the current offset loaded from checkpoint final Set newPartitionsWithoutStartOffset = new HashSet<>(); + final Map newPartitionsWithTimestampSeek = new HashMap<>(); for (final ChangelogMetadata changelogMetadata : newPartitionsToRestore) { final StateStoreMetadata storeMetadata = changelogMetadata.storeMetadata; @@ -984,20 +986,24 @@ private void prepareChangelogs(final Map tasks, restoreConsumer.seek(partition, currentOffset + 1); log.debug("Start restoring changelog partition {} from current offset {} to end offset {}.", - partition, currentOffset, recordEndOffset(endOffset)); + partition, currentOffset, recordEndOffset(endOffset)); } else { - log.debug("Start restoring changelog partition {} from the beginning offset to end offset {} " + - "since we cannot find current offset.", partition, recordEndOffset(endOffset)); - - newPartitionsWithoutStartOffset.add(partition); + final long retentionPeriod = storeMetadata.retentionPeriod(); + final long seekTimestamp = retentionPeriod > 0 && retentionPeriod != Long.MAX_VALUE + ? time.milliseconds() - retentionPeriod : -1L; + if (seekTimestamp > 0) { + newPartitionsWithTimestampSeek.put(partition, seekTimestamp); + log.debug("Start restoring windowed changelog partition {} from timestamp {} to end offset {}.", + partition, seekTimestamp, recordEndOffset(endOffset)); + } else { + log.debug("Start restoring changelog partition {} from the beginning offset to end offset {} " + + "since we cannot find current offset.", partition, recordEndOffset(endOffset)); + newPartitionsWithoutStartOffset.add(partition); + } } } - // optimization: batch all seek-to-beginning offsets in a single request - // seek is not a blocking call so there's nothing to capture - if (!newPartitionsWithoutStartOffset.isEmpty()) { - restoreConsumer.seekToBeginning(newPartitionsWithoutStartOffset); - } + seekToTimestampOrBeginning(newPartitionsWithTimestampSeek, newPartitionsWithoutStartOffset); for (final ChangelogMetadata changelogMetadata : newPartitionsToRestore) { final StateStoreMetadata storeMetadata = changelogMetadata.storeMetadata; @@ -1039,6 +1045,29 @@ private void prepareChangelogs(final Map tasks, } } + private void seekToTimestampOrBeginning(final Map partitionsWithTimestampSeek, + final Set partitionsWithoutStartOffset) { + // optimization: seek windowed stores by timestamp to skip expired data + if (!partitionsWithTimestampSeek.isEmpty()) { + final Map offsetsByTimestamp = + restoreConsumer.offsetsForTimes(partitionsWithTimestampSeek); + offsetsByTimestamp.forEach((key, value) -> { + if (value != null) { + restoreConsumer.seek(key, value.offset()); + } else { + // no offset found for the timestamp, fall back to seeking to the beginning + partitionsWithoutStartOffset.add(key); + } + }); + } + + // optimization: batch all seek-to-beginning offsets in a single request + // seek is not a blocking call so there's nothing to capture + if (!partitionsWithoutStartOffset.isEmpty()) { + restoreConsumer.seekToBeginning(partitionsWithoutStartOffset); + } + } + @Override public void unregister(final Collection revokedChangelogs) { unregister(revokedChangelogs, StandbyUpdateListener.SuspendReason.MIGRATED); @@ -1086,8 +1115,8 @@ public void unregister(final Collection revokedChangelogs, changelogMetadata.clear(); } else { log.debug("Changelog partition {} could not be found," + - " it could be already cleaned up during the handling" + - " of task corruption and never restore again", partition); + " it could be already cleaned up during the handling" + + " of task corruption and never restore again", partition); } } @@ -1132,4 +1161,4 @@ ChangelogMetadata changelogMetadata(final TopicPartition partition) { ChangelogReaderState state() { return state; } -} +} \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractDualSchemaRocksDBSegmentedBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractDualSchemaRocksDBSegmentedBytesStore.java index f353e56f7df77..9606e89ece57c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractDualSchemaRocksDBSegmentedBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractDualSchemaRocksDBSegmentedBytesStore.java @@ -74,6 +74,11 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStore all() { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java index c6b57411067c2..7a83ea5d19c7f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java @@ -73,6 +73,11 @@ public class AbstractRocksDBSegmentedBytesStore implements Se this.segments = segments; } + @Override + public long retentionPeriod() { + return retentionPeriod; + } + @Override public KeyValueIterator fetch(final Bytes key, final long from, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java index 9d3936d0eb0b6..d84acacf0cbb0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java @@ -54,7 +54,7 @@ import static org.apache.kafka.streams.StreamsConfig.InternalConfig.IQ_CONSISTENCY_OFFSET_VECTOR_ENABLED; -public class InMemorySessionStore implements SessionStore { +public class InMemorySessionStore implements SessionStore, WithRetentionPeriod { private static final Logger LOG = LoggerFactory.getLogger(InMemorySessionStore.class); @@ -67,10 +67,10 @@ public class InMemorySessionStore implements SessionStore { private final long retentionPeriod; private static final String INVALID_RANGE_WARN_MSG = - "Returning empty iterator for fetch with invalid key range: from > to. " + - "This may be due to range arguments set in the wrong order, " + - "or serdes that don't preserve ordering when lexicographically comparing the serialized bytes. " + - "Note that the built-in numerical serdes do not follow this for negative numbers"; + "Returning empty iterator for fetch with invalid key range: from > to. " + + "This may be due to range arguments set in the wrong order, " + + "or serdes that don't preserve ordering when lexicographically comparing the serialized bytes. " + + "Note that the built-in numerical serdes do not follow this for negative numbers"; private final ConcurrentNavigableMap>> endTimeMap = new ConcurrentSkipListMap<>(); private final Set openIterators = ConcurrentHashMap.newKeySet(); @@ -91,6 +91,11 @@ public InMemorySessionStore( this.position = Position.emptyPosition(); } + @Override + public long retentionPeriod() { + return retentionPeriod; + } + @Override public String name() { return name; @@ -109,9 +114,9 @@ public void init(final StateStoreContext stateStoreContext, this.context = (InternalProcessorContext) stateStoreContext; final StreamsMetricsImpl metrics = this.context.metrics(); expiredRecordSensor = TaskMetrics.droppedRecordsSensor( - threadId, - taskName, - metrics + threadId, + taskName, + metrics ); } else { this.context = null; @@ -120,24 +125,24 @@ public void init(final StateStoreContext stateStoreContext, if (root != null) { final boolean consistencyEnabled = StreamsConfig.InternalConfig.getBoolean( - stateStoreContext.appConfigs(), - IQ_CONSISTENCY_OFFSET_VECTOR_ENABLED, - false + stateStoreContext.appConfigs(), + IQ_CONSISTENCY_OFFSET_VECTOR_ENABLED, + false ); stateStoreContext.register( - root, - (RecordBatchingStateRestoreCallback) records -> { - synchronized (position) { - for (final ConsumerRecord record : records) { - put(SessionKeySchema.from(Bytes.wrap(record.key())), record.value()); - ChangelogRecordDeserializationHelper.applyChecksAndUpdatePosition( - record, - consistencyEnabled, - position - ); + root, + (RecordBatchingStateRestoreCallback) records -> { + synchronized (position) { + for (final ConsumerRecord record : records) { + put(SessionKeySchema.from(Bytes.wrap(record.key())), record.value()); + ChangelogRecordDeserializationHelper.applyChecksAndUpdatePosition( + record, + consistencyEnabled, + position + ); + } } } - } ); } open = true; @@ -227,7 +232,7 @@ public KeyValueIterator, byte[]> findSessions(final long earlies removeExpiredSegments(); final ConcurrentNavigableMap>> endTimSubMap - = endTimeMap.subMap(earliestSessionEndTime, true, latestSessionEndTime, true); + = endTimeMap.subMap(earliestSessionEndTime, true, latestSessionEndTime, true); return registerNewIterator(null, null, Long.MAX_VALUE, endTimSubMap.entrySet().iterator(), true); } @@ -241,10 +246,10 @@ public KeyValueIterator, byte[]> findSessions(final Bytes key, removeExpiredSegments(); return registerNewIterator(key, - key, - latestSessionStartTime, - endTimeMap.tailMap(earliestSessionEndTime, true).entrySet().iterator(), - true); + key, + latestSessionStartTime, + endTimeMap.tailMap(earliestSessionEndTime, true).entrySet().iterator(), + true); } @Override @@ -256,11 +261,11 @@ public KeyValueIterator, byte[]> backwardFindSessions(final Byte removeExpiredSegments(); return registerNewIterator( - key, - key, - latestSessionStartTime, - endTimeMap.tailMap(earliestSessionEndTime, true).descendingMap().entrySet().iterator(), - false + key, + key, + latestSessionStartTime, + endTimeMap.tailMap(earliestSessionEndTime, true).descendingMap().entrySet().iterator(), + false ); } @@ -277,10 +282,10 @@ public KeyValueIterator, byte[]> findSessions(final Bytes keyFro } return registerNewIterator(keyFrom, - keyTo, - latestSessionStartTime, - endTimeMap.tailMap(earliestSessionEndTime, true).entrySet().iterator(), - true); + keyTo, + latestSessionStartTime, + endTimeMap.tailMap(earliestSessionEndTime, true).entrySet().iterator(), + true); } @Override @@ -296,11 +301,11 @@ public KeyValueIterator, byte[]> backwardFindSessions(final Byte } return registerNewIterator( - keyFrom, - keyTo, - latestSessionStartTime, - endTimeMap.tailMap(earliestSessionEndTime, true).descendingMap().entrySet().iterator(), - false + keyFrom, + keyTo, + latestSessionStartTime, + endTimeMap.tailMap(earliestSessionEndTime, true).descendingMap().entrySet().iterator(), + false ); } @@ -336,7 +341,7 @@ public KeyValueIterator, byte[]> backwardFetch(final Bytes keyFr removeExpiredSegments(); return registerNewIterator( - keyFrom, keyTo, Long.MAX_VALUE, endTimeMap.descendingMap().entrySet().iterator(), false); + keyFrom, keyTo, Long.MAX_VALUE, endTimeMap.descendingMap().entrySet().iterator(), false); } @Override @@ -355,12 +360,12 @@ public QueryResult query(final Query query, final QueryConfig config) { return StoreQueryUtils.handleBasicQueries( - query, - positionBound, - config, - this, - position, - context + query, + positionBound, + config, + this, + position, + context ); } @@ -385,9 +390,9 @@ public void close() { long numEntries() { return endTimeMap.values().stream() - .flatMap(keyMap -> keyMap.values().stream()) - .mapToLong(Map::size) - .sum(); + .flatMap(keyMap -> keyMap.values().stream()) + .mapToLong(Map::size) + .sum(); } private void removeExpiredSegments() { @@ -406,14 +411,14 @@ private InMemorySessionStoreIterator registerNewIterator(final Bytes keyFrom, final Iterator>>> endTimeIterator, final boolean forward) { final InMemorySessionStoreIterator iterator = - new InMemorySessionStoreIterator( - keyFrom, - keyTo, - latestSessionStartTime, - endTimeIterator, - openIterators::remove, - forward - ); + new InMemorySessionStoreIterator( + keyFrom, + keyTo, + latestSessionStartTime, + endTimeIterator, + openIterators::remove, + forward + ); openIterators.add(iterator); return iterator; } @@ -564,13 +569,13 @@ private boolean setInnerIterators() { } else { if (forward) { recordIterator = nextKeyEntry.getValue() - .headMap(latestSessionStartTime, true) - .descendingMap() - .entrySet().iterator(); + .headMap(latestSessionStartTime, true) + .descendingMap() + .entrySet().iterator(); } else { recordIterator = nextKeyEntry.getValue() - .headMap(latestSessionStartTime, true) - .entrySet().iterator(); + .headMap(latestSessionStartTime, true) + .entrySet().iterator(); } } @@ -592,4 +597,4 @@ private void getNextIterators() { } } -} +} \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java index 8d2228db5c3d5..92b7f4362973e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java @@ -59,7 +59,7 @@ import static org.apache.kafka.streams.state.internals.WindowKeySchema.extractStoreTimestamp; -public class InMemoryWindowStore implements WindowStore { +public class InMemoryWindowStore implements WindowStore, WithRetentionPeriod { private static final Logger LOG = LoggerFactory.getLogger(InMemoryWindowStore.class); private static final int SEQNUM_SIZE = 4; @@ -95,6 +95,11 @@ public InMemoryWindowStore(final String name, this.position = Position.emptyPosition(); } + @Override + public long retentionPeriod() { + return retentionPeriod; + } + @Override public String name() { return name; @@ -109,35 +114,35 @@ public void init(final StateStoreContext stateStoreContext, final String threadId = Thread.currentThread().getName(); final String taskName = stateStoreContext.taskId().toString(); expiredRecordSensor = TaskMetrics.droppedRecordsSensor( - threadId, - taskName, - metrics + threadId, + taskName, + metrics ); if (root != null) { final boolean consistencyEnabled = StreamsConfig.InternalConfig.getBoolean( - stateStoreContext.appConfigs(), - IQ_CONSISTENCY_OFFSET_VECTOR_ENABLED, - false + stateStoreContext.appConfigs(), + IQ_CONSISTENCY_OFFSET_VECTOR_ENABLED, + false ); stateStoreContext.register( - root, - (RecordBatchingStateRestoreCallback) records -> { - synchronized (position) { - for (final ConsumerRecord record : records) { - put( - Bytes.wrap(extractStoreKeyBytes(record.key())), - record.value(), - extractStoreTimestamp(record.key()) - ); - ChangelogRecordDeserializationHelper.applyChecksAndUpdatePosition( - record, - consistencyEnabled, - position - ); + root, + (RecordBatchingStateRestoreCallback) records -> { + synchronized (position) { + for (final ConsumerRecord record : records) { + put( + Bytes.wrap(extractStoreKeyBytes(record.key())), + record.value(), + extractStoreTimestamp(record.key()) + ); + ChangelogRecordDeserializationHelper.applyChecksAndUpdatePosition( + record, + consistencyEnabled, + position + ); + } } } - } ); } open = true; @@ -221,17 +226,17 @@ WindowStoreIterator fetch(final Bytes key, final long timeFrom, final lo if (forward) { return registerNewWindowStoreIterator( - key, - segmentMap.subMap(minTime, true, timeTo, true) - .entrySet().iterator(), - true + key, + segmentMap.subMap(minTime, true, timeTo, true) + .entrySet().iterator(), + true ); } else { return registerNewWindowStoreIterator( - key, - segmentMap.subMap(minTime, true, timeTo, true) - .descendingMap().entrySet().iterator(), - false + key, + segmentMap.subMap(minTime, true, timeTo, true) + .descendingMap().entrySet().iterator(), + false ); } } @@ -261,9 +266,9 @@ KeyValueIterator, byte[]> fetch(final Bytes from, if (from != null && to != null && from.compareTo(to) > 0) { LOG.warn("Returning empty iterator for fetch with invalid key range: from > to. " + - "This may be due to range arguments set in the wrong order, " + - "or serdes that don't preserve ordering when lexicographically comparing the serialized bytes. " + - "Note that the built-in numerical serdes do not follow this for negative numbers"); + "This may be due to range arguments set in the wrong order, " + + "or serdes that don't preserve ordering when lexicographically comparing the serialized bytes. " + + "Note that the built-in numerical serdes do not follow this for negative numbers"); return KeyValueIterators.emptyIterator(); } @@ -276,19 +281,19 @@ KeyValueIterator, byte[]> fetch(final Bytes from, if (forward) { return registerNewWindowedKeyValueIterator( - from, - to, - segmentMap.subMap(minTime, true, timeTo, true) - .entrySet().iterator(), - true + from, + to, + segmentMap.subMap(minTime, true, timeTo, true) + .entrySet().iterator(), + true ); } else { return registerNewWindowedKeyValueIterator( - from, - to, - segmentMap.subMap(minTime, true, timeTo, true) - .descendingMap().entrySet().iterator(), - false + from, + to, + segmentMap.subMap(minTime, true, timeTo, true) + .descendingMap().entrySet().iterator(), + false ); } } @@ -315,19 +320,19 @@ KeyValueIterator, byte[]> fetchAll(final long timeFrom, final lo if (forward) { return registerNewWindowedKeyValueIterator( - null, - null, - segmentMap.subMap(minTime, true, timeTo, true) - .entrySet().iterator(), - true + null, + null, + segmentMap.subMap(minTime, true, timeTo, true) + .entrySet().iterator(), + true ); } else { return registerNewWindowedKeyValueIterator( - null, - null, - segmentMap.subMap(minTime, true, timeTo, true) - .descendingMap().entrySet().iterator(), - false + null, + null, + segmentMap.subMap(minTime, true, timeTo, true) + .descendingMap().entrySet().iterator(), + false ); } } @@ -339,10 +344,10 @@ public KeyValueIterator, byte[]> all() { final long minTime = observedStreamTime - retentionPeriod; return registerNewWindowedKeyValueIterator( - null, - null, - segmentMap.tailMap(minTime, false).entrySet().iterator(), - true + null, + null, + segmentMap.tailMap(minTime, false).entrySet().iterator(), + true ); } @@ -353,10 +358,10 @@ public KeyValueIterator, byte[]> backwardAll() { final long minTime = observedStreamTime - retentionPeriod; return registerNewWindowedKeyValueIterator( - null, - null, - segmentMap.tailMap(minTime, false).descendingMap().entrySet().iterator(), - false + null, + null, + segmentMap.tailMap(minTime, false).descendingMap().entrySet().iterator(), + false ); } @@ -376,12 +381,12 @@ public QueryResult query(final Query query, final QueryConfig config) { return StoreQueryUtils.handleBasicQueries( - query, - positionBound, - config, - this, - position, - internalProcessorContext + query, + positionBound, + config, + this, + position, + internalProcessorContext ); } @@ -405,8 +410,8 @@ public void close() { long numEntries() { return segmentMap.values().stream() - .mapToLong(Map::size) - .sum(); + .mapToLong(Map::size) + .sum(); } private void removeExpiredSegments() { @@ -444,7 +449,7 @@ private WrappedInMemoryWindowStoreIterator registerNewWindowStoreIterator(final final Bytes keyTo = retainDuplicates ? wrapForDups(key, Integer.MAX_VALUE) : key; final WrappedInMemoryWindowStoreIterator iterator = - new WrappedInMemoryWindowStoreIterator(keyFrom, keyTo, segmentIterator, openIterators::remove, retainDuplicates, forward); + new WrappedInMemoryWindowStoreIterator(keyFrom, keyTo, segmentIterator, openIterators::remove, retainDuplicates, forward); openIterators.add(iterator); return iterator; @@ -458,14 +463,14 @@ private WrappedWindowedKeyValueIterator registerNewWindowedKeyValueIterator(fina final Bytes to = (retainDuplicates && keyTo != null) ? wrapForDups(keyTo, Integer.MAX_VALUE) : keyTo; final WrappedWindowedKeyValueIterator iterator = - new WrappedWindowedKeyValueIterator( - from, - to, - segmentIterator, - openIterators::remove, - retainDuplicates, - windowSize, - forward); + new WrappedWindowedKeyValueIterator( + from, + to, + segmentIterator, + openIterators::remove, + retainDuplicates, + windowSize, + forward); openIterators.add(iterator); return iterator; } @@ -638,8 +643,8 @@ public static WrappedInMemoryWindowStoreIterator emptyIterator() { } private static class WrappedWindowedKeyValueIterator - extends InMemoryWindowStoreIteratorWrapper - implements KeyValueIterator, byte[]> { + extends InMemoryWindowStoreIteratorWrapper + implements KeyValueIterator, byte[]> { private final long windowSize; @@ -684,4 +689,4 @@ private Windowed getWindowedKey() { return new Windowed<>(key, timeWindow); } } -} +} \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WithRetentionPeriod.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WithRetentionPeriod.java new file mode 100644 index 0000000000000..0ffa460e8b151 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WithRetentionPeriod.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.state.internals; + +public interface WithRetentionPeriod { + long retentionPeriod(); +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java index 72954175f5b4a..6c92e36e1ecad 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java @@ -27,6 +27,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.MockConsumer; import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.clients.consumer.internals.AutoOffsetResetStrategy; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.PartitionInfo; @@ -59,6 +60,7 @@ import java.time.Duration; import java.util.Collections; +import java.util.HashMap; import java.util.Map; import java.util.Properties; import java.util.Set; @@ -146,7 +148,7 @@ public void onRestoreEnd(final TopicPartition tp, final String store, final long private final MockConsumer consumer = new MockConsumer<>(AutoOffsetResetStrategy.EARLIEST.name()); private final MockAdminClient adminClient = new MockAdminClient(); private final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); private void setupStateManagerMock(final Task.TaskType type) { when(stateManager.storeMetadata(tp)).thenReturn(storeMetadata); @@ -189,7 +191,7 @@ public void shouldNotRegisterSameStoreMultipleTimes(final Task.TaskType type) { @Test public void shouldNotRegisterStoreWithoutMetadata() { assertThrows(IllegalStateException.class, - () -> changelogReader.register(new TopicPartition("ChangelogWithoutStoreMetadata", 0), stateManager)); + () -> changelogReader.register(new TopicPartition("ChangelogWithoutStoreMetadata", 0), stateManager)); } @ParameterizedTest @@ -200,7 +202,7 @@ public void shouldSupportUnregisterChangelogBeforeInitialization(final Task.Task adminClient.updateEndOffsets(Collections.singletonMap(tp, 100L)); final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); changelogReader.register(tp, stateManager); @@ -241,7 +243,7 @@ public void shouldSupportUnregisterChangelogBeforeCompletion(final Task.TaskType adminClient.updateEndOffsets(Collections.singletonMap(tp, 100L)); final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); changelogReader.register(tp, stateManager); @@ -296,7 +298,7 @@ public void shouldSupportUnregisterChangelogAfterCompletion(final Task.TaskType adminClient.updateEndOffsets(Collections.singletonMap(tp, 10L)); final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); changelogReader.register(tp, stateManager); @@ -354,7 +356,7 @@ public void shouldPassSuspendReasonToStandbyListener() { adminClient.updateEndOffsets(Collections.singletonMap(tp, 100L)); final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); changelogReader.register(tp, stateManager); changelogReader.transitToUpdateStandby(); @@ -433,7 +435,7 @@ public long position(final TopicPartition partition) { consumer.updateBeginningOffsets(Collections.singletonMap(tp, 5L)); final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); changelogReader.register(tp, stateManager); changelogReader.restore(mockTasks); @@ -488,7 +490,7 @@ public void shouldRestoreFromPositionAndCheckForCompletion(final Task.TaskType t adminClient.updateEndOffsets(Collections.singletonMap(tp, 10L)); final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); changelogReader.register(tp, stateManager); @@ -565,7 +567,7 @@ public void shouldRestoreFromBeginningAndCheckCompletion(final Task.TaskType typ adminClient.updateEndOffsets(Collections.singletonMap(tp, 11L)); final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); changelogReader.register(tp, stateManager); @@ -641,7 +643,7 @@ public void shouldCheckCompletionIfPositionLargerThanEndOffset() { adminClient.updateEndOffsets(Collections.singletonMap(tp, 0L)); final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); changelogReader.register(tp, activeStateManager); changelogReader.restore(mockTasks); @@ -685,7 +687,7 @@ public long position(final TopicPartition partition) { adminClient.updateEndOffsets(Collections.singletonMap(tp, 10L)); final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); changelogReader.register(tp, activeStateManager); changelogReader.restore(Collections.singletonMap(taskId, mockTask)); @@ -727,13 +729,13 @@ public long position(final TopicPartition partition) { adminClient.updateEndOffsets(Collections.singletonMap(tp, 10L)); final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); changelogReader.register(tp, activeStateManager); final StreamsException thrown = assertThrows( - StreamsException.class, - () -> changelogReader.restore(Collections.singletonMap(taskId, mock(Task.class))) + StreamsException.class, + () -> changelogReader.restore(Collections.singletonMap(taskId, mock(Task.class))) ); assertEquals(kaboom, thrown.getCause()); } @@ -775,7 +777,7 @@ public Map committed(final Set topic adminClient.updateEndOffsets(Collections.singletonMap(tp, 0L)); final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); changelogReader.register(tp, activeStateManager); final StreamsException thrown = assertThrows( - StreamsException.class, - () -> changelogReader.restore(Collections.singletonMap(taskId, mock(Task.class))) + StreamsException.class, + () -> changelogReader.restore(Collections.singletonMap(taskId, mock(Task.class))) ); assertEquals(kaboom, thrown.getCause()); } @@ -860,16 +862,16 @@ public synchronized ListConsumerGroupOffsetsResult listConsumerGroupOffsets(fina adminClient.updateConsumerGroupOffsets(Collections.singletonMap(tp, 10L)); final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); changelogReader.register(tp, stateManager); changelogReader.restore(Collections.singletonMap(taskId, mockTask)); assertEquals( - type == ACTIVE ? - StoreChangelogReader.ChangelogState.REGISTERED : - StoreChangelogReader.ChangelogState.RESTORING, - changelogReader.changelogMetadata(tp).state() + type == ACTIVE ? + StoreChangelogReader.ChangelogState.REGISTERED : + StoreChangelogReader.ChangelogState.RESTORING, + changelogReader.changelogMetadata(tp).state() ); if (type == ACTIVE) { assertNull(changelogReader.changelogMetadata(tp).endOffset()); @@ -912,13 +914,13 @@ public synchronized ListConsumerGroupOffsetsResult listConsumerGroupOffsets(fina adminClient.updateEndOffsets(Collections.singletonMap(tp, 10L)); final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); changelogReader.register(tp, stateManager); final StreamsException thrown = assertThrows( - StreamsException.class, - () -> changelogReader.restore(Collections.singletonMap(taskId, mock(Task.class))) + StreamsException.class, + () -> changelogReader.restore(Collections.singletonMap(taskId, mock(Task.class))) ); assertEquals(kaboom, thrown.getCause()); } @@ -932,7 +934,7 @@ public void unsubscribe() { } }; final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); final StreamsException thrown = assertThrows(StreamsException.class, changelogReader::clear); assertEquals(kaboom, thrown.getCause()); @@ -1178,9 +1180,9 @@ public void shouldRestoreMultipleChangelogs() { when(activeStateManager.storeMetadata(tp1)).thenReturn(storeMetadataOne); when(activeStateManager.storeMetadata(tp2)).thenReturn(storeMetadataTwo); when(activeStateManager.changelogOffsets()).thenReturn(mkMap( - mkEntry(tp, 5L), - mkEntry(tp1, 5L), - mkEntry(tp2, 5L) + mkEntry(tp, 5L), + mkEntry(tp1, 5L), + mkEntry(tp2, 5L) )); setupConsumer(10, tp); @@ -1276,8 +1278,8 @@ public void shouldTransitState() { // if a new active is registered, we should immediately transit to standby updating assertThrows( - IllegalStateException.class, - () -> changelogReader.restore(Collections.singletonMap(taskId, mock(Task.class))) + IllegalStateException.class, + () -> changelogReader.restore(Collections.singletonMap(taskId, mock(Task.class))) ); assertEquals(StoreChangelogReader.ChangelogState.RESTORING, changelogReader.changelogMetadata(tp).state()); @@ -1319,13 +1321,13 @@ public void shouldThrowIfRestoreCallbackThrows() { adminClient.updateEndOffsets(Collections.singletonMap(tp, 10L)); final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, exceptionCallback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, exceptionCallback, standbyListener); changelogReader.register(tp, activeStateManager); StreamsException thrown = assertThrows( - StreamsException.class, - () -> changelogReader.restore(Collections.singletonMap(taskId, mock(Task.class))) + StreamsException.class, + () -> changelogReader.restore(Collections.singletonMap(taskId, mock(Task.class))) ); assertEquals(kaboom, thrown.getCause()); @@ -1333,16 +1335,16 @@ public void shouldThrowIfRestoreCallbackThrows() { consumer.addRecord(new ConsumerRecord<>(topicName, 0, 7L, "key".getBytes(), "value".getBytes())); thrown = assertThrows( - StreamsException.class, - () -> changelogReader.restore(Collections.singletonMap(taskId, mock(Task.class))) + StreamsException.class, + () -> changelogReader.restore(Collections.singletonMap(taskId, mock(Task.class))) ); assertEquals(kaboom, thrown.getCause()); consumer.seek(tp, 10L); thrown = assertThrows( - StreamsException.class, - () -> changelogReader.restore(Collections.singletonMap(taskId, mock(Task.class))) + StreamsException.class, + () -> changelogReader.restore(Collections.singletonMap(taskId, mock(Task.class))) ); assertEquals(kaboom, thrown.getCause()); } @@ -1354,9 +1356,9 @@ public void shouldNotThrowOnUnknownRevokedPartition() { changelogReader.unregister(Collections.singletonList(new TopicPartition("unknown", 0))); assertThat( - appender.getMessages(), - hasItem("test-reader Changelog partition unknown-0 could not be found," + - " it could be already cleaned up during the handling of task corruption and never restore again") + appender.getMessages(), + hasItem("test-reader Changelog partition unknown-0 could not be found," + + " it could be already cleaned up during the handling of task corruption and never restore again") ); } } @@ -1422,27 +1424,137 @@ private void setupConsumer(final long messages, final TopicPartition topicPartit private void addRecords(final long messages, final TopicPartition topicPartition) { for (int i = 0; i < messages; i++) { consumer.addRecord(new ConsumerRecord<>( - topicPartition.topic(), - topicPartition.partition(), - i, - new byte[0], - new byte[0])); + topicPartition.topic(), + topicPartition.partition(), + i, + new byte[0], + new byte[0])); } } + @Test + public void shouldSeekByTimestampForWindowedStoreWithoutCheckpoint() { + final long retentionMs = Duration.ofHours(2).toMillis(); + final long offsetForTimestamp = 42L; + + // Use a MockConsumer subclass that supports offsetsForTimes + final MockConsumer timestampConsumer = new MockConsumer<>(AutoOffsetResetStrategy.EARLIEST.name()) { + @Override + public synchronized Map offsetsForTimes(final Map timestampsToSearch) { + final Map result = new HashMap<>(); + timestampsToSearch.forEach((key, value) -> result.put(key, new OffsetAndTimestamp(offsetForTimestamp, value))); + return result; + } + }; + + // Set up mocks - storeMetadata returns null offset (no checkpoint) and positive retentionPeriod + final StateStoreMetadata windowStoreMetadata = mock(StateStoreMetadata.class); + final ProcessorStateManager windowStateManager = mock(ProcessorStateManager.class); + final StateStore windowStore = mock(StateStore.class); + when(windowStoreMetadata.changelogPartition()).thenReturn(tp); + when(windowStoreMetadata.store()).thenReturn(windowStore); + when(windowStoreMetadata.offset()).thenReturn(null); + when(windowStoreMetadata.retentionPeriod()).thenReturn(retentionMs); + when(windowStore.name()).thenReturn(storeName); + when(windowStateManager.storeMetadata(tp)).thenReturn(windowStoreMetadata); + when(windowStateManager.taskType()).thenReturn(ACTIVE); + + final TaskId taskId = new TaskId(0, 0); + when(windowStateManager.taskId()).thenReturn(taskId); + + timestampConsumer.updateBeginningOffsets(Collections.singletonMap(tp, 0L)); + adminClient.updateEndOffsets(Collections.singletonMap(tp, 100L)); + + final StoreChangelogReader reader = + new StoreChangelogReader(time, config, logContext, adminClient, timestampConsumer, callback, standbyListener); + + reader.register(tp, windowStateManager); + reader.restore(Collections.singletonMap(taskId, mock(Task.class))); + + assertEquals(offsetForTimestamp, timestampConsumer.position(tp),"The consumer should be seeked to the offset returned by offsetsForTimes, not to the beginning"); + } + + @Test + public void shouldSeekToBeginningWhenBrokerReturnsNullForOffsetsForTimes() { + final long retentionMs = Duration.ofHours(2).toMillis(); + + // Use a MockConsumer subclass that returns null for offsetsForTimes + final MockConsumer timestampConsumer = new MockConsumer<>(AutoOffsetResetStrategy.EARLIEST.name()) { + @Override + public synchronized Map offsetsForTimes(final Map timestampsToSearch) { + final Map result = new HashMap<>(); + timestampsToSearch.forEach((key, value) -> result.put(key, null)); + return result; + } + }; + + final StateStoreMetadata windowStoreMetadata = mock(StateStoreMetadata.class); + final ProcessorStateManager windowStateManager = mock(ProcessorStateManager.class); + final StateStore windowStore = mock(StateStore.class); + when(windowStoreMetadata.changelogPartition()).thenReturn(tp); + when(windowStoreMetadata.store()).thenReturn(windowStore); + when(windowStoreMetadata.offset()).thenReturn(null); + when(windowStoreMetadata.retentionPeriod()).thenReturn(retentionMs); + when(windowStore.name()).thenReturn(storeName); + when(windowStateManager.storeMetadata(tp)).thenReturn(windowStoreMetadata); + when(windowStateManager.taskType()).thenReturn(ACTIVE); + + final TaskId taskId = new TaskId(0, 0); + when(windowStateManager.taskId()).thenReturn(taskId); + + timestampConsumer.updateBeginningOffsets(Collections.singletonMap(tp, 0L)); + adminClient.updateEndOffsets(Collections.singletonMap(tp, 100L)); + + final StoreChangelogReader reader = + new StoreChangelogReader(time, config, logContext, adminClient, timestampConsumer, callback, standbyListener); + + reader.register(tp, windowStateManager); + reader.restore(Collections.singletonMap(taskId, mock(Task.class))); + + assertEquals(0L, timestampConsumer.position(tp),"When broker returns null, should fall back to seeking to the beginning"); + } + + @Test + public void shouldSeekToBeginningForNonWindowedStoreWithoutCheckpoint() { + final StateStoreMetadata kvStoreMetadata = mock(StateStoreMetadata.class); + final ProcessorStateManager kvStateManager = mock(ProcessorStateManager.class); + final StateStore kvStore = mock(StateStore.class); + when(kvStoreMetadata.changelogPartition()).thenReturn(tp); + when(kvStoreMetadata.store()).thenReturn(kvStore); + when(kvStoreMetadata.offset()).thenReturn(null); + when(kvStoreMetadata.retentionPeriod()).thenReturn(-1L); + when(kvStore.name()).thenReturn(storeName); + when(kvStateManager.storeMetadata(tp)).thenReturn(kvStoreMetadata); + when(kvStateManager.taskType()).thenReturn(ACTIVE); + + final TaskId taskId = new TaskId(0, 0); + when(kvStateManager.taskId()).thenReturn(taskId); + + consumer.updateBeginningOffsets(Collections.singletonMap(tp, 0L)); + adminClient.updateEndOffsets(Collections.singletonMap(tp, 100L)); + + final StoreChangelogReader reader = + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + + reader.register(tp, kvStateManager); + reader.restore(Collections.singletonMap(taskId, mock(Task.class))); + + assertEquals(0L, consumer.position(tp),"Non-windowed store should seek to beginning, not by timestamp"); + } + private void assignPartition(final long messages, final TopicPartition topicPartition) { consumer.updatePartitions( - topicPartition.topic(), - Collections.singletonList(new PartitionInfo( topicPartition.topic(), - topicPartition.partition(), - null, - null, - null))); + Collections.singletonList(new PartitionInfo( + topicPartition.topic(), + topicPartition.partition(), + null, + null, + null))); consumer.updateBeginningOffsets(Collections.singletonMap(topicPartition, 0L)); consumer.updateEndOffsets(Collections.singletonMap(topicPartition, Math.max(0, messages) + 1)); adminClient.updateEndOffsets(Collections.singletonMap(topicPartition, Math.max(0, messages) + 1)); consumer.assign(Collections.singletonList(topicPartition)); } -} +} \ No newline at end of file From 87ed81ffdb702ab86cb149539391ce52f91a00b1 Mon Sep 17 00:00:00 2001 From: Gabriella Fu <2234862823@qq.com> Date: Mon, 30 Mar 2026 17:40:10 -0400 Subject: [PATCH 02/23] revert some space --- .../state/internals/InMemorySessionStore.java | 114 +++++++++--------- 1 file changed, 57 insertions(+), 57 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java index d84acacf0cbb0..b524f49f1281d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java @@ -67,10 +67,10 @@ public class InMemorySessionStore implements SessionStore, WithRe private final long retentionPeriod; private static final String INVALID_RANGE_WARN_MSG = - "Returning empty iterator for fetch with invalid key range: from > to. " + - "This may be due to range arguments set in the wrong order, " + - "or serdes that don't preserve ordering when lexicographically comparing the serialized bytes. " + - "Note that the built-in numerical serdes do not follow this for negative numbers"; + "Returning empty iterator for fetch with invalid key range: from > to. " + + "This may be due to range arguments set in the wrong order, " + + "or serdes that don't preserve ordering when lexicographically comparing the serialized bytes. " + + "Note that the built-in numerical serdes do not follow this for negative numbers"; private final ConcurrentNavigableMap>> endTimeMap = new ConcurrentSkipListMap<>(); private final Set openIterators = ConcurrentHashMap.newKeySet(); @@ -130,19 +130,19 @@ public void init(final StateStoreContext stateStoreContext, false ); stateStoreContext.register( - root, - (RecordBatchingStateRestoreCallback) records -> { - synchronized (position) { - for (final ConsumerRecord record : records) { - put(SessionKeySchema.from(Bytes.wrap(record.key())), record.value()); - ChangelogRecordDeserializationHelper.applyChecksAndUpdatePosition( - record, - consistencyEnabled, - position - ); - } + root, + (RecordBatchingStateRestoreCallback) records -> { + synchronized (position) { + for (final ConsumerRecord record : records) { + put(SessionKeySchema.from(Bytes.wrap(record.key())), record.value()); + ChangelogRecordDeserializationHelper.applyChecksAndUpdatePosition( + record, + consistencyEnabled, + position + ); } } + } ); } open = true; @@ -232,7 +232,7 @@ public KeyValueIterator, byte[]> findSessions(final long earlies removeExpiredSegments(); final ConcurrentNavigableMap>> endTimSubMap - = endTimeMap.subMap(earliestSessionEndTime, true, latestSessionEndTime, true); + = endTimeMap.subMap(earliestSessionEndTime, true, latestSessionEndTime, true); return registerNewIterator(null, null, Long.MAX_VALUE, endTimSubMap.entrySet().iterator(), true); } @@ -246,10 +246,10 @@ public KeyValueIterator, byte[]> findSessions(final Bytes key, removeExpiredSegments(); return registerNewIterator(key, - key, - latestSessionStartTime, - endTimeMap.tailMap(earliestSessionEndTime, true).entrySet().iterator(), - true); + key, + latestSessionStartTime, + endTimeMap.tailMap(earliestSessionEndTime, true).entrySet().iterator(), + true); } @Override @@ -261,11 +261,11 @@ public KeyValueIterator, byte[]> backwardFindSessions(final Byte removeExpiredSegments(); return registerNewIterator( - key, - key, - latestSessionStartTime, - endTimeMap.tailMap(earliestSessionEndTime, true).descendingMap().entrySet().iterator(), - false + key, + key, + latestSessionStartTime, + endTimeMap.tailMap(earliestSessionEndTime, true).descendingMap().entrySet().iterator(), + false ); } @@ -282,10 +282,10 @@ public KeyValueIterator, byte[]> findSessions(final Bytes keyFro } return registerNewIterator(keyFrom, - keyTo, - latestSessionStartTime, - endTimeMap.tailMap(earliestSessionEndTime, true).entrySet().iterator(), - true); + keyTo, + latestSessionStartTime, + endTimeMap.tailMap(earliestSessionEndTime, true).entrySet().iterator(), + true); } @Override @@ -301,11 +301,11 @@ public KeyValueIterator, byte[]> backwardFindSessions(final Byte } return registerNewIterator( - keyFrom, - keyTo, - latestSessionStartTime, - endTimeMap.tailMap(earliestSessionEndTime, true).descendingMap().entrySet().iterator(), - false + keyFrom, + keyTo, + latestSessionStartTime, + endTimeMap.tailMap(earliestSessionEndTime, true).descendingMap().entrySet().iterator(), + false ); } @@ -341,7 +341,7 @@ public KeyValueIterator, byte[]> backwardFetch(final Bytes keyFr removeExpiredSegments(); return registerNewIterator( - keyFrom, keyTo, Long.MAX_VALUE, endTimeMap.descendingMap().entrySet().iterator(), false); + keyFrom, keyTo, Long.MAX_VALUE, endTimeMap.descendingMap().entrySet().iterator(), false); } @Override @@ -360,12 +360,12 @@ public QueryResult query(final Query query, final QueryConfig config) { return StoreQueryUtils.handleBasicQueries( - query, - positionBound, - config, - this, - position, - context + query, + positionBound, + config, + this, + position, + context ); } @@ -390,9 +390,9 @@ public void close() { long numEntries() { return endTimeMap.values().stream() - .flatMap(keyMap -> keyMap.values().stream()) - .mapToLong(Map::size) - .sum(); + .flatMap(keyMap -> keyMap.values().stream()) + .mapToLong(Map::size) + .sum(); } private void removeExpiredSegments() { @@ -411,14 +411,14 @@ private InMemorySessionStoreIterator registerNewIterator(final Bytes keyFrom, final Iterator>>> endTimeIterator, final boolean forward) { final InMemorySessionStoreIterator iterator = - new InMemorySessionStoreIterator( - keyFrom, - keyTo, - latestSessionStartTime, - endTimeIterator, - openIterators::remove, - forward - ); + new InMemorySessionStoreIterator( + keyFrom, + keyTo, + latestSessionStartTime, + endTimeIterator, + openIterators::remove, + forward + ); openIterators.add(iterator); return iterator; } @@ -569,13 +569,13 @@ private boolean setInnerIterators() { } else { if (forward) { recordIterator = nextKeyEntry.getValue() - .headMap(latestSessionStartTime, true) - .descendingMap() - .entrySet().iterator(); + .headMap(latestSessionStartTime, true) + .descendingMap() + .entrySet().iterator(); } else { recordIterator = nextKeyEntry.getValue() - .headMap(latestSessionStartTime, true) - .entrySet().iterator(); + .headMap(latestSessionStartTime, true) + .entrySet().iterator(); } } From b4a9ff0b89888b17da289286188be7a476f94a75 Mon Sep 17 00:00:00 2001 From: Gabriella Fu <2234862823@qq.com> Date: Mon, 30 Mar 2026 17:46:54 -0400 Subject: [PATCH 03/23] revert StoreChangelogReader.java --- .../internals/StoreChangelogReader.java | 180 +++++++++--------- 1 file changed, 90 insertions(+), 90 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java index 7b43cee270ff4..aa7098400a1f6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java @@ -168,7 +168,7 @@ private void transitTo(final ChangelogState newState) { public String toString() { final Long currentOffset = storeMetadata.offset(); return changelogState + " " + stateManager.taskType() + - " (currentOffset " + currentOffset + ", endOffset " + restoreEndOffset + ")"; + " (currentOffset " + currentOffset + ", endOffset " + restoreEndOffset + ")"; } // for testing only below @@ -244,7 +244,7 @@ public StoreChangelogReader(final Time time, this.groupId = config.getString(StreamsConfig.APPLICATION_ID_CONFIG); this.pollTime = Duration.ofMillis(config.getLong(StreamsConfig.POLL_MS_CONFIG)); this.updateOffsetIntervalMs = config.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG) == Long.MAX_VALUE ? - DEFAULT_OFFSET_UPDATE_MS : config.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG); + DEFAULT_OFFSET_UPDATE_MS : config.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG); this.lastUpdateOffsetTime = 0L; this.changelogs = new HashMap<>(); @@ -260,7 +260,7 @@ private boolean hasRestoredToEnd(final ChangelogMetadata metadata) { // end offset is not initialized meaning that it is from a standby task, // this should never happen since we only call this function for active task in restoring phase throw new IllegalStateException("End offset for changelog " + metadata + " is unknown when deciding " + - "if it has completed restoration, this should never happen."); + "if it has completed restoration, this should never happen."); } else if (endOffset == 0) { // this is a special case, meaning there's nothing to be restored since the changelog has no data // OR the changelog is a source topic and there's no committed offset @@ -288,7 +288,7 @@ private boolean hasRestoredToEnd(final ChangelogMetadata metadata) { // this also includes InvalidOffsetException, which should not happen under normal // execution, hence it is also okay to wrap it as fatal StreamsException throw new StreamsException("Restore consumer get unexpected error trying to get the position " + - " of " + partition, e); + " of " + partition, e); } } else { return metadata.bufferedRecords.get(0).offset() >= endOffset; @@ -325,8 +325,8 @@ public void enforceRestoreActive() { public void transitToUpdateStandby() { if (state != ChangelogReaderState.ACTIVE_RESTORING) { throw new IllegalStateException( - "The changelog reader is not restoring active tasks (is " + state + ") while trying to " + - "transit to update standby tasks: " + changelogs + "The changelog reader is not restoring active tasks (is " + state + ") while trying to " + + "transit to update standby tasks: " + changelogs ); } @@ -352,7 +352,7 @@ public void register(final TopicPartition partition, final ProcessorStateManager final StateStoreMetadata storeMetadata = stateManager.storeMetadata(partition); if (storeMetadata == null) { throw new IllegalStateException("Cannot find the corresponding state store metadata for changelog " + - partition); + partition); } final ChangelogMetadata changelogMetadata = new ChangelogMetadata(storeMetadata, stateManager); @@ -364,7 +364,7 @@ public void register(final TopicPartition partition, final ProcessorStateManager if (changelogs.putIfAbsent(partition, changelogMetadata) != null) { throw new IllegalStateException("There is already a changelog registered for " + partition + - ", this should not happen: " + changelogs); + ", this should not happen: " + changelogs); } } @@ -379,11 +379,11 @@ private ChangelogMetadata restoringChangelogByPartition(final TopicPartition par final ChangelogMetadata changelogMetadata = changelogs.get(partition); if (changelogMetadata == null) { throw new IllegalStateException("The corresponding changelog restorer for " + partition + - " does not exist, this should not happen."); + " does not exist, this should not happen."); } if (changelogMetadata.changelogState != ChangelogState.RESTORING) { throw new IllegalStateException("The corresponding changelog restorer for " + partition + - " has already transited to completed state, this should not happen."); + " has already transited to completed state, this should not happen."); } return changelogMetadata; @@ -391,45 +391,45 @@ private ChangelogMetadata restoringChangelogByPartition(final TopicPartition par private Set registeredChangelogs() { return changelogs.values().stream() - .filter(metadata -> metadata.changelogState == ChangelogState.REGISTERED) - .collect(Collectors.toSet()); + .filter(metadata -> metadata.changelogState == ChangelogState.REGISTERED) + .collect(Collectors.toSet()); } private Set restoringChangelogs() { return changelogs.values().stream() - .filter(metadata -> metadata.changelogState == ChangelogState.RESTORING) - .map(metadata -> metadata.storeMetadata.changelogPartition()) - .collect(Collectors.toSet()); + .filter(metadata -> metadata.changelogState == ChangelogState.RESTORING) + .map(metadata -> metadata.storeMetadata.changelogPartition()) + .collect(Collectors.toSet()); } private Set activeRestoringChangelogs() { return changelogs.values().stream() - .filter(metadata -> metadata.changelogState == ChangelogState.RESTORING && - metadata.stateManager.taskType() == Task.TaskType.ACTIVE) - .map(metadata -> metadata.storeMetadata.changelogPartition()) - .collect(Collectors.toSet()); + .filter(metadata -> metadata.changelogState == ChangelogState.RESTORING && + metadata.stateManager.taskType() == Task.TaskType.ACTIVE) + .map(metadata -> metadata.storeMetadata.changelogPartition()) + .collect(Collectors.toSet()); } private Set standbyRestoringChangelogs() { return changelogs.values().stream() - .filter(metadata -> metadata.changelogState == ChangelogState.RESTORING && - metadata.stateManager.taskType() == Task.TaskType.STANDBY) - .map(metadata -> metadata.storeMetadata.changelogPartition()) - .collect(Collectors.toSet()); + .filter(metadata -> metadata.changelogState == ChangelogState.RESTORING && + metadata.stateManager.taskType() == Task.TaskType.STANDBY) + .map(metadata -> metadata.storeMetadata.changelogPartition()) + .collect(Collectors.toSet()); } @Override public boolean allChangelogsCompleted() { return changelogs.values().stream() - .allMatch(metadata -> metadata.changelogState == ChangelogState.COMPLETED); + .allMatch(metadata -> metadata.changelogState == ChangelogState.COMPLETED); } @Override public Set completedChangelogs() { return changelogs.values().stream() - .filter(metadata -> metadata.changelogState == ChangelogState.COMPLETED) - .map(metadata -> metadata.storeMetadata.changelogPartition()) - .collect(Collectors.toSet()); + .filter(metadata -> metadata.changelogState == ChangelogState.COMPLETED) + .map(metadata -> metadata.storeMetadata.changelogPartition()) + .collect(Collectors.toSet()); } // 1. if there are any registered changelogs that needs initialization, try to initialize them first; @@ -471,8 +471,8 @@ public long restore(final Map tasks) { totalRestored += restoreChangelog(task, changelogMetadata); } catch (final TimeoutException timeoutException) { tasks.get(taskId).maybeInitTaskTimeoutOrThrow( - time.milliseconds(), - timeoutException + time.milliseconds(), + timeoutException ); } } @@ -498,10 +498,10 @@ private ConsumerRecords pollRecordsFromRestoreConsumer(final Map // TODO (?) If we cannot fetch records for standby task, should we trigger `task.timeout.ms` ? } catch (final InvalidOffsetException e) { log.warn("Encountered " + e.getClass().getName() + - " fetching records from restore consumer for partitions " + e.partitions() + ", it is likely that " + - "the consumer's position has fallen out of the topic partition offset range because the topic was " + - "truncated or compacted on the broker, marking the corresponding tasks as corrupted and re-initializing " + - "it later.", e); + " fetching records from restore consumer for partitions " + e.partitions() + ", it is likely that " + + "the consumer's position has fallen out of the topic partition offset range because the topic was " + + "truncated or compacted on the broker, marking the corresponding tasks as corrupted and re-initializing " + + "it later.", e); final Set corruptedTasks = new HashSet<>(); e.partitions().forEach(partition -> corruptedTasks.add(changelogs.get(partition).stateManager.taskId())); @@ -529,9 +529,9 @@ private void updatePartitionsByType(final Map tasks, final Set restoringChangelogs, final TaskType taskType) { final Collection toResume = - restoringChangelogs.stream().filter(t -> shouldResume(tasks, t, taskType)).collect(Collectors.toList()); + restoringChangelogs.stream().filter(t -> shouldResume(tasks, t, taskType)).collect(Collectors.toList()); final Collection toPause = - restoringChangelogs.stream().filter(t -> shouldPause(tasks, t, taskType)).collect(Collectors.toList()); + restoringChangelogs.stream().filter(t -> shouldPause(tasks, t, taskType)).collect(Collectors.toList()); restoreConsumer.resume(toResume); restoreConsumer.pause(toPause); } @@ -562,20 +562,20 @@ private void maybeLogRestorationProgress() { final Set topicPartitions = activeRestoringChangelogs(); if (!topicPartitions.isEmpty()) { final StringBuilder builder = new StringBuilder().append("Restoration in progress for ") - .append(topicPartitions.size()) - .append(" partitions."); + .append(topicPartitions.size()) + .append(" partitions."); for (final TopicPartition partition : topicPartitions) { final ChangelogMetadata changelogMetadata = restoringChangelogByPartition(partition); builder.append(" {") - .append(partition) - .append(": ") - .append("position=") - .append(getPositionString(partition, changelogMetadata)) - .append(", end=") - .append(changelogMetadata.restoreEndOffset) - .append(", totalRestored=") - .append(changelogMetadata.totalRestored) - .append("}"); + .append(partition) + .append(": ") + .append("position=") + .append(getPositionString(partition, changelogMetadata)) + .append(", end=") + .append(changelogMetadata.restoreEndOffset) + .append(", totalRestored=") + .append(changelogMetadata.totalRestored) + .append("}"); } log.info(builder.toString()); lastRestoreLogTime = time.milliseconds(); @@ -596,18 +596,18 @@ private static String getPositionString(final TopicPartition partition, private void maybeUpdateLimitOffsetsForStandbyChangelogs(final Map tasks) { // we only consider updating the limit offset for standbys if we are not restoring active tasks if (state == ChangelogReaderState.STANDBY_UPDATING && - updateOffsetIntervalMs < time.milliseconds() - lastUpdateOffsetTime) { + updateOffsetIntervalMs < time.milliseconds() - lastUpdateOffsetTime) { // when the interval has elapsed we should try to update the limit offset for standbys reading from - // a source changelog with the new committed offset, unless there are no buffered records since + // a source changelog with the new committed offset, unless there are no buffered records since // we only need the limit when processing new records // for other changelog partitions we do not need to update limit offset at all since we never need to // check when it completes based on limit offset anyways: the end offset would keep increasing and the // standby never need to stop final Set changelogsWithLimitOffsets = changelogs.entrySet().stream() - .filter(entry -> entry.getValue().stateManager.taskType() == Task.TaskType.STANDBY && - entry.getValue().stateManager.changelogAsSource(entry.getKey())) - .map(Map.Entry::getKey).collect(Collectors.toSet()); + .filter(entry -> entry.getValue().stateManager.taskType() == Task.TaskType.STANDBY && + entry.getValue().stateManager.changelogAsSource(entry.getKey())) + .map(Map.Entry::getKey).collect(Collectors.toSet()); for (final TopicPartition partition : changelogsWithLimitOffsets) { if (!changelogs.get(partition).bufferedRecords().isEmpty()) { @@ -625,7 +625,7 @@ private void bufferChangelogRecords(final ChangelogMetadata changelogMetadata, f // filter polled records for null-keys and also possibly update buffer limit index if (record.key() == null) { log.warn("Read changelog record with null key from changelog {} at offset {}, " + - "skipping it for restoration", changelogMetadata.storeMetadata.changelogPartition(), record.offset()); + "skipping it for restoration", changelogMetadata.storeMetadata.changelogPartition(), record.offset()); } else { changelogMetadata.bufferedRecords.add(record); final long offset = record.offset(); @@ -667,7 +667,7 @@ private int restoreChangelog(final Task task, final ChangelogMetadata changelogM final Long currentOffset = storeMetadata.offset(); log.trace("Restored {} records from changelog {} to store {}, end offset is {}, current offset is {}", - numRecords, partition, storeName, recordEndOffset(changelogMetadata.restoreEndOffset), currentOffset); + numRecords, partition, storeName, recordEndOffset(changelogMetadata.restoreEndOffset), currentOffset); changelogMetadata.bufferedLimitIndex = 0; changelogMetadata.totalRestored += numRecords; @@ -691,7 +691,7 @@ private int restoreChangelog(final Task task, final ChangelogMetadata changelogM // we should check even if there's nothing restored, but do not check completed if we are processing standby tasks if (changelogMetadata.stateManager.taskType() == Task.TaskType.ACTIVE && hasRestoredToEnd(changelogMetadata)) { log.info("Finished restoring changelog {} to store {} with a total number of {} records", - partition, storeName, changelogMetadata.totalRestored); + partition, storeName, changelogMetadata.totalRestored); changelogMetadata.transitTo(ChangelogState.COMPLETED); pauseChangelogsFromRestoreConsumer(Collections.singleton(partition)); @@ -744,23 +744,23 @@ private Map committedOffsetForChangelogs(final Map(partitions)); + .topicPartitions(new ArrayList<>(partitions)); final Map committedOffsets = - adminClient.listConsumerGroupOffsets( - Collections.singletonMap(groupId, spec), - options - ) - .partitionsToOffsetAndMetadata(groupId).get().entrySet() - .stream() - .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue() == null ? 0L : e.getValue().offset())); + adminClient.listConsumerGroupOffsets( + Collections.singletonMap(groupId, spec), + options + ) + .partitionsToOffsetAndMetadata(groupId).get().entrySet() + .stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue() == null ? 0L : e.getValue().offset())); clearTaskTimeout(getTasksFromPartitions(tasks, partitions)); return committedOffsets; } catch (final TimeoutException | InterruptedException | ExecutionException retriableException) { log.debug("Could not retrieve the committed offsets for partitions {} due to {}, will retry in the next run loop", - partitions, retriableException.toString()); + partitions, retriableException.toString()); maybeInitTaskTimeoutOrThrow(getTasksFromPartitions(tasks, partitions), retriableException); return Collections.emptyMap(); } catch (final KafkaException e) { @@ -770,7 +770,7 @@ private Map committedOffsetForChangelogs(final Map tasks, final Set newPartitionsToRestore) { newPartitionsToRestore.removeIf(changelogMetadata -> - !tasks.containsKey(changelogs.get(changelogMetadata.storeMetadata.changelogPartition()).stateManager.taskId())); + !tasks.containsKey(changelogs.get(changelogMetadata.storeMetadata.changelogPartition()).stateManager.taskId())); } private Map endOffsetForChangelogs(final Map tasks, final Set partitions) { @@ -783,16 +783,16 @@ private Map endOffsetForChangelogs(final Map // see KAFKA-10167 for more details final ListOffsetsOptions options = new ListOffsetsOptions(IsolationLevel.READ_UNCOMMITTED); final Map offsetSpecs = - partitions.stream().collect(Collectors.toMap(Function.identity(), tp -> OffsetSpec.latest())); + partitions.stream().collect(Collectors.toMap(Function.identity(), tp -> OffsetSpec.latest())); final Map logEndOffsets = adminClient.listOffsets(offsetSpecs, options) - .all().get().entrySet() - .stream().collect(Collectors.toMap(Map.Entry::getKey, entry -> entry.getValue().offset())); + .all().get().entrySet() + .stream().collect(Collectors.toMap(Map.Entry::getKey, entry -> entry.getValue().offset())); clearTaskTimeout(getTasksFromPartitions(tasks, partitions)); return logEndOffsets; } catch (final TimeoutException | InterruptedException | ExecutionException retriableException) { log.debug("Could not fetch all end offsets for {} due to {}, will retry in the next run loop", - partitions, retriableException.toString()); + partitions, retriableException.toString()); maybeInitTaskTimeoutOrThrow(getTasksFromPartitions(tasks, partitions), retriableException); return Collections.emptyMap(); } catch (final KafkaException e) { @@ -804,22 +804,22 @@ private void updateLimitOffsetsForStandbyChangelogs(final Map newLimit) { throw new IllegalStateException("Offset limit should monotonically increase, but was reduced for partition " + - partition + ". New limit: " + newLimit + ". Previous limit: " + previousLimit); + partition + ". New limit: " + newLimit + ". Previous limit: " + previousLimit); } metadata.restoreEndOffset = newLimit; // update the limit index for buffered records while (metadata.bufferedLimitIndex < metadata.bufferedRecords.size() && - metadata.bufferedRecords.get(metadata.bufferedLimitIndex).offset() < metadata.restoreEndOffset) + metadata.bufferedRecords.get(metadata.bufferedLimitIndex).offset() < metadata.restoreEndOffset) metadata.bufferedLimitIndex++; } } @@ -865,13 +865,13 @@ private void initializeChangelogs(final Map tasks, final ChangelogMetadata changelogMetadata = changelogs.get(partition); final Long endOffset = endOffsets.get(partition); final Long committedOffset = newPartitionsToFindCommittedOffset.contains(partition) ? - committedOffsets.get(partition) : Long.valueOf(Long.MAX_VALUE); + committedOffsets.get(partition) : Long.valueOf(Long.MAX_VALUE); if (endOffset != null && committedOffset != null) { if (changelogMetadata.restoreEndOffset != null) { throw new IllegalStateException("End offset for " + partition + - " should only be initialized once. Existing value: " + changelogMetadata.restoreEndOffset + - ", new value: (" + endOffset + ", " + committedOffset + ")"); + " should only be initialized once. Existing value: " + changelogMetadata.restoreEndOffset + + ", new value: (" + endOffset + ", " + committedOffset + ")"); } changelogMetadata.restoreEndOffset = Math.min(endOffset, committedOffset); @@ -880,7 +880,7 @@ private void initializeChangelogs(final Map tasks, } else { if (!newPartitionsToRestore.remove(changelogMetadata)) { throw new IllegalStateException("New changelogs to restore " + newPartitionsToRestore + - " does not contain the one looking for end offset: " + partition + ", this should not happen."); + " does not contain the one looking for end offset: " + partition + ", this should not happen."); } log.info("End offset for changelog {} cannot be found; will retry in the next time.", partition); @@ -894,7 +894,7 @@ private void initializeChangelogs(final Map tasks, // add new partitions to the restore consumer and transit them to restoring state addChangelogsToRestoreConsumer(newPartitionsToRestore.stream().map(metadata -> metadata.storeMetadata.changelogPartition()) - .collect(Collectors.toSet())); + .collect(Collectors.toSet())); newPartitionsToRestore.forEach(metadata -> metadata.transitTo(ChangelogState.RESTORING)); @@ -917,7 +917,7 @@ private void addChangelogsToRestoreConsumer(final Set partitions // the current assignment should not contain any of the new partitions if (assignment.removeAll(partitions)) { throw new IllegalStateException("The current assignment " + restoreConsumer.assignment() + " " + - "already contains some of the new partitions " + partitions); + "already contains some of the new partitions " + partitions); } assignment.addAll(partitions); restoreConsumer.assign(assignment); @@ -931,7 +931,7 @@ private void pauseChangelogsFromRestoreConsumer(final Collection // the current assignment should contain all the partitions to pause if (!assignment.containsAll(partitions)) { throw new IllegalStateException("The current assignment " + assignment + " " + - "does not contain some of the partitions " + partitions + " for pausing."); + "does not contain some of the partitions " + partitions + " for pausing."); } restoreConsumer.pause(partitions); @@ -948,7 +948,7 @@ private void removeChangelogsFromRestoreConsumer(final Collection tasks, restoreConsumer.seek(partition, currentOffset + 1); log.debug("Start restoring changelog partition {} from current offset {} to end offset {}.", - partition, currentOffset, recordEndOffset(endOffset)); + partition, currentOffset, recordEndOffset(endOffset)); } else { final long retentionPeriod = storeMetadata.retentionPeriod(); final long seekTimestamp = retentionPeriod > 0 && retentionPeriod != Long.MAX_VALUE - ? time.milliseconds() - retentionPeriod : -1L; + ? time.milliseconds() - retentionPeriod : -1L; if (seekTimestamp > 0) { newPartitionsWithTimestampSeek.put(partition, seekTimestamp); log.debug("Start restoring windowed changelog partition {} from timestamp {} to end offset {}.", - partition, seekTimestamp, recordEndOffset(endOffset)); + partition, seekTimestamp, recordEndOffset(endOffset)); } else { log.debug("Start restoring changelog partition {} from the beginning offset to end offset {} " + - "since we cannot find current offset.", partition, recordEndOffset(endOffset)); + "since we cannot find current offset.", partition, recordEndOffset(endOffset)); newPartitionsWithoutStartOffset.add(partition); } } @@ -1050,7 +1050,7 @@ private void seekToTimestampOrBeginning(final Map partitio // optimization: seek windowed stores by timestamp to skip expired data if (!partitionsWithTimestampSeek.isEmpty()) { final Map offsetsByTimestamp = - restoreConsumer.offsetsForTimes(partitionsWithTimestampSeek); + restoreConsumer.offsetsForTimes(partitionsWithTimestampSeek); offsetsByTimestamp.forEach((key, value) -> { if (value != null) { restoreConsumer.seek(key, value.offset()); @@ -1115,8 +1115,8 @@ public void unregister(final Collection revokedChangelogs, changelogMetadata.clear(); } else { log.debug("Changelog partition {} could not be found," + - " it could be already cleaned up during the handling" + - " of task corruption and never restore again", partition); + " it could be already cleaned up during the handling" + + " of task corruption and never restore again", partition); } } @@ -1161,4 +1161,4 @@ ChangelogMetadata changelogMetadata(final TopicPartition partition) { ChangelogReaderState state() { return state; } -} \ No newline at end of file +} From 8d7418fc998d5eacb0b8ad5973cc311760d0f948 Mon Sep 17 00:00:00 2001 From: Gabriella Fu <2234862823@qq.com> Date: Mon, 30 Mar 2026 17:52:12 -0400 Subject: [PATCH 04/23] revert ProcessorStateManager.java --- .../internals/ProcessorStateManager.java | 64 +++++++++---------- 1 file changed, 32 insertions(+), 32 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java index a67ba230d3775..10963c8bf82e6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java @@ -357,7 +357,7 @@ void initializeStoreOffsets(final boolean storeDirIsEmpty) { stateDirectory.updateTaskOffsets(taskId, changelogOffsets()); } catch (final RuntimeException e) { throw new ProcessorStateException(format("%sError updating state directory offsets when creating the state manager", - logPrefix), e); + logPrefix), e); } } @@ -377,7 +377,7 @@ public void registerStore(final StateStore store, if (LegacyCheckpointingStateStore.CHECKPOINT_FILE_NAME.startsWith(storeName)) { store.close(); throw new IllegalArgumentException(format("%sIllegal store name: %s, which collides with the pre-defined " + - "checkpoint file name", logPrefix, storeName)); + "checkpoint file name", logPrefix, storeName)); } if (stores.containsKey(storeName)) { @@ -391,13 +391,13 @@ public void registerStore(final StateStore store, } final StateStoreMetadata storeMetadata = isLoggingEnabled(storeName) ? - new StateStoreMetadata( - store, - getStorePartition(storeName), - stateRestoreCallback, - commitCallback, - converterForStore(store)) : - new StateStoreMetadata(store, commitCallback); + new StateStoreMetadata( + store, + getStorePartition(storeName), + stateRestoreCallback, + commitCallback, + converterForStore(store)) : + new StateStoreMetadata(store, commitCallback); // register the store first, so that if later an exception is thrown then eventually while we call `close` // on the state manager this state store would be closed as well @@ -430,7 +430,7 @@ void markChangelogAsCorrupted(final Collection partitions) { if (!partitionsToMarkAsCorrupted.isEmpty()) { throw new IllegalStateException("Some partitions " + partitionsToMarkAsCorrupted + " are not contained in " + - "the store list of task " + taskId + " marking as corrupted, this is not expected"); + "the store list of task " + taskId + " marking as corrupted, this is not expected"); } } @@ -443,8 +443,8 @@ public Map changelogOffsets() { // for changelog whose offset is unknown, use 0L indicating earliest offset // otherwise return the current offset + 1 as the next offset to fetch changelogOffsets.put( - storeMetadata.changelogPartition, - storeMetadata.offset == null ? 0L : storeMetadata.offset + 1L); + storeMetadata.changelogPartition, + storeMetadata.offset == null ? 0L : storeMetadata.offset + 1L); } } return changelogOffsets; @@ -486,7 +486,7 @@ StateStoreMetadata storeMetadata(final TopicPartition partition) { void restore(final StateStoreMetadata storeMetadata, final List> restoreRecords, final OptionalLong optionalLag) { if (!stores.containsValue(storeMetadata)) { throw new IllegalStateException("Restoring " + storeMetadata + " which is not registered in this state manager, " + - "this should not happen."); + "this should not happen."); } if (!restoreRecords.isEmpty()) { @@ -494,15 +494,15 @@ void restore(final StateStoreMetadata storeMetadata, final List> convertedRecords = restoreRecords.stream() - .map(storeMetadata.recordConverter::convert) - .collect(Collectors.toList()); + .map(storeMetadata.recordConverter::convert) + .collect(Collectors.toList()); try { restoreCallback.restoreBatch(convertedRecords); } catch (final RuntimeException e) { throw new ProcessorStateException( - format("%sException caught while trying to restore state from %s", logPrefix, storeMetadata.changelogPartition), - e + format("%sException caught while trying to restore state from %s", logPrefix, storeMetadata.changelogPartition), + e ); } @@ -598,14 +598,14 @@ public void flushCache() { // In case of FailedProcessingException Do not keep the failed processing exception in the stack trace if (exception instanceof FailedProcessingException) { firstException = new ProcessorStateException( - format("%sFailed to flush cache of store %s", logPrefix, store.name()), - exception.getCause()); + format("%sFailed to flush cache of store %s", logPrefix, store.name()), + exception.getCause()); } else if (exception instanceof StreamsException) { firstException = exception; } else { firstException = new ProcessorStateException( - format("%sFailed to flush cache of store %s", logPrefix, store.name()), - exception + format("%sFailed to flush cache of store %s", logPrefix, store.name()), + exception ); } log.error("Failed to flush cache of store {}: ", store.name(), firstException); @@ -648,13 +648,13 @@ public void close() throws ProcessorStateException { // In case of FailedProcessingException Do not keep the failed processing exception in the stack trace if (exception instanceof FailedProcessingException) firstException = new ProcessorStateException( - format("%sFailed to close state store %s", logPrefix, store.name()), - exception.getCause()); + format("%sFailed to close state store %s", logPrefix, store.name()), + exception.getCause()); else if (exception instanceof StreamsException) firstException = exception; else firstException = new ProcessorStateException( - format("%sFailed to close state store %s", logPrefix, store.name()), exception); + format("%sFailed to close state store %s", logPrefix, store.name()), exception); log.error("Failed to close state store {}: ", store.name(), firstException); } else { log.error("Failed to close state store {}: ", store.name(), exception); @@ -750,12 +750,12 @@ private boolean isLoggingEnabled(final String storeName) { private StateStoreMetadata findStore(final TopicPartition changelogPartition) { final List found = stores.values().stream() - .filter(metadata -> changelogPartition.equals(metadata.changelogPartition)) - .collect(Collectors.toList()); + .filter(metadata -> changelogPartition.equals(metadata.changelogPartition)) + .collect(Collectors.toList()); if (found.size() > 1) { throw new IllegalStateException("Multiple state stores are found for changelog partition " + changelogPartition + - ", this should never happen: " + found); + ", this should never happen: " + found); } return found.isEmpty() ? null : found.get(0); @@ -765,14 +765,14 @@ public TopicPartition registeredChangelogPartitionFor(final String storeName) { final StateStoreMetadata storeMetadata = stores.get(storeName); if (storeMetadata == null) { throw new IllegalStateException("State store " + storeName - + " for which the registered changelog partition should be" - + " retrieved has not been registered" + + " for which the registered changelog partition should be" + + " retrieved has not been registered" ); } if (storeMetadata.changelogPartition == null) { throw new IllegalStateException("Registered state store " + storeName - + " does not have a registered changelog partition." - + " This may happen if logging is disabled for the state store." + + " does not have a registered changelog partition." + + " This may happen if logging is disabled for the state store." ); } return storeMetadata.changelogPartition; @@ -781,4 +781,4 @@ public TopicPartition registeredChangelogPartitionFor(final String storeName) { public String changelogFor(final String storeName) { return storeToChangelogTopic.get(storeName); } -} \ No newline at end of file +} From 4b16700276f4a5316b31ec17723a34a798569c4c Mon Sep 17 00:00:00 2001 From: Gabriella Fu <2234862823@qq.com> Date: Mon, 30 Mar 2026 17:55:36 -0400 Subject: [PATCH 05/23] streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java revert --- .../state/internals/InMemorySessionStore.java | 12 +- .../state/internals/InMemoryWindowStore.java | 160 +++++++++--------- 2 files changed, 86 insertions(+), 86 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java index b524f49f1281d..78b161a90f57d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java @@ -114,9 +114,9 @@ public void init(final StateStoreContext stateStoreContext, this.context = (InternalProcessorContext) stateStoreContext; final StreamsMetricsImpl metrics = this.context.metrics(); expiredRecordSensor = TaskMetrics.droppedRecordsSensor( - threadId, - taskName, - metrics + threadId, + taskName, + metrics ); } else { this.context = null; @@ -125,9 +125,9 @@ public void init(final StateStoreContext stateStoreContext, if (root != null) { final boolean consistencyEnabled = StreamsConfig.InternalConfig.getBoolean( - stateStoreContext.appConfigs(), - IQ_CONSISTENCY_OFFSET_VECTOR_ENABLED, - false + stateStoreContext.appConfigs(), + IQ_CONSISTENCY_OFFSET_VECTOR_ENABLED, + false ); stateStoreContext.register( root, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java index 92b7f4362973e..6f5b1095f47ca 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java @@ -114,35 +114,35 @@ public void init(final StateStoreContext stateStoreContext, final String threadId = Thread.currentThread().getName(); final String taskName = stateStoreContext.taskId().toString(); expiredRecordSensor = TaskMetrics.droppedRecordsSensor( - threadId, - taskName, - metrics + threadId, + taskName, + metrics ); if (root != null) { final boolean consistencyEnabled = StreamsConfig.InternalConfig.getBoolean( - stateStoreContext.appConfigs(), - IQ_CONSISTENCY_OFFSET_VECTOR_ENABLED, - false + stateStoreContext.appConfigs(), + IQ_CONSISTENCY_OFFSET_VECTOR_ENABLED, + false ); stateStoreContext.register( - root, - (RecordBatchingStateRestoreCallback) records -> { - synchronized (position) { - for (final ConsumerRecord record : records) { - put( - Bytes.wrap(extractStoreKeyBytes(record.key())), - record.value(), - extractStoreTimestamp(record.key()) - ); - ChangelogRecordDeserializationHelper.applyChecksAndUpdatePosition( - record, - consistencyEnabled, - position - ); - } + root, + (RecordBatchingStateRestoreCallback) records -> { + synchronized (position) { + for (final ConsumerRecord record : records) { + put( + Bytes.wrap(extractStoreKeyBytes(record.key())), + record.value(), + extractStoreTimestamp(record.key()) + ); + ChangelogRecordDeserializationHelper.applyChecksAndUpdatePosition( + record, + consistencyEnabled, + position + ); } } + } ); } open = true; @@ -226,17 +226,17 @@ WindowStoreIterator fetch(final Bytes key, final long timeFrom, final lo if (forward) { return registerNewWindowStoreIterator( - key, - segmentMap.subMap(minTime, true, timeTo, true) - .entrySet().iterator(), - true + key, + segmentMap.subMap(minTime, true, timeTo, true) + .entrySet().iterator(), + true ); } else { return registerNewWindowStoreIterator( - key, - segmentMap.subMap(minTime, true, timeTo, true) - .descendingMap().entrySet().iterator(), - false + key, + segmentMap.subMap(minTime, true, timeTo, true) + .descendingMap().entrySet().iterator(), + false ); } } @@ -266,9 +266,9 @@ KeyValueIterator, byte[]> fetch(final Bytes from, if (from != null && to != null && from.compareTo(to) > 0) { LOG.warn("Returning empty iterator for fetch with invalid key range: from > to. " + - "This may be due to range arguments set in the wrong order, " + - "or serdes that don't preserve ordering when lexicographically comparing the serialized bytes. " + - "Note that the built-in numerical serdes do not follow this for negative numbers"); + "This may be due to range arguments set in the wrong order, " + + "or serdes that don't preserve ordering when lexicographically comparing the serialized bytes. " + + "Note that the built-in numerical serdes do not follow this for negative numbers"); return KeyValueIterators.emptyIterator(); } @@ -281,19 +281,19 @@ KeyValueIterator, byte[]> fetch(final Bytes from, if (forward) { return registerNewWindowedKeyValueIterator( - from, - to, - segmentMap.subMap(minTime, true, timeTo, true) - .entrySet().iterator(), - true + from, + to, + segmentMap.subMap(minTime, true, timeTo, true) + .entrySet().iterator(), + true ); } else { return registerNewWindowedKeyValueIterator( - from, - to, - segmentMap.subMap(minTime, true, timeTo, true) - .descendingMap().entrySet().iterator(), - false + from, + to, + segmentMap.subMap(minTime, true, timeTo, true) + .descendingMap().entrySet().iterator(), + false ); } } @@ -320,19 +320,19 @@ KeyValueIterator, byte[]> fetchAll(final long timeFrom, final lo if (forward) { return registerNewWindowedKeyValueIterator( - null, - null, - segmentMap.subMap(minTime, true, timeTo, true) - .entrySet().iterator(), - true + null, + null, + segmentMap.subMap(minTime, true, timeTo, true) + .entrySet().iterator(), + true ); } else { return registerNewWindowedKeyValueIterator( - null, - null, - segmentMap.subMap(minTime, true, timeTo, true) - .descendingMap().entrySet().iterator(), - false + null, + null, + segmentMap.subMap(minTime, true, timeTo, true) + .descendingMap().entrySet().iterator(), + false ); } } @@ -344,10 +344,10 @@ public KeyValueIterator, byte[]> all() { final long minTime = observedStreamTime - retentionPeriod; return registerNewWindowedKeyValueIterator( - null, - null, - segmentMap.tailMap(minTime, false).entrySet().iterator(), - true + null, + null, + segmentMap.tailMap(minTime, false).entrySet().iterator(), + true ); } @@ -358,10 +358,10 @@ public KeyValueIterator, byte[]> backwardAll() { final long minTime = observedStreamTime - retentionPeriod; return registerNewWindowedKeyValueIterator( - null, - null, - segmentMap.tailMap(minTime, false).descendingMap().entrySet().iterator(), - false + null, + null, + segmentMap.tailMap(minTime, false).descendingMap().entrySet().iterator(), + false ); } @@ -381,12 +381,12 @@ public QueryResult query(final Query query, final QueryConfig config) { return StoreQueryUtils.handleBasicQueries( - query, - positionBound, - config, - this, - position, - internalProcessorContext + query, + positionBound, + config, + this, + position, + internalProcessorContext ); } @@ -410,8 +410,8 @@ public void close() { long numEntries() { return segmentMap.values().stream() - .mapToLong(Map::size) - .sum(); + .mapToLong(Map::size) + .sum(); } private void removeExpiredSegments() { @@ -449,7 +449,7 @@ private WrappedInMemoryWindowStoreIterator registerNewWindowStoreIterator(final final Bytes keyTo = retainDuplicates ? wrapForDups(key, Integer.MAX_VALUE) : key; final WrappedInMemoryWindowStoreIterator iterator = - new WrappedInMemoryWindowStoreIterator(keyFrom, keyTo, segmentIterator, openIterators::remove, retainDuplicates, forward); + new WrappedInMemoryWindowStoreIterator(keyFrom, keyTo, segmentIterator, openIterators::remove, retainDuplicates, forward); openIterators.add(iterator); return iterator; @@ -463,14 +463,14 @@ private WrappedWindowedKeyValueIterator registerNewWindowedKeyValueIterator(fina final Bytes to = (retainDuplicates && keyTo != null) ? wrapForDups(keyTo, Integer.MAX_VALUE) : keyTo; final WrappedWindowedKeyValueIterator iterator = - new WrappedWindowedKeyValueIterator( - from, - to, - segmentIterator, - openIterators::remove, - retainDuplicates, - windowSize, - forward); + new WrappedWindowedKeyValueIterator( + from, + to, + segmentIterator, + openIterators::remove, + retainDuplicates, + windowSize, + forward); openIterators.add(iterator); return iterator; } @@ -643,8 +643,8 @@ public static WrappedInMemoryWindowStoreIterator emptyIterator() { } private static class WrappedWindowedKeyValueIterator - extends InMemoryWindowStoreIteratorWrapper - implements KeyValueIterator, byte[]> { + extends InMemoryWindowStoreIteratorWrapper + implements KeyValueIterator, byte[]> { private final long windowSize; @@ -689,4 +689,4 @@ private Windowed getWindowedKey() { return new Windowed<>(key, timeWindow); } } -} \ No newline at end of file +} From feff5e6b61d6a3d4db3082a5df481f36219dd413 Mon Sep 17 00:00:00 2001 From: Gabriella Fu <2234862823@qq.com> Date: Mon, 30 Mar 2026 17:59:41 -0400 Subject: [PATCH 06/23] streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java --- .../internals/StoreChangelogReaderTest.java | 120 +++++++++--------- 1 file changed, 60 insertions(+), 60 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java index 6c92e36e1ecad..e24b5c5c3e801 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java @@ -148,7 +148,7 @@ public void onRestoreEnd(final TopicPartition tp, final String store, final long private final MockConsumer consumer = new MockConsumer<>(AutoOffsetResetStrategy.EARLIEST.name()); private final MockAdminClient adminClient = new MockAdminClient(); private final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); private void setupStateManagerMock(final Task.TaskType type) { when(stateManager.storeMetadata(tp)).thenReturn(storeMetadata); @@ -191,7 +191,7 @@ public void shouldNotRegisterSameStoreMultipleTimes(final Task.TaskType type) { @Test public void shouldNotRegisterStoreWithoutMetadata() { assertThrows(IllegalStateException.class, - () -> changelogReader.register(new TopicPartition("ChangelogWithoutStoreMetadata", 0), stateManager)); + () -> changelogReader.register(new TopicPartition("ChangelogWithoutStoreMetadata", 0), stateManager)); } @ParameterizedTest @@ -202,7 +202,7 @@ public void shouldSupportUnregisterChangelogBeforeInitialization(final Task.Task adminClient.updateEndOffsets(Collections.singletonMap(tp, 100L)); final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); changelogReader.register(tp, stateManager); @@ -243,7 +243,7 @@ public void shouldSupportUnregisterChangelogBeforeCompletion(final Task.TaskType adminClient.updateEndOffsets(Collections.singletonMap(tp, 100L)); final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); changelogReader.register(tp, stateManager); @@ -298,7 +298,7 @@ public void shouldSupportUnregisterChangelogAfterCompletion(final Task.TaskType adminClient.updateEndOffsets(Collections.singletonMap(tp, 10L)); final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); changelogReader.register(tp, stateManager); @@ -356,7 +356,7 @@ public void shouldPassSuspendReasonToStandbyListener() { adminClient.updateEndOffsets(Collections.singletonMap(tp, 100L)); final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); changelogReader.register(tp, stateManager); changelogReader.transitToUpdateStandby(); @@ -435,7 +435,7 @@ public long position(final TopicPartition partition) { consumer.updateBeginningOffsets(Collections.singletonMap(tp, 5L)); final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); changelogReader.register(tp, stateManager); changelogReader.restore(mockTasks); @@ -490,7 +490,7 @@ public void shouldRestoreFromPositionAndCheckForCompletion(final Task.TaskType t adminClient.updateEndOffsets(Collections.singletonMap(tp, 10L)); final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); changelogReader.register(tp, stateManager); @@ -567,7 +567,7 @@ public void shouldRestoreFromBeginningAndCheckCompletion(final Task.TaskType typ adminClient.updateEndOffsets(Collections.singletonMap(tp, 11L)); final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); changelogReader.register(tp, stateManager); @@ -643,7 +643,7 @@ public void shouldCheckCompletionIfPositionLargerThanEndOffset() { adminClient.updateEndOffsets(Collections.singletonMap(tp, 0L)); final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); changelogReader.register(tp, activeStateManager); changelogReader.restore(mockTasks); @@ -687,7 +687,7 @@ public long position(final TopicPartition partition) { adminClient.updateEndOffsets(Collections.singletonMap(tp, 10L)); final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); changelogReader.register(tp, activeStateManager); changelogReader.restore(Collections.singletonMap(taskId, mockTask)); @@ -729,13 +729,13 @@ public long position(final TopicPartition partition) { adminClient.updateEndOffsets(Collections.singletonMap(tp, 10L)); final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); changelogReader.register(tp, activeStateManager); final StreamsException thrown = assertThrows( - StreamsException.class, - () -> changelogReader.restore(Collections.singletonMap(taskId, mock(Task.class))) + StreamsException.class, + () -> changelogReader.restore(Collections.singletonMap(taskId, mock(Task.class))) ); assertEquals(kaboom, thrown.getCause()); } @@ -777,7 +777,7 @@ public Map committed(final Set topic adminClient.updateEndOffsets(Collections.singletonMap(tp, 0L)); final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); changelogReader.register(tp, activeStateManager); final StreamsException thrown = assertThrows( - StreamsException.class, - () -> changelogReader.restore(Collections.singletonMap(taskId, mock(Task.class))) + StreamsException.class, + () -> changelogReader.restore(Collections.singletonMap(taskId, mock(Task.class))) ); assertEquals(kaboom, thrown.getCause()); } @@ -862,16 +862,16 @@ public synchronized ListConsumerGroupOffsetsResult listConsumerGroupOffsets(fina adminClient.updateConsumerGroupOffsets(Collections.singletonMap(tp, 10L)); final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); changelogReader.register(tp, stateManager); changelogReader.restore(Collections.singletonMap(taskId, mockTask)); assertEquals( - type == ACTIVE ? - StoreChangelogReader.ChangelogState.REGISTERED : - StoreChangelogReader.ChangelogState.RESTORING, - changelogReader.changelogMetadata(tp).state() + type == ACTIVE ? + StoreChangelogReader.ChangelogState.REGISTERED : + StoreChangelogReader.ChangelogState.RESTORING, + changelogReader.changelogMetadata(tp).state() ); if (type == ACTIVE) { assertNull(changelogReader.changelogMetadata(tp).endOffset()); @@ -914,13 +914,13 @@ public synchronized ListConsumerGroupOffsetsResult listConsumerGroupOffsets(fina adminClient.updateEndOffsets(Collections.singletonMap(tp, 10L)); final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); changelogReader.register(tp, stateManager); final StreamsException thrown = assertThrows( - StreamsException.class, - () -> changelogReader.restore(Collections.singletonMap(taskId, mock(Task.class))) + StreamsException.class, + () -> changelogReader.restore(Collections.singletonMap(taskId, mock(Task.class))) ); assertEquals(kaboom, thrown.getCause()); } @@ -934,7 +934,7 @@ public void unsubscribe() { } }; final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); final StreamsException thrown = assertThrows(StreamsException.class, changelogReader::clear); assertEquals(kaboom, thrown.getCause()); @@ -1180,9 +1180,9 @@ public void shouldRestoreMultipleChangelogs() { when(activeStateManager.storeMetadata(tp1)).thenReturn(storeMetadataOne); when(activeStateManager.storeMetadata(tp2)).thenReturn(storeMetadataTwo); when(activeStateManager.changelogOffsets()).thenReturn(mkMap( - mkEntry(tp, 5L), - mkEntry(tp1, 5L), - mkEntry(tp2, 5L) + mkEntry(tp, 5L), + mkEntry(tp1, 5L), + mkEntry(tp2, 5L) )); setupConsumer(10, tp); @@ -1278,8 +1278,8 @@ public void shouldTransitState() { // if a new active is registered, we should immediately transit to standby updating assertThrows( - IllegalStateException.class, - () -> changelogReader.restore(Collections.singletonMap(taskId, mock(Task.class))) + IllegalStateException.class, + () -> changelogReader.restore(Collections.singletonMap(taskId, mock(Task.class))) ); assertEquals(StoreChangelogReader.ChangelogState.RESTORING, changelogReader.changelogMetadata(tp).state()); @@ -1321,13 +1321,13 @@ public void shouldThrowIfRestoreCallbackThrows() { adminClient.updateEndOffsets(Collections.singletonMap(tp, 10L)); final StoreChangelogReader changelogReader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, exceptionCallback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, exceptionCallback, standbyListener); changelogReader.register(tp, activeStateManager); StreamsException thrown = assertThrows( - StreamsException.class, - () -> changelogReader.restore(Collections.singletonMap(taskId, mock(Task.class))) + StreamsException.class, + () -> changelogReader.restore(Collections.singletonMap(taskId, mock(Task.class))) ); assertEquals(kaboom, thrown.getCause()); @@ -1335,16 +1335,16 @@ public void shouldThrowIfRestoreCallbackThrows() { consumer.addRecord(new ConsumerRecord<>(topicName, 0, 7L, "key".getBytes(), "value".getBytes())); thrown = assertThrows( - StreamsException.class, - () -> changelogReader.restore(Collections.singletonMap(taskId, mock(Task.class))) + StreamsException.class, + () -> changelogReader.restore(Collections.singletonMap(taskId, mock(Task.class))) ); assertEquals(kaboom, thrown.getCause()); consumer.seek(tp, 10L); thrown = assertThrows( - StreamsException.class, - () -> changelogReader.restore(Collections.singletonMap(taskId, mock(Task.class))) + StreamsException.class, + () -> changelogReader.restore(Collections.singletonMap(taskId, mock(Task.class))) ); assertEquals(kaboom, thrown.getCause()); } @@ -1356,9 +1356,9 @@ public void shouldNotThrowOnUnknownRevokedPartition() { changelogReader.unregister(Collections.singletonList(new TopicPartition("unknown", 0))); assertThat( - appender.getMessages(), - hasItem("test-reader Changelog partition unknown-0 could not be found," + - " it could be already cleaned up during the handling of task corruption and never restore again") + appender.getMessages(), + hasItem("test-reader Changelog partition unknown-0 could not be found," + + " it could be already cleaned up during the handling of task corruption and never restore again") ); } } @@ -1424,11 +1424,11 @@ private void setupConsumer(final long messages, final TopicPartition topicPartit private void addRecords(final long messages, final TopicPartition topicPartition) { for (int i = 0; i < messages; i++) { consumer.addRecord(new ConsumerRecord<>( - topicPartition.topic(), - topicPartition.partition(), - i, - new byte[0], - new byte[0])); + topicPartition.topic(), + topicPartition.partition(), + i, + new byte[0], + new byte[0])); } } @@ -1466,12 +1466,12 @@ public synchronized Map offsetsForTimes(fina adminClient.updateEndOffsets(Collections.singletonMap(tp, 100L)); final StoreChangelogReader reader = - new StoreChangelogReader(time, config, logContext, adminClient, timestampConsumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, timestampConsumer, callback, standbyListener); reader.register(tp, windowStateManager); reader.restore(Collections.singletonMap(taskId, mock(Task.class))); - assertEquals(offsetForTimestamp, timestampConsumer.position(tp),"The consumer should be seeked to the offset returned by offsetsForTimes, not to the beginning"); + assertEquals(offsetForTimestamp, timestampConsumer.position(tp), "The consumer should be seeked to the offset returned by offsetsForTimes, not to the beginning"); } @Test @@ -1506,12 +1506,12 @@ public synchronized Map offsetsForTimes(fina adminClient.updateEndOffsets(Collections.singletonMap(tp, 100L)); final StoreChangelogReader reader = - new StoreChangelogReader(time, config, logContext, adminClient, timestampConsumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, timestampConsumer, callback, standbyListener); reader.register(tp, windowStateManager); reader.restore(Collections.singletonMap(taskId, mock(Task.class))); - assertEquals(0L, timestampConsumer.position(tp),"When broker returns null, should fall back to seeking to the beginning"); + assertEquals(0L, timestampConsumer.position(tp), "When broker returns null, should fall back to seeking to the beginning"); } @Test @@ -1534,27 +1534,27 @@ public void shouldSeekToBeginningForNonWindowedStoreWithoutCheckpoint() { adminClient.updateEndOffsets(Collections.singletonMap(tp, 100L)); final StoreChangelogReader reader = - new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); + new StoreChangelogReader(time, config, logContext, adminClient, consumer, callback, standbyListener); reader.register(tp, kvStateManager); reader.restore(Collections.singletonMap(taskId, mock(Task.class))); - assertEquals(0L, consumer.position(tp),"Non-windowed store should seek to beginning, not by timestamp"); + assertEquals(0L, consumer.position(tp), "Non-windowed store should seek to beginning, not by timestamp"); } private void assignPartition(final long messages, final TopicPartition topicPartition) { consumer.updatePartitions( + topicPartition.topic(), + Collections.singletonList(new PartitionInfo( topicPartition.topic(), - Collections.singletonList(new PartitionInfo( - topicPartition.topic(), - topicPartition.partition(), - null, - null, - null))); + topicPartition.partition(), + null, + null, + null))); consumer.updateBeginningOffsets(Collections.singletonMap(topicPartition, 0L)); consumer.updateEndOffsets(Collections.singletonMap(topicPartition, Math.max(0, messages) + 1)); adminClient.updateEndOffsets(Collections.singletonMap(topicPartition, Math.max(0, messages) + 1)); consumer.assign(Collections.singletonList(topicPartition)); } -} \ No newline at end of file +} From 76d3d946aced5468f5e12e4ab35d2e0ec5bcdfd9 Mon Sep 17 00:00:00 2001 From: Gabriella Fu <2234862823@qq.com> Date: Tue, 31 Mar 2026 00:16:56 -0400 Subject: [PATCH 07/23] adding the interface --- .../internals/AbstractDualSchemaRocksDBSegmentedBytesStore.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractDualSchemaRocksDBSegmentedBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractDualSchemaRocksDBSegmentedBytesStore.java index 9606e89ece57c..9dbb4210aa7c9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractDualSchemaRocksDBSegmentedBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractDualSchemaRocksDBSegmentedBytesStore.java @@ -46,7 +46,7 @@ import static org.apache.kafka.streams.StreamsConfig.InternalConfig.IQ_CONSISTENCY_OFFSET_VECTOR_ENABLED; import static org.apache.kafka.streams.processor.internals.ProcessorContextUtils.asInternalProcessorContext; -public abstract class AbstractDualSchemaRocksDBSegmentedBytesStore implements SegmentedBytesStore { +public abstract class AbstractDualSchemaRocksDBSegmentedBytesStore implements SegmentedBytesStore, WithRetentionPeriod { private static final Logger LOG = LoggerFactory.getLogger(AbstractDualSchemaRocksDBSegmentedBytesStore.class); private final String name; From d7c0b72b056494e8e93438bd78d2d15c304e4a01 Mon Sep 17 00:00:00 2001 From: Gabriella Fu <2234862823@qq.com> Date: Tue, 31 Mar 2026 10:02:35 -0400 Subject: [PATCH 08/23] adding WithRetentionPeriod --- .../state/internals/AbstractRocksDBSegmentedBytesStore.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java index 7a83ea5d19c7f..c11e276a19e3c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java @@ -48,7 +48,7 @@ import static org.apache.kafka.streams.StreamsConfig.InternalConfig.IQ_CONSISTENCY_OFFSET_VECTOR_ENABLED; import static org.apache.kafka.streams.processor.internals.ProcessorContextUtils.asInternalProcessorContext; -public class AbstractRocksDBSegmentedBytesStore implements SegmentedBytesStore { +public class AbstractRocksDBSegmentedBytesStore implements SegmentedBytesStore, WithRetentionPeriod { private static final Logger LOG = LoggerFactory.getLogger(AbstractRocksDBSegmentedBytesStore.class); private final String name; From 72413c0a9550257e3e06c6e97409c873c44f15d6 Mon Sep 17 00:00:00 2001 From: Gabriella Fu <2234862823@qq.com> Date: Tue, 21 Apr 2026 17:30:15 -0400 Subject: [PATCH 09/23] fix the bug cause by the gap between wall clock and streams clock --- .../kafka/clients/admin/MockAdminClient.java | 12 +- smoke_test_output.log | 578 ++++++++++++++++++ .../internals/StoreChangelogReader.java | 64 +- .../internals/StoreChangelogReaderTest.java | 2 + 4 files changed, 648 insertions(+), 8 deletions(-) create mode 100644 smoke_test_output.log diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java index bb460855df01a..ab249b2a90998 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java @@ -89,6 +89,7 @@ public class MockAdminClient extends AdminClient { private final Map beginningOffsets; private final Map endOffsets; private final Map committedOffsets; + private final Map maxTimestamps; private final boolean usingRaftController; private final Map featureLevels; private final Map minSupportedFeatureLevels; @@ -269,6 +270,7 @@ private MockAdminClient( this.beginningOffsets = new HashMap<>(); this.endOffsets = new HashMap<>(); this.committedOffsets = new HashMap<>(); + this.maxTimestamps = new HashMap<>(); this.usingRaftController = usingRaftController; this.featureLevels = new HashMap<>(featureLevels); this.minSupportedFeatureLevels = new HashMap<>(minSupportedFeatureLevels); @@ -1231,7 +1233,11 @@ public synchronized ListOffsetsResult listOffsets(Map newOff endOffsets.putAll(newOffsets); } + public synchronized void updateMaxTimestamps(final Map newTimestamps) { + maxTimestamps.putAll(newTimestamps); + } + public synchronized void updateConsumerGroupOffsets(final Map newOffsets) { committedOffsets.putAll(newOffsets); } diff --git a/smoke_test_output.log b/smoke_test_output.log new file mode 100644 index 0000000000000..e61a445abce5b --- /dev/null +++ b/smoke_test_output.log @@ -0,0 +1,578 @@ +WARNING: Ignoring custom format, because both --format and --quiet are set. + +> Configure project : +Starting build with version 4.4.0-SNAPSHOT (commit id 54f7c7a9) using Gradle 9.4.1, Java 17 and Scala 2.13.18 +Build properties: ignoreFailures=false, maxParallelForks=14, maxScalacThreads=8, maxTestRetries=0 + +> Task :core:processResources NO-SOURCE +> Task :storage:processResources UP-TO-DATE +> Task :server-common:processResources NO-SOURCE +> Task :server:processResources NO-SOURCE +> Task :share-coordinator:processResources UP-TO-DATE +> Task :coordinator-common:processResources NO-SOURCE +> Task :storage:createVersionFile UP-TO-DATE +> Task :group-coordinator:group-coordinator-api:processResources NO-SOURCE +> Task :server-common:createVersionFile UP-TO-DATE +> Task :connect:compileJava NO-SOURCE +> Task :storage:storage-api:processResources NO-SOURCE +> Task :server:createVersionFile UP-TO-DATE +> Task :raft:processResources UP-TO-DATE +> Task :connect:processResources NO-SOURCE +> Task :transaction-coordinator:processResources UP-TO-DATE +> Task :group-coordinator:group-coordinator-api:createVersionFile UP-TO-DATE +> Task :connect:classes UP-TO-DATE +> Task :tools:tools-api:processResources NO-SOURCE +> Task :coordinator-common:processTestResources UP-TO-DATE +> Task :metadata:processResources UP-TO-DATE +> Task :server-common:processTestResources UP-TO-DATE +> Task :storage:storage-api:createVersionFile UP-TO-DATE +> Task :raft:createVersionFile UP-TO-DATE +> Task :group-coordinator:processResources UP-TO-DATE +> Task :server:processTestResources UP-TO-DATE +> Task :share-coordinator:processTestResources UP-TO-DATE +> Task :tools:tools-api:createVersionFile UP-TO-DATE +> Task :test-common:test-common-internal-api:processResources NO-SOURCE +> Task :streams:processResources UP-TO-DATE +> Task :metadata:processTestResources UP-TO-DATE +> Task :test-common:test-common-runtime:processResources UP-TO-DATE +> Task :clients:processResources UP-TO-DATE +> Task :storage:storage-api:processTestResources UP-TO-DATE +> Task :streams:createStreamsVersionFile UP-TO-DATE +> Task :raft:processTestResources UP-TO-DATE +> Task :connect:jar UP-TO-DATE +> Task :clients:createVersionFile UP-TO-DATE +> Task :connect:systemTestLibs UP-TO-DATE +> Task :examples:processResources NO-SOURCE +> Task :core:processTestResources UP-TO-DATE +> Task :group-coordinator:processTestResources UP-TO-DATE +> Task :shell:processResources NO-SOURCE +> Task :clients:processTestResources UP-TO-DATE +> Task :connect:api:processResources UP-TO-DATE +> Task :connect:json:processResources UP-TO-DATE +> Task :test-common:compileJava NO-SOURCE +> Task :streams:processTestResources UP-TO-DATE +> Task :jmh-benchmarks:processResources NO-SOURCE +> Task :connect:runtime:processResources UP-TO-DATE +> Task :tools:processResources NO-SOURCE +> Task :connect:transforms:processResources UP-TO-DATE +> Task :trogdor:processResources NO-SOURCE +> Task :test-common:processResources NO-SOURCE +> Task :test-common:classes UP-TO-DATE +> Task :connect:basic-auth-extension:processResources UP-TO-DATE +> Task :clients:clients-integration-tests:processResources NO-SOURCE +> Task :connect:file:processResources UP-TO-DATE +> Task :connect:mirror:processResources UP-TO-DATE +> Task :connect:mirror-client:processResources NO-SOURCE +> Task :connect:test-plugins:processResources UP-TO-DATE +> Task :streams:examples:processResources NO-SOURCE +> Task :streams:test-utils:processResources NO-SOURCE +> Task :streams:streams-scala:processResources NO-SOURCE +> Task :streams:integration-tests:compileJava NO-SOURCE +> Task :streams:upgrade-system-tests-0110:compileJava NO-SOURCE +> Task :streams:upgrade-system-tests-11:compileJava NO-SOURCE +> Task :streams:upgrade-system-tests-23:compileJava NO-SOURCE +> Task :streams:upgrade-system-tests-0110:processResources NO-SOURCE +> Task :streams:upgrade-system-tests-11:processResources NO-SOURCE +> Task :streams:upgrade-system-tests-11:classes UP-TO-DATE +> Task :streams:upgrade-system-tests-20:compileJava NO-SOURCE +> Task :streams:upgrade-system-tests-0110:classes UP-TO-DATE +> Task :streams:upgrade-system-tests-25:compileJava NO-SOURCE +> Task :streams:upgrade-system-tests-22:compileJava NO-SOURCE +> Task :streams:integration-tests:compileScala NO-SOURCE +> Task :streams:upgrade-system-tests-23:processResources NO-SOURCE +> Task :streams:upgrade-system-tests-10:compileJava NO-SOURCE +> Task :streams:upgrade-system-tests-24:compileJava NO-SOURCE +> Task :streams:upgrade-system-tests-21:compileJava NO-SOURCE +> Task :test-common:jar UP-TO-DATE +> Task :test-common:systemTestLibs UP-TO-DATE +> Task :streams:upgrade-system-tests-23:classes UP-TO-DATE +> Task :streams:upgrade-system-tests-20:processResources NO-SOURCE +> Task :streams:upgrade-system-tests-22:processResources NO-SOURCE +> Task :streams:upgrade-system-tests-20:classes UP-TO-DATE +> Task :streams:upgrade-system-tests-11:jar UP-TO-DATE +> Task :streams:integration-tests:processResources NO-SOURCE +> Task :streams:upgrade-system-tests-0110:jar UP-TO-DATE +> Task :streams:integration-tests:classes UP-TO-DATE +> Task :streams:upgrade-system-tests-21:processResources NO-SOURCE +> Task :streams:upgrade-system-tests-10:processResources NO-SOURCE +> Task :streams:upgrade-system-tests-10:classes UP-TO-DATE +> Task :streams:upgrade-system-tests-23:jar UP-TO-DATE +> Task :streams:upgrade-system-tests-26:compileJava NO-SOURCE +> Task :streams:upgrade-system-tests-25:processResources NO-SOURCE +> Task :streams:upgrade-system-tests-25:classes UP-TO-DATE +> Task :streams:upgrade-system-tests-20:jar UP-TO-DATE +> Task :streams:integration-tests:jar UP-TO-DATE +> Task :streams:upgrade-system-tests-26:processResources NO-SOURCE +> Task :streams:integration-tests:systemTestLibs UP-TO-DATE +> Task :streams:upgrade-system-tests-24:processResources NO-SOURCE +> Task :streams:upgrade-system-tests-21:classes UP-TO-DATE +> Task :streams:upgrade-system-tests-10:jar UP-TO-DATE +> Task :streams:upgrade-system-tests-22:classes UP-TO-DATE +> Task :streams:upgrade-system-tests-24:classes UP-TO-DATE +> Task :streams:upgrade-system-tests-26:classes UP-TO-DATE +> Task :streams:upgrade-system-tests-27:compileJava NO-SOURCE +> Task :streams:upgrade-system-tests-27:processResources NO-SOURCE +> Task :streams:upgrade-system-tests-27:classes UP-TO-DATE +> Task :streams:upgrade-system-tests-25:jar UP-TO-DATE +> Task :streams:upgrade-system-tests-21:jar UP-TO-DATE +> Task :streams:upgrade-system-tests-26:jar UP-TO-DATE +> Task :streams:upgrade-system-tests-28:compileJava NO-SOURCE +> Task :streams:upgrade-system-tests-22:jar UP-TO-DATE +> Task :streams:upgrade-system-tests-24:jar UP-TO-DATE +> Task :streams:upgrade-system-tests-27:jar UP-TO-DATE +> Task :streams:upgrade-system-tests-28:processResources NO-SOURCE +> Task :streams:upgrade-system-tests-28:classes UP-TO-DATE +> Task :streams:upgrade-system-tests-28:jar UP-TO-DATE +> Task :test-common:test-common-util:compileJava UP-TO-DATE +> Task :test-common:test-common-util:processResources UP-TO-DATE +> Task :test-common:test-common-util:classes UP-TO-DATE +> Task :test-common:test-common-util:jar UP-TO-DATE +> Task :streams:upgrade-system-tests-30:compileJava NO-SOURCE +> Task :streams:upgrade-system-tests-30:processResources NO-SOURCE +> Task :streams:upgrade-system-tests-30:classes UP-TO-DATE +> Task :streams:upgrade-system-tests-0110:compileTestJava UP-TO-DATE +> Task :streams:upgrade-system-tests-0110:processTestResources NO-SOURCE +> Task :streams:upgrade-system-tests-30:jar UP-TO-DATE +> Task :streams:upgrade-system-tests-0110:testClasses UP-TO-DATE +> Task :streams:upgrade-system-tests-27:compileTestJava UP-TO-DATE +> Task :streams:upgrade-system-tests-26:compileTestJava UP-TO-DATE +> Task :streams:upgrade-system-tests-27:processTestResources NO-SOURCE +> Task :streams:upgrade-system-tests-10:compileTestJava UP-TO-DATE +> Task :streams:upgrade-system-tests-11:compileTestJava UP-TO-DATE +> Task :streams:upgrade-system-tests-25:compileTestJava UP-TO-DATE +> Task :streams:upgrade-system-tests-0110:testJar UP-TO-DATE +> Task :streams:upgrade-system-tests-22:compileTestJava UP-TO-DATE +> Task :streams:upgrade-system-tests-24:compileTestJava UP-TO-DATE +> Task :streams:upgrade-system-tests-20:compileTestJava UP-TO-DATE +> Task :streams:upgrade-system-tests-21:compileTestJava UP-TO-DATE +> Task :streams:upgrade-system-tests-28:compileTestJava UP-TO-DATE +> Task :streams:upgrade-system-tests-23:compileTestJava UP-TO-DATE +> Task :streams:upgrade-system-tests-0110:systemTestLibs UP-TO-DATE +> Task :generator:compileJava UP-TO-DATE +> Task :streams:upgrade-system-tests-10:processTestResources NO-SOURCE +> Task :streams:upgrade-system-tests-27:testClasses UP-TO-DATE +> Task :streams:upgrade-system-tests-11:processTestResources NO-SOURCE +> Task :streams:upgrade-system-tests-20:processTestResources NO-SOURCE +> Task :streams:upgrade-system-tests-26:processTestResources NO-SOURCE +> Task :streams:upgrade-system-tests-26:testClasses UP-TO-DATE +> Task :streams:upgrade-system-tests-24:processTestResources NO-SOURCE +> Task :streams:upgrade-system-tests-21:processTestResources NO-SOURCE +> Task :streams:upgrade-system-tests-22:processTestResources NO-SOURCE +> Task :streams:upgrade-system-tests-20:testClasses UP-TO-DATE +> Task :streams:upgrade-system-tests-25:processTestResources NO-SOURCE +> Task :streams:upgrade-system-tests-10:testClasses UP-TO-DATE +> Task :streams:upgrade-system-tests-28:processTestResources NO-SOURCE +> Task :generator:processResources NO-SOURCE +> Task :streams:upgrade-system-tests-21:testClasses UP-TO-DATE +> Task :streams:upgrade-system-tests-22:testClasses UP-TO-DATE +> Task :streams:upgrade-system-tests-11:testClasses UP-TO-DATE +> Task :streams:upgrade-system-tests-23:processTestResources NO-SOURCE +> Task :streams:upgrade-system-tests-23:testClasses UP-TO-DATE +> Task :streams:upgrade-system-tests-24:testClasses UP-TO-DATE +> Task :generator:classes UP-TO-DATE +> Task :streams:upgrade-system-tests-25:testClasses UP-TO-DATE +> Task :streams:upgrade-system-tests-28:testClasses UP-TO-DATE +> Task :streams:upgrade-system-tests-27:testJar UP-TO-DATE +> Task :streams:upgrade-system-tests-20:testJar UP-TO-DATE +> Task :streams:upgrade-system-tests-27:systemTestLibs UP-TO-DATE +> Task :generator:jar UP-TO-DATE +> Task :streams:upgrade-system-tests-20:systemTestLibs UP-TO-DATE +> Task :streams:upgrade-system-tests-26:testJar UP-TO-DATE +> Task :streams:upgrade-system-tests-30:compileTestJava UP-TO-DATE +> Task :streams:upgrade-system-tests-31:compileJava NO-SOURCE +> Task :streams:upgrade-system-tests-23:testJar UP-TO-DATE +> Task :streams:upgrade-system-tests-11:testJar UP-TO-DATE +> Task :streams:upgrade-system-tests-22:testJar UP-TO-DATE +> Task :streams:upgrade-system-tests-10:testJar UP-TO-DATE +> Task :streams:upgrade-system-tests-21:testJar UP-TO-DATE +> Task :streams:upgrade-system-tests-28:testJar UP-TO-DATE +> Task :streams:upgrade-system-tests-24:testJar UP-TO-DATE +> Task :streams:upgrade-system-tests-25:testJar UP-TO-DATE +> Task :streams:upgrade-system-tests-10:systemTestLibs UP-TO-DATE +> Task :streams:upgrade-system-tests-11:systemTestLibs UP-TO-DATE +> Task :streams:upgrade-system-tests-21:systemTestLibs UP-TO-DATE +> Task :streams:upgrade-system-tests-22:systemTestLibs UP-TO-DATE +> Task :streams:upgrade-system-tests-23:systemTestLibs UP-TO-DATE +> Task :streams:upgrade-system-tests-24:systemTestLibs UP-TO-DATE +> Task :streams:upgrade-system-tests-26:systemTestLibs UP-TO-DATE +> Task :streams:upgrade-system-tests-30:processTestResources NO-SOURCE +> Task :streams:upgrade-system-tests-31:processResources NO-SOURCE +> Task :generator:systemTestLibs UP-TO-DATE +> Task :streams:upgrade-system-tests-25:systemTestLibs UP-TO-DATE +> Task :streams:upgrade-system-tests-28:systemTestLibs UP-TO-DATE +> Task :streams:upgrade-system-tests-30:testClasses UP-TO-DATE +> Task :streams:upgrade-system-tests-31:classes UP-TO-DATE +> Task :streams:upgrade-system-tests-32:compileJava NO-SOURCE +> Task :streams:upgrade-system-tests-33:compileJava NO-SOURCE +> Task :streams:upgrade-system-tests-34:compileJava NO-SOURCE +> Task :streams:upgrade-system-tests-32:processResources NO-SOURCE +> Task :streams:upgrade-system-tests-32:classes UP-TO-DATE +> Task :streams:upgrade-system-tests-34:processResources NO-SOURCE +> Task :streams:upgrade-system-tests-34:classes UP-TO-DATE +> Task :share-coordinator:processMessages UP-TO-DATE +> Task :streams:upgrade-system-tests-33:processResources NO-SOURCE +> Task :streams:upgrade-system-tests-35:compileJava NO-SOURCE +> Task :streams:upgrade-system-tests-30:testJar UP-TO-DATE +> Task :streams:upgrade-system-tests-31:jar UP-TO-DATE +> Task :streams:processMessages UP-TO-DATE +> Task :streams:upgrade-system-tests-34:jar UP-TO-DATE +> Task :raft:processMessages UP-TO-DATE +> Task :clients:processMessages UP-TO-DATE +> Task :metadata:processMessages UP-TO-DATE +> Task :group-coordinator:processMessages UP-TO-DATE +> Task :streams:upgrade-system-tests-33:classes UP-TO-DATE +> Task :transaction-coordinator:processMessages UP-TO-DATE +> Task :streams:upgrade-system-tests-35:processResources NO-SOURCE +> Task :streams:upgrade-system-tests-35:classes UP-TO-DATE +> Task :streams:upgrade-system-tests-36:compileJava NO-SOURCE +> Task :streams:upgrade-system-tests-32:jar UP-TO-DATE +> Task :storage:processMessages UP-TO-DATE +> Task :streams:upgrade-system-tests-36:processResources NO-SOURCE +> Task :test-common:test-common-util:systemTestLibs UP-TO-DATE +> Task :streams:upgrade-system-tests-30:systemTestLibs UP-TO-DATE +> Task :streams:upgrade-system-tests-36:classes UP-TO-DATE +> Task :streams:upgrade-system-tests-37:compileJava NO-SOURCE +> Task :streams:upgrade-system-tests-39:compileJava NO-SOURCE +> Task :streams:upgrade-system-tests-35:jar UP-TO-DATE +> Task :streams:upgrade-system-tests-37:processResources NO-SOURCE +> Task :streams:upgrade-system-tests-41:compileJava NO-SOURCE +> Task :streams:upgrade-system-tests-37:classes UP-TO-DATE +> Task :streams:upgrade-system-tests-39:processResources NO-SOURCE +> Task :streams:upgrade-system-tests-41:processResources NO-SOURCE +> Task :streams:upgrade-system-tests-39:classes UP-TO-DATE +> Task :streams:upgrade-system-tests-33:jar UP-TO-DATE +> Task :streams:upgrade-system-tests-41:classes UP-TO-DATE +> Task :streams:upgrade-system-tests-38:compileJava NO-SOURCE +> Task :streams:upgrade-system-tests-36:jar UP-TO-DATE +> Task :streams:upgrade-system-tests-40:compileJava NO-SOURCE +> Task :streams:upgrade-system-tests-39:jar UP-TO-DATE +> Task :streams:upgrade-system-tests-38:processResources NO-SOURCE +> Task :streams:upgrade-system-tests-37:jar UP-TO-DATE +> Task :streams:upgrade-system-tests-40:processResources NO-SOURCE +> Task :streams:upgrade-system-tests-40:classes UP-TO-DATE +> Task :streams:upgrade-system-tests-38:classes UP-TO-DATE +> Task :streams:upgrade-system-tests-41:jar UP-TO-DATE +> Task :streams:upgrade-system-tests-38:jar UP-TO-DATE +> Task :streams:upgrade-system-tests-40:jar UP-TO-DATE +> Task :streams:upgrade-system-tests-32:compileTestJava UP-TO-DATE +> Task :streams:upgrade-system-tests-34:compileTestJava UP-TO-DATE +> Task :streams:upgrade-system-tests-37:compileTestJava UP-TO-DATE +> Task :streams:upgrade-system-tests-36:compileTestJava UP-TO-DATE +> Task :streams:upgrade-system-tests-32:processTestResources NO-SOURCE +> Task :streams:upgrade-system-tests-35:compileTestJava UP-TO-DATE +> Task :streams:upgrade-system-tests-36:processTestResources NO-SOURCE +> Task :streams:upgrade-system-tests-33:compileTestJava UP-TO-DATE +> Task :streams:upgrade-system-tests-34:processTestResources NO-SOURCE +> Task :streams:upgrade-system-tests-32:testClasses UP-TO-DATE +> Task :streams:upgrade-system-tests-34:testClasses UP-TO-DATE +> Task :streams:upgrade-system-tests-31:compileTestJava UP-TO-DATE +> Task :streams:upgrade-system-tests-37:processTestResources NO-SOURCE +> Task :streams:upgrade-system-tests-36:testClasses UP-TO-DATE +> Task :streams:upgrade-system-tests-38:compileTestJava UP-TO-DATE +> Task :streams:upgrade-system-tests-41:compileTestJava UP-TO-DATE +> Task :streams:upgrade-system-tests-39:compileTestJava UP-TO-DATE +> Task :streams:upgrade-system-tests-33:processTestResources NO-SOURCE +> Task :streams:upgrade-system-tests-33:testClasses UP-TO-DATE +> Task :streams:upgrade-system-tests-35:processTestResources NO-SOURCE +> Task :streams:upgrade-system-tests-40:compileTestJava UP-TO-DATE +> Task :streams:upgrade-system-tests-35:testClasses UP-TO-DATE +> Task :streams:upgrade-system-tests-39:processTestResources NO-SOURCE +> Task :streams:upgrade-system-tests-37:testClasses UP-TO-DATE +> Task :streams:upgrade-system-tests-38:processTestResources NO-SOURCE +> Task :streams:upgrade-system-tests-31:processTestResources NO-SOURCE +> Task :streams:upgrade-system-tests-41:processTestResources NO-SOURCE +> Task :streams:upgrade-system-tests-31:testClasses UP-TO-DATE +> Task :streams:upgrade-system-tests-40:processTestResources NO-SOURCE +> Task :streams:upgrade-system-tests-36:testJar UP-TO-DATE +> Task :streams:upgrade-system-tests-41:testClasses UP-TO-DATE +> Task :streams:upgrade-system-tests-33:testJar UP-TO-DATE +> Task :streams:upgrade-system-tests-39:testClasses UP-TO-DATE +> Task :streams:upgrade-system-tests-33:systemTestLibs UP-TO-DATE +> Task :streams:upgrade-system-tests-38:testClasses UP-TO-DATE +> Task :streams:upgrade-system-tests-32:testJar UP-TO-DATE +> Task :streams:upgrade-system-tests-34:testJar UP-TO-DATE +> Task :clients:compileJava UP-TO-DATE +> Task :streams:upgrade-system-tests-31:testJar UP-TO-DATE +> Task :streams:upgrade-system-tests-35:testJar UP-TO-DATE +> Task :streams:upgrade-system-tests-32:systemTestLibs UP-TO-DATE +> Task :streams:upgrade-system-tests-37:testJar UP-TO-DATE +> Task :streams:upgrade-system-tests-34:systemTestLibs UP-TO-DATE +> Task :streams:upgrade-system-tests-36:systemTestLibs UP-TO-DATE +> Task :streams:upgrade-system-tests-31:systemTestLibs UP-TO-DATE +> Task :streams:upgrade-system-tests-37:systemTestLibs UP-TO-DATE +> Task :streams:upgrade-system-tests-38:testJar UP-TO-DATE +> Task :streams:upgrade-system-tests-39:testJar UP-TO-DATE +> Task :streams:upgrade-system-tests-38:systemTestLibs UP-TO-DATE +> Task :streams:upgrade-system-tests-39:systemTestLibs UP-TO-DATE +> Task :clients:classes UP-TO-DATE +> Task :streams:upgrade-system-tests-40:testClasses UP-TO-DATE +> Task :streams:upgrade-system-tests-41:testJar UP-TO-DATE +> Task :streams:upgrade-system-tests-35:systemTestLibs UP-TO-DATE +> Task :streams:upgrade-system-tests-41:systemTestLibs UP-TO-DATE +> Task :connect:mirror-client:compileJava UP-TO-DATE +> Task :connect:mirror-client:classes UP-TO-DATE +> Task :tools:tools-api:compileJava UP-TO-DATE +> Task :examples:compileJava UP-TO-DATE +> Task :tools:tools-api:classes UP-TO-DATE +> Task :group-coordinator:group-coordinator-api:compileJava UP-TO-DATE +> Task :examples:classes UP-TO-DATE +> Task :group-coordinator:group-coordinator-api:classes UP-TO-DATE +> Task :streams:upgrade-system-tests-40:testJar UP-TO-DATE +> Task :streams:upgrade-system-tests-40:systemTestLibs UP-TO-DATE +> Task :examples:jar UP-TO-DATE +> Task :examples:systemTestLibs UP-TO-DATE +> Task :connect:api:compileJava UP-TO-DATE +> Task :group-coordinator:group-coordinator-api:jar UP-TO-DATE +> Task :tools:tools-api:jar UP-TO-DATE +> Task :connect:api:classes UP-TO-DATE +> Task :server-common:compileJava UP-TO-DATE +> Task :tools:tools-api:systemTestLibs UP-TO-DATE +> Task :connect:file:compileJava UP-TO-DATE +> Task :connect:json:compileJava UP-TO-DATE +> Task :connect:basic-auth-extension:compileJava UP-TO-DATE +> Task :group-coordinator:group-coordinator-api:systemTestLibs UP-TO-DATE +> Task :connect:transforms:compileJava UP-TO-DATE +> Task :connect:basic-auth-extension:classes UP-TO-DATE +> Task :connect:json:classes UP-TO-DATE +> Task :connect:transforms:classes UP-TO-DATE +> Task :server-common:classes UP-TO-DATE +> Task :storage:storage-api:compileJava UP-TO-DATE +> Task :connect:file:classes UP-TO-DATE +> Task :connect:test-plugins:compileJava UP-TO-DATE +> Task :test-common:test-common-internal-api:compileJava UP-TO-DATE +> Task :storage:storage-api:classes UP-TO-DATE +> Task :test-common:test-common-internal-api:classes UP-TO-DATE +> Task :connect:test-plugins:classes UP-TO-DATE +> Task :server-common:jar UP-TO-DATE +> Task :server-common:systemTestLibs UP-TO-DATE +> Task :clients:shadowJar UP-TO-DATE +> Task :connect:test-plugins:jar UP-TO-DATE +> Task :clients:jar SKIPPED +> Task :test-common:test-common-internal-api:jar UP-TO-DATE +> Task :clients:systemTestLibs UP-TO-DATE +> Task :connect:test-plugins:systemTestLibs UP-TO-DATE +> Task :test-common:test-common-internal-api:systemTestLibs UP-TO-DATE +> Task :storage:storage-api:jar UP-TO-DATE +> Task :storage:storage-api:systemTestLibs UP-TO-DATE +> Task :clients:processTestMessages UP-TO-DATE +> Task :connect:mirror-client:copyDependantLibs UP-TO-DATE +> Task :connect:api:copyDependantLibs UP-TO-DATE +> Task :storage:compileJava UP-TO-DATE +> Task :storage:classes UP-TO-DATE +> Task :connect:mirror-client:jar UP-TO-DATE +> Task :connect:api:jar UP-TO-DATE +> Task :connect:mirror-client:systemTestLibs UP-TO-DATE +> Task :connect:api:systemTestLibs UP-TO-DATE +> Task :connect:transforms:copyDependantLibs UP-TO-DATE +> Task :connect:file:copyDependantLibs UP-TO-DATE +> Task :storage:jar UP-TO-DATE +> Task :raft:compileJava UP-TO-DATE +> Task :connect:transforms:jar UP-TO-DATE +> Task :connect:basic-auth-extension:copyDependantLibs UP-TO-DATE +> Task :storage:systemTestLibs UP-TO-DATE +> Task :connect:json:copyDependantLibs UP-TO-DATE +> Task :raft:classes UP-TO-DATE +> Task :connect:file:jar UP-TO-DATE +> Task :connect:file:systemTestLibs UP-TO-DATE +> Task :connect:transforms:systemTestLibs UP-TO-DATE +> Task :connect:basic-auth-extension:jar UP-TO-DATE +> Task :connect:json:jar UP-TO-DATE +> Task :connect:basic-auth-extension:systemTestLibs UP-TO-DATE +> Task :connect:json:systemTestLibs UP-TO-DATE +> Task :raft:jar UP-TO-DATE +> Task :raft:systemTestLibs UP-TO-DATE +> Task :metadata:compileJava UP-TO-DATE +> Task :metadata:classes UP-TO-DATE +> Task :metadata:jar UP-TO-DATE +> Task :metadata:systemTestLibs UP-TO-DATE +> Task :streams:compileJava UP-TO-DATE +> Task :streams:classes UP-TO-DATE +> Task :streams:streams-scala:compileJava NO-SOURCE +> Task :streams:test-utils:compileJava UP-TO-DATE +> Task :streams:test-utils:classes UP-TO-DATE +> Task :streams:examples:compileJava UP-TO-DATE +> Task :streams:copyDependantLibs UP-TO-DATE +> Task :streams:examples:classes UP-TO-DATE +> Task :streams:jar UP-TO-DATE +> Task :coordinator-common:compileJava UP-TO-DATE +> Task :coordinator-common:classes UP-TO-DATE +> Task :coordinator-common:jar UP-TO-DATE +> Task :coordinator-common:systemTestLibs UP-TO-DATE +> Task :streams:examples:copyDependantLibs UP-TO-DATE +> Task :streams:test-utils:copyDependantLibs UP-TO-DATE +> Task :share-coordinator:compileJava UP-TO-DATE +> Task :transaction-coordinator:compileJava UP-TO-DATE +> Task :share-coordinator:classes UP-TO-DATE +> Task :streams:test-utils:jar UP-TO-DATE +> Task :transaction-coordinator:classes UP-TO-DATE +> Task :streams:examples:jar UP-TO-DATE +> Task :streams:test-utils:systemTestLibs UP-TO-DATE +> Task :streams:examples:systemTestLibs UP-TO-DATE +> Task :share-coordinator:jar UP-TO-DATE +> Task :transaction-coordinator:jar UP-TO-DATE +> Task :transaction-coordinator:systemTestLibs UP-TO-DATE +> Task :share-coordinator:systemTestLibs UP-TO-DATE +> Task :group-coordinator:compileJava UP-TO-DATE +> Task :group-coordinator:classes UP-TO-DATE +> Task :clients:clients-integration-tests:compileJava NO-SOURCE +> Task :clients:clients-integration-tests:classes UP-TO-DATE +> Task :group-coordinator:jar UP-TO-DATE +> Task :group-coordinator:systemTestLibs UP-TO-DATE +> Task :clients:clients-integration-tests:jar UP-TO-DATE +> Task :clients:clients-integration-tests:systemTestLibs UP-TO-DATE +> Task :server:compileJava UP-TO-DATE +> Task :server:classes UP-TO-DATE +> Task :core:compileJava NO-SOURCE +> Task :server:jar UP-TO-DATE +> Task :server:systemTestLibs UP-TO-DATE +> Task :trogdor:compileJava UP-TO-DATE +> Task :trogdor:classes UP-TO-DATE +> Task :connect:runtime:compileJava UP-TO-DATE +> Task :connect:runtime:classes UP-TO-DATE +> Task :tools:compileJava UP-TO-DATE +> Task :tools:classes UP-TO-DATE +> Task :connect:mirror:compileJava UP-TO-DATE +> Task :connect:mirror:classes UP-TO-DATE +> Task :connect:runtime:copyDependantLibs UP-TO-DATE +> Task :trogdor:copyDependantLibs UP-TO-DATE +> Task :connect:runtime:jar UP-TO-DATE +> Task :connect:runtime:systemTestLibs UP-TO-DATE +> Task :trogdor:jar UP-TO-DATE +> Task :trogdor:systemTestLibs UP-TO-DATE +> Task :connect:mirror:copyDependantLibs UP-TO-DATE +> Task :connect:mirror:jar UP-TO-DATE +> Task :connect:mirror:systemTestLibs UP-TO-DATE +> Task :tools:copyDependantLibs UP-TO-DATE +> Task :tools:jar UP-TO-DATE +> Task :tools:systemTestLibs UP-TO-DATE +> Task :streams:streams-scala:compileScala UP-TO-DATE +> Task :core:compileScala UP-TO-DATE +> Task :streams:streams-scala:classes UP-TO-DATE +> Task :core:classes UP-TO-DATE +> Task :streams:streams-scala:copyDependantLibs UP-TO-DATE +> Task :streams:streams-scala:jar UP-TO-DATE +> Task :streams:streams-scala:systemTestLibs UP-TO-DATE +> Task :test-common:test-common-runtime:compileJava UP-TO-DATE +> Task :test-common:test-common-runtime:classes UP-TO-DATE +> Task :core:copyDependantLibs UP-TO-DATE +> Task :test-common:test-common-runtime:jar UP-TO-DATE +> Task :shell:compileJava UP-TO-DATE +> Task :shell:classes UP-TO-DATE +> Task :test-common:test-common-runtime:systemTestLibs UP-TO-DATE +> Task :core:jar UP-TO-DATE +> Task :clients:compileTestJava UP-TO-DATE +> Task :clients:testClasses UP-TO-DATE +> Task :clients:testJar UP-TO-DATE +> Task :storage:storage-api:compileTestJava UP-TO-DATE +> Task :storage:storage-api:testClasses UP-TO-DATE +> Task :server-common:compileTestJava UP-TO-DATE +> Task :shell:copyDependantLibs UP-TO-DATE +> Task :server-common:testClasses UP-TO-DATE +> Task :shell:jar UP-TO-DATE +> Task :shell:systemTestLibs UP-TO-DATE +> Task :coordinator-common:compileTestJava UP-TO-DATE +> Task :streams:compileTestJava UP-TO-DATE +> Task :coordinator-common:testClasses UP-TO-DATE +> Task :streams:testClasses UP-TO-DATE +> Task :raft:compileTestJava UP-TO-DATE +> Task :raft:testClasses UP-TO-DATE +> Task :server:compileTestJava UP-TO-DATE +> Task :server:testClasses UP-TO-DATE +> Task :streams:testJar UP-TO-DATE +> Task :streams:systemTestLibs UP-TO-DATE +> Task :share-coordinator:compileTestJava UP-TO-DATE +> Task :share-coordinator:testClasses UP-TO-DATE +> Task :group-coordinator:compileTestJava UP-TO-DATE +> Task :group-coordinator:testClasses UP-TO-DATE +> Task :metadata:compileTestJava UP-TO-DATE +> Task :metadata:testClasses UP-TO-DATE +> Task :jmh-benchmarks:compileJava UP-TO-DATE +> Task :jmh-benchmarks:classes UP-TO-DATE +> Task :jmh-benchmarks:jar UP-TO-DATE +> Task :jmh-benchmarks:systemTestLibs UP-TO-DATE +> Task :core:copyDependantTestLibs UP-TO-DATE +> Task :core:compileTestJava NO-SOURCE +> Task :core:compileTestScala UP-TO-DATE +> Task :core:testClasses UP-TO-DATE +> Task :core:testJar UP-TO-DATE +> Task :core:systemTestLibs UP-TO-DATE + +[Incubating] Problems report is available at: file:///Users/gabriellafu/kafka/build/reports/problems/problems-report.html + +Deprecated Gradle features were used in this build, making it incompatible with Gradle 10. + +You can use '--warning-mode all' to show the individual deprecation warnings and determine if they come from your own scripts or plugins. + +For more on this, please refer to https://docs.gradle.org/9.4.1/userguide/command_line_interface.html#sec:command_line_warnings in the Gradle documentation. + +BUILD SUCCESSFUL in 1s +226 actionable tasks: 226 up-to-date +Consider enabling configuration cache to speed up this build: https://docs.gradle.org/9.4.1/userguide/configuration_cache_enabling.html +docker exec ducker01 bash -c "cd /opt/kafka-dev && ducktape --cluster-file /opt/kafka-dev/tests/docker/build/cluster.json ./tests/kafkatest/ " +[WARNING:2026-04-20 14:23:12,089]: No tests loaded for /opt/kafka-dev/tests/kafkatest/tests - verifiable_share_consumer_test.py - - - None +[WARNING:2026-04-20 14:23:12,089]: Didn't find any tests in /opt/kafka-dev/tests/kafkatest/tests/verifiable_share_consumer_test.py +[WARNING:2026-04-20 14:23:12,090]: No tests loaded for /opt/kafka-dev/tests/kafkatest/tests - kafka_test.py - - - None +[WARNING:2026-04-20 14:23:12,090]: Didn't find any tests in /opt/kafka-dev/tests/kafkatest/tests/kafka_test.py +[WARNING:2026-04-20 14:23:12,091]: No tests loaded for /opt/kafka-dev/tests/kafkatest/tests - verifiable_consumer_test.py - - - None +[WARNING:2026-04-20 14:23:12,091]: Didn't find any tests in /opt/kafka-dev/tests/kafkatest/tests/verifiable_consumer_test.py +[WARNING:2026-04-20 14:23:12,130]: No tests loaded for /opt/kafka-dev/tests/kafkatest/tests/streams - base_streams_test.py - - - None +[WARNING:2026-04-20 14:23:12,130]: Didn't find any tests in /opt/kafka-dev/tests/kafkatest/tests/streams/base_streams_test.py +[INFO:2026-04-20 14:23:12,217]: Discovered 1116 tests to run +[INFO:2026-04-20 14:23:12,243]: starting test run with session id 2026-04-20--002... +[INFO:2026-04-20 14:23:12,243]: running 1116 tests... +[INFO:2026-04-20 14:23:12,243]: Triggering test 1 of 1116... +[INFO:2026-04-20 14:23:12,253]: RunnerClient: Loading test {'directory': '/opt/kafka-dev/tests/kafkatest/tests/core', 'file_name': 'replica_scale_test.py', 'cls_name': 'ReplicaScaleTest', 'method_name': 'test_clean_bounce', 'injected_args': {'topic_count': 50, 'partition_count': 34, 'replication_factor': 3, 'metadata_quorum': 'ISOLATED_KRAFT'}} +[INFO:2026-04-20 14:23:12,256]: RunnerClient: kafkatest.tests.core.replica_scale_test.ReplicaScaleTest.test_clean_bounce.topic_count=50.partition_count=34.replication_factor=3.metadata_quorum=ISOLATED_KRAFT: on run 1/1 +[INFO:2026-04-20 14:23:12,257]: RunnerClient: kafkatest.tests.core.replica_scale_test.ReplicaScaleTest.test_clean_bounce.topic_count=50.partition_count=34.replication_factor=3.metadata_quorum=ISOLATED_KRAFT: Setting up... +Creating topic topic-0000 +[INFO:2026-04-20 14:23:35,226]: RunnerClient: kafkatest.tests.core.replica_scale_test.ReplicaScaleTest.test_clean_bounce.topic_count=50.partition_count=34.replication_factor=3.metadata_quorum=ISOLATED_KRAFT: Running... +Creating topic topic-0001 +Creating topic topic-0002 +Creating topic topic-0003 +Creating topic topic-0004 +Creating topic topic-0005 +Creating topic topic-0006 +Creating topic topic-0007 +Creating topic topic-0008 +Creating topic topic-0009 +Creating topic topic-0010 +Creating topic topic-0011 +Creating topic topic-0012 +Creating topic topic-0013 +Creating topic topic-0014 +Creating topic topic-0015 +Creating topic topic-0016 +Creating topic topic-0017 +Creating topic topic-0018 +Creating topic topic-0019 +Creating topic topic-0020 +Creating topic topic-0021 +Creating topic topic-0022 +Creating topic topic-0023 +Creating topic topic-0024 +Creating topic topic-0025 +Creating topic topic-0026 +Creating topic topic-0027 +Creating topic topic-0028 +Creating topic topic-0029 +Creating topic topic-0030 +Creating topic topic-0031 +Creating topic topic-0032 +Creating topic topic-0033 +Creating topic topic-0034 +Creating topic topic-0035 +Creating topic topic-0036 +Creating topic topic-0037 +Creating topic topic-0038 +Creating topic topic-0039 +Creating topic topic-0040 +Creating topic topic-0041 +Creating topic topic-0042 +Creating topic topic-0043 +Creating topic topic-0044 +Creating topic topic-0045 +Creating topic topic-0046 +Creating topic topic-0047 +Creating topic topic-0048 +Creating topic topic-0049 +ducker-ak test failed diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java index aa7098400a1f6..5ee58f393d11e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java @@ -26,6 +26,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.InvalidOffsetException; import org.apache.kafka.clients.consumer.OffsetAndTimestamp; +import org.apache.kafka.clients.admin.ListOffsetsResult.ListOffsetsResultInfo; import org.apache.kafka.common.IsolationLevel; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; @@ -971,7 +972,7 @@ private void prepareChangelogs(final Map tasks, final Set newPartitionsToRestore) { // separate those who do not have the current offset loaded from checkpoint final Set newPartitionsWithoutStartOffset = new HashSet<>(); - final Map newPartitionsWithTimestampSeek = new HashMap<>(); + final Map newPartitionsWithRetentionPeriod = new HashMap<>(); for (final ChangelogMetadata changelogMetadata : newPartitionsToRestore) { final StateStoreMetadata storeMetadata = changelogMetadata.storeMetadata; @@ -989,12 +990,8 @@ private void prepareChangelogs(final Map tasks, partition, currentOffset, recordEndOffset(endOffset)); } else { final long retentionPeriod = storeMetadata.retentionPeriod(); - final long seekTimestamp = retentionPeriod > 0 && retentionPeriod != Long.MAX_VALUE - ? time.milliseconds() - retentionPeriod : -1L; - if (seekTimestamp > 0) { - newPartitionsWithTimestampSeek.put(partition, seekTimestamp); - log.debug("Start restoring windowed changelog partition {} from timestamp {} to end offset {}.", - partition, seekTimestamp, recordEndOffset(endOffset)); + if (retentionPeriod > 0 && retentionPeriod != Long.MAX_VALUE) { + newPartitionsWithRetentionPeriod.put(partition, retentionPeriod); } else { log.debug("Start restoring changelog partition {} from the beginning offset to end offset {} " + "since we cannot find current offset.", partition, recordEndOffset(endOffset)); @@ -1003,6 +1000,9 @@ private void prepareChangelogs(final Map tasks, } } + final Map newPartitionsWithTimestampSeek = + computeTimestampSeekFromStreamTime(newPartitionsWithRetentionPeriod, newPartitionsWithoutStartOffset); + seekToTimestampOrBeginning(newPartitionsWithTimestampSeek, newPartitionsWithoutStartOffset); for (final ChangelogMetadata changelogMetadata : newPartitionsToRestore) { @@ -1045,6 +1045,56 @@ private void prepareChangelogs(final Map tasks, } } + private Map computeTimestampSeekFromStreamTime( + final Map partitionsWithRetentionPeriod, + final Set fallbackPartitions) { + if (partitionsWithRetentionPeriod.isEmpty()) { + return Collections.emptyMap(); + } + + final Map result = new HashMap<>(); + try { + final ListOffsetsOptions options = new ListOffsetsOptions(IsolationLevel.READ_UNCOMMITTED); + final Map offsetSpecs = partitionsWithRetentionPeriod.keySet().stream() + .collect(Collectors.toMap(Function.identity(), tp -> OffsetSpec.maxTimestamp())); + final Map maxTimestamps = + adminClient.listOffsets(offsetSpecs, options).all().get(); + + for (final Map.Entry entry : partitionsWithRetentionPeriod.entrySet()) { + final TopicPartition partition = entry.getKey(); + final long retentionPeriod = entry.getValue(); + final ListOffsetsResultInfo info = maxTimestamps.get(partition); + + if (info != null && info.timestamp() > 0) { + final long seekTimestamp = info.timestamp() - retentionPeriod; + if (seekTimestamp > 0) { + result.put(partition, seekTimestamp); + log.debug("Start restoring windowed changelog partition {} from stream-time-based timestamp {} " + + "(maxStreamTime={}, retention={}).", partition, seekTimestamp, info.timestamp(), retentionPeriod); + } else { + log.debug("Start restoring changelog partition {} from the beginning (computed seek timestamp " + + "is non-positive).", partition); + fallbackPartitions.add(partition); + } + } else { + log.debug("Start restoring changelog partition {} from the beginning (no max timestamp available).", + partition); + fallbackPartitions.add(partition); + } + } + } catch (final TimeoutException | InterruptedException | ExecutionException e) { + log.debug("Could not fetch max timestamps for {}, falling back to seek-to-beginning", + partitionsWithRetentionPeriod.keySet(), e); + fallbackPartitions.addAll(partitionsWithRetentionPeriod.keySet()); + } catch (final KafkaException e) { + log.warn("Failed to fetch max timestamps for {}, falling back to seek-to-beginning", + partitionsWithRetentionPeriod.keySet(), e); + fallbackPartitions.addAll(partitionsWithRetentionPeriod.keySet()); + } + + return result; + } + private void seekToTimestampOrBeginning(final Map partitionsWithTimestampSeek, final Set partitionsWithoutStartOffset) { // optimization: seek windowed stores by timestamp to skip expired data diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java index e24b5c5c3e801..292d166687125 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java @@ -1464,6 +1464,7 @@ public synchronized Map offsetsForTimes(fina timestampConsumer.updateBeginningOffsets(Collections.singletonMap(tp, 0L)); adminClient.updateEndOffsets(Collections.singletonMap(tp, 100L)); + adminClient.updateMaxTimestamps(Collections.singletonMap(tp, time.milliseconds())); final StoreChangelogReader reader = new StoreChangelogReader(time, config, logContext, adminClient, timestampConsumer, callback, standbyListener); @@ -1504,6 +1505,7 @@ public synchronized Map offsetsForTimes(fina timestampConsumer.updateBeginningOffsets(Collections.singletonMap(tp, 0L)); adminClient.updateEndOffsets(Collections.singletonMap(tp, 100L)); + adminClient.updateMaxTimestamps(Collections.singletonMap(tp, time.milliseconds())); final StoreChangelogReader reader = new StoreChangelogReader(time, config, logContext, adminClient, timestampConsumer, callback, standbyListener); From 4c5feae0a7419432c1ccbea3f3db404e97f528de Mon Sep 17 00:00:00 2001 From: Gabriella Fu <2234862823@qq.com> Date: Tue, 21 Apr 2026 17:31:17 -0400 Subject: [PATCH 10/23] fix the bug cause by the gap between wall clock and streams clock --- smoke_test_output.log | 578 ------------------------------------------ 1 file changed, 578 deletions(-) delete mode 100644 smoke_test_output.log diff --git a/smoke_test_output.log b/smoke_test_output.log deleted file mode 100644 index e61a445abce5b..0000000000000 --- a/smoke_test_output.log +++ /dev/null @@ -1,578 +0,0 @@ -WARNING: Ignoring custom format, because both --format and --quiet are set. - -> Configure project : -Starting build with version 4.4.0-SNAPSHOT (commit id 54f7c7a9) using Gradle 9.4.1, Java 17 and Scala 2.13.18 -Build properties: ignoreFailures=false, maxParallelForks=14, maxScalacThreads=8, maxTestRetries=0 - -> Task :core:processResources NO-SOURCE -> Task :storage:processResources UP-TO-DATE -> Task :server-common:processResources NO-SOURCE -> Task :server:processResources NO-SOURCE -> Task :share-coordinator:processResources UP-TO-DATE -> Task :coordinator-common:processResources NO-SOURCE -> Task :storage:createVersionFile UP-TO-DATE -> Task :group-coordinator:group-coordinator-api:processResources NO-SOURCE -> Task :server-common:createVersionFile UP-TO-DATE -> Task :connect:compileJava NO-SOURCE -> Task :storage:storage-api:processResources NO-SOURCE -> Task :server:createVersionFile UP-TO-DATE -> Task :raft:processResources UP-TO-DATE -> Task :connect:processResources NO-SOURCE -> Task :transaction-coordinator:processResources UP-TO-DATE -> Task :group-coordinator:group-coordinator-api:createVersionFile UP-TO-DATE -> Task :connect:classes UP-TO-DATE -> Task :tools:tools-api:processResources NO-SOURCE -> Task :coordinator-common:processTestResources UP-TO-DATE -> Task :metadata:processResources UP-TO-DATE -> Task :server-common:processTestResources UP-TO-DATE -> Task :storage:storage-api:createVersionFile UP-TO-DATE -> Task :raft:createVersionFile UP-TO-DATE -> Task :group-coordinator:processResources UP-TO-DATE -> Task :server:processTestResources UP-TO-DATE -> Task :share-coordinator:processTestResources UP-TO-DATE -> Task :tools:tools-api:createVersionFile UP-TO-DATE -> Task :test-common:test-common-internal-api:processResources NO-SOURCE -> Task :streams:processResources UP-TO-DATE -> Task :metadata:processTestResources UP-TO-DATE -> Task :test-common:test-common-runtime:processResources UP-TO-DATE -> Task :clients:processResources UP-TO-DATE -> Task :storage:storage-api:processTestResources UP-TO-DATE -> Task :streams:createStreamsVersionFile UP-TO-DATE -> Task :raft:processTestResources UP-TO-DATE -> Task :connect:jar UP-TO-DATE -> Task :clients:createVersionFile UP-TO-DATE -> Task :connect:systemTestLibs UP-TO-DATE -> Task :examples:processResources NO-SOURCE -> Task :core:processTestResources UP-TO-DATE -> Task :group-coordinator:processTestResources UP-TO-DATE -> Task :shell:processResources NO-SOURCE -> Task :clients:processTestResources UP-TO-DATE -> Task :connect:api:processResources UP-TO-DATE -> Task :connect:json:processResources UP-TO-DATE -> Task :test-common:compileJava NO-SOURCE -> Task :streams:processTestResources UP-TO-DATE -> Task :jmh-benchmarks:processResources NO-SOURCE -> Task :connect:runtime:processResources UP-TO-DATE -> Task :tools:processResources NO-SOURCE -> Task :connect:transforms:processResources UP-TO-DATE -> Task :trogdor:processResources NO-SOURCE -> Task :test-common:processResources NO-SOURCE -> Task :test-common:classes UP-TO-DATE -> Task :connect:basic-auth-extension:processResources UP-TO-DATE -> Task :clients:clients-integration-tests:processResources NO-SOURCE -> Task :connect:file:processResources UP-TO-DATE -> Task :connect:mirror:processResources UP-TO-DATE -> Task :connect:mirror-client:processResources NO-SOURCE -> Task :connect:test-plugins:processResources UP-TO-DATE -> Task :streams:examples:processResources NO-SOURCE -> Task :streams:test-utils:processResources NO-SOURCE -> Task :streams:streams-scala:processResources NO-SOURCE -> Task :streams:integration-tests:compileJava NO-SOURCE -> Task :streams:upgrade-system-tests-0110:compileJava NO-SOURCE -> Task :streams:upgrade-system-tests-11:compileJava NO-SOURCE -> Task :streams:upgrade-system-tests-23:compileJava NO-SOURCE -> Task :streams:upgrade-system-tests-0110:processResources NO-SOURCE -> Task :streams:upgrade-system-tests-11:processResources NO-SOURCE -> Task :streams:upgrade-system-tests-11:classes UP-TO-DATE -> Task :streams:upgrade-system-tests-20:compileJava NO-SOURCE -> Task :streams:upgrade-system-tests-0110:classes UP-TO-DATE -> Task :streams:upgrade-system-tests-25:compileJava NO-SOURCE -> Task :streams:upgrade-system-tests-22:compileJava NO-SOURCE -> Task :streams:integration-tests:compileScala NO-SOURCE -> Task :streams:upgrade-system-tests-23:processResources NO-SOURCE -> Task :streams:upgrade-system-tests-10:compileJava NO-SOURCE -> Task :streams:upgrade-system-tests-24:compileJava NO-SOURCE -> Task :streams:upgrade-system-tests-21:compileJava NO-SOURCE -> Task :test-common:jar UP-TO-DATE -> Task :test-common:systemTestLibs UP-TO-DATE -> Task :streams:upgrade-system-tests-23:classes UP-TO-DATE -> Task :streams:upgrade-system-tests-20:processResources NO-SOURCE -> Task :streams:upgrade-system-tests-22:processResources NO-SOURCE -> Task :streams:upgrade-system-tests-20:classes UP-TO-DATE -> Task :streams:upgrade-system-tests-11:jar UP-TO-DATE -> Task :streams:integration-tests:processResources NO-SOURCE -> Task :streams:upgrade-system-tests-0110:jar UP-TO-DATE -> Task :streams:integration-tests:classes UP-TO-DATE -> Task :streams:upgrade-system-tests-21:processResources NO-SOURCE -> Task :streams:upgrade-system-tests-10:processResources NO-SOURCE -> Task :streams:upgrade-system-tests-10:classes UP-TO-DATE -> Task :streams:upgrade-system-tests-23:jar UP-TO-DATE -> Task :streams:upgrade-system-tests-26:compileJava NO-SOURCE -> Task :streams:upgrade-system-tests-25:processResources NO-SOURCE -> Task :streams:upgrade-system-tests-25:classes UP-TO-DATE -> Task :streams:upgrade-system-tests-20:jar UP-TO-DATE -> Task :streams:integration-tests:jar UP-TO-DATE -> Task :streams:upgrade-system-tests-26:processResources NO-SOURCE -> Task :streams:integration-tests:systemTestLibs UP-TO-DATE -> Task :streams:upgrade-system-tests-24:processResources NO-SOURCE -> Task :streams:upgrade-system-tests-21:classes UP-TO-DATE -> Task :streams:upgrade-system-tests-10:jar UP-TO-DATE -> Task :streams:upgrade-system-tests-22:classes UP-TO-DATE -> Task :streams:upgrade-system-tests-24:classes UP-TO-DATE -> Task :streams:upgrade-system-tests-26:classes UP-TO-DATE -> Task :streams:upgrade-system-tests-27:compileJava NO-SOURCE -> Task :streams:upgrade-system-tests-27:processResources NO-SOURCE -> Task :streams:upgrade-system-tests-27:classes UP-TO-DATE -> Task :streams:upgrade-system-tests-25:jar UP-TO-DATE -> Task :streams:upgrade-system-tests-21:jar UP-TO-DATE -> Task :streams:upgrade-system-tests-26:jar UP-TO-DATE -> Task :streams:upgrade-system-tests-28:compileJava NO-SOURCE -> Task :streams:upgrade-system-tests-22:jar UP-TO-DATE -> Task :streams:upgrade-system-tests-24:jar UP-TO-DATE -> Task :streams:upgrade-system-tests-27:jar UP-TO-DATE -> Task :streams:upgrade-system-tests-28:processResources NO-SOURCE -> Task :streams:upgrade-system-tests-28:classes UP-TO-DATE -> Task :streams:upgrade-system-tests-28:jar UP-TO-DATE -> Task :test-common:test-common-util:compileJava UP-TO-DATE -> Task :test-common:test-common-util:processResources UP-TO-DATE -> Task :test-common:test-common-util:classes UP-TO-DATE -> Task :test-common:test-common-util:jar UP-TO-DATE -> Task :streams:upgrade-system-tests-30:compileJava NO-SOURCE -> Task :streams:upgrade-system-tests-30:processResources NO-SOURCE -> Task :streams:upgrade-system-tests-30:classes UP-TO-DATE -> Task :streams:upgrade-system-tests-0110:compileTestJava UP-TO-DATE -> Task :streams:upgrade-system-tests-0110:processTestResources NO-SOURCE -> Task :streams:upgrade-system-tests-30:jar UP-TO-DATE -> Task :streams:upgrade-system-tests-0110:testClasses UP-TO-DATE -> Task :streams:upgrade-system-tests-27:compileTestJava UP-TO-DATE -> Task :streams:upgrade-system-tests-26:compileTestJava UP-TO-DATE -> Task :streams:upgrade-system-tests-27:processTestResources NO-SOURCE -> Task :streams:upgrade-system-tests-10:compileTestJava UP-TO-DATE -> Task :streams:upgrade-system-tests-11:compileTestJava UP-TO-DATE -> Task :streams:upgrade-system-tests-25:compileTestJava UP-TO-DATE -> Task :streams:upgrade-system-tests-0110:testJar UP-TO-DATE -> Task :streams:upgrade-system-tests-22:compileTestJava UP-TO-DATE -> Task :streams:upgrade-system-tests-24:compileTestJava UP-TO-DATE -> Task :streams:upgrade-system-tests-20:compileTestJava UP-TO-DATE -> Task :streams:upgrade-system-tests-21:compileTestJava UP-TO-DATE -> Task :streams:upgrade-system-tests-28:compileTestJava UP-TO-DATE -> Task :streams:upgrade-system-tests-23:compileTestJava UP-TO-DATE -> Task :streams:upgrade-system-tests-0110:systemTestLibs UP-TO-DATE -> Task :generator:compileJava UP-TO-DATE -> Task :streams:upgrade-system-tests-10:processTestResources NO-SOURCE -> Task :streams:upgrade-system-tests-27:testClasses UP-TO-DATE -> Task :streams:upgrade-system-tests-11:processTestResources NO-SOURCE -> Task :streams:upgrade-system-tests-20:processTestResources NO-SOURCE -> Task :streams:upgrade-system-tests-26:processTestResources NO-SOURCE -> Task :streams:upgrade-system-tests-26:testClasses UP-TO-DATE -> Task :streams:upgrade-system-tests-24:processTestResources NO-SOURCE -> Task :streams:upgrade-system-tests-21:processTestResources NO-SOURCE -> Task :streams:upgrade-system-tests-22:processTestResources NO-SOURCE -> Task :streams:upgrade-system-tests-20:testClasses UP-TO-DATE -> Task :streams:upgrade-system-tests-25:processTestResources NO-SOURCE -> Task :streams:upgrade-system-tests-10:testClasses UP-TO-DATE -> Task :streams:upgrade-system-tests-28:processTestResources NO-SOURCE -> Task :generator:processResources NO-SOURCE -> Task :streams:upgrade-system-tests-21:testClasses UP-TO-DATE -> Task :streams:upgrade-system-tests-22:testClasses UP-TO-DATE -> Task :streams:upgrade-system-tests-11:testClasses UP-TO-DATE -> Task :streams:upgrade-system-tests-23:processTestResources NO-SOURCE -> Task :streams:upgrade-system-tests-23:testClasses UP-TO-DATE -> Task :streams:upgrade-system-tests-24:testClasses UP-TO-DATE -> Task :generator:classes UP-TO-DATE -> Task :streams:upgrade-system-tests-25:testClasses UP-TO-DATE -> Task :streams:upgrade-system-tests-28:testClasses UP-TO-DATE -> Task :streams:upgrade-system-tests-27:testJar UP-TO-DATE -> Task :streams:upgrade-system-tests-20:testJar UP-TO-DATE -> Task :streams:upgrade-system-tests-27:systemTestLibs UP-TO-DATE -> Task :generator:jar UP-TO-DATE -> Task :streams:upgrade-system-tests-20:systemTestLibs UP-TO-DATE -> Task :streams:upgrade-system-tests-26:testJar UP-TO-DATE -> Task :streams:upgrade-system-tests-30:compileTestJava UP-TO-DATE -> Task :streams:upgrade-system-tests-31:compileJava NO-SOURCE -> Task :streams:upgrade-system-tests-23:testJar UP-TO-DATE -> Task :streams:upgrade-system-tests-11:testJar UP-TO-DATE -> Task :streams:upgrade-system-tests-22:testJar UP-TO-DATE -> Task :streams:upgrade-system-tests-10:testJar UP-TO-DATE -> Task :streams:upgrade-system-tests-21:testJar UP-TO-DATE -> Task :streams:upgrade-system-tests-28:testJar UP-TO-DATE -> Task :streams:upgrade-system-tests-24:testJar UP-TO-DATE -> Task :streams:upgrade-system-tests-25:testJar UP-TO-DATE -> Task :streams:upgrade-system-tests-10:systemTestLibs UP-TO-DATE -> Task :streams:upgrade-system-tests-11:systemTestLibs UP-TO-DATE -> Task :streams:upgrade-system-tests-21:systemTestLibs UP-TO-DATE -> Task :streams:upgrade-system-tests-22:systemTestLibs UP-TO-DATE -> Task :streams:upgrade-system-tests-23:systemTestLibs UP-TO-DATE -> Task :streams:upgrade-system-tests-24:systemTestLibs UP-TO-DATE -> Task :streams:upgrade-system-tests-26:systemTestLibs UP-TO-DATE -> Task :streams:upgrade-system-tests-30:processTestResources NO-SOURCE -> Task :streams:upgrade-system-tests-31:processResources NO-SOURCE -> Task :generator:systemTestLibs UP-TO-DATE -> Task :streams:upgrade-system-tests-25:systemTestLibs UP-TO-DATE -> Task :streams:upgrade-system-tests-28:systemTestLibs UP-TO-DATE -> Task :streams:upgrade-system-tests-30:testClasses UP-TO-DATE -> Task :streams:upgrade-system-tests-31:classes UP-TO-DATE -> Task :streams:upgrade-system-tests-32:compileJava NO-SOURCE -> Task :streams:upgrade-system-tests-33:compileJava NO-SOURCE -> Task :streams:upgrade-system-tests-34:compileJava NO-SOURCE -> Task :streams:upgrade-system-tests-32:processResources NO-SOURCE -> Task :streams:upgrade-system-tests-32:classes UP-TO-DATE -> Task :streams:upgrade-system-tests-34:processResources NO-SOURCE -> Task :streams:upgrade-system-tests-34:classes UP-TO-DATE -> Task :share-coordinator:processMessages UP-TO-DATE -> Task :streams:upgrade-system-tests-33:processResources NO-SOURCE -> Task :streams:upgrade-system-tests-35:compileJava NO-SOURCE -> Task :streams:upgrade-system-tests-30:testJar UP-TO-DATE -> Task :streams:upgrade-system-tests-31:jar UP-TO-DATE -> Task :streams:processMessages UP-TO-DATE -> Task :streams:upgrade-system-tests-34:jar UP-TO-DATE -> Task :raft:processMessages UP-TO-DATE -> Task :clients:processMessages UP-TO-DATE -> Task :metadata:processMessages UP-TO-DATE -> Task :group-coordinator:processMessages UP-TO-DATE -> Task :streams:upgrade-system-tests-33:classes UP-TO-DATE -> Task :transaction-coordinator:processMessages UP-TO-DATE -> Task :streams:upgrade-system-tests-35:processResources NO-SOURCE -> Task :streams:upgrade-system-tests-35:classes UP-TO-DATE -> Task :streams:upgrade-system-tests-36:compileJava NO-SOURCE -> Task :streams:upgrade-system-tests-32:jar UP-TO-DATE -> Task :storage:processMessages UP-TO-DATE -> Task :streams:upgrade-system-tests-36:processResources NO-SOURCE -> Task :test-common:test-common-util:systemTestLibs UP-TO-DATE -> Task :streams:upgrade-system-tests-30:systemTestLibs UP-TO-DATE -> Task :streams:upgrade-system-tests-36:classes UP-TO-DATE -> Task :streams:upgrade-system-tests-37:compileJava NO-SOURCE -> Task :streams:upgrade-system-tests-39:compileJava NO-SOURCE -> Task :streams:upgrade-system-tests-35:jar UP-TO-DATE -> Task :streams:upgrade-system-tests-37:processResources NO-SOURCE -> Task :streams:upgrade-system-tests-41:compileJava NO-SOURCE -> Task :streams:upgrade-system-tests-37:classes UP-TO-DATE -> Task :streams:upgrade-system-tests-39:processResources NO-SOURCE -> Task :streams:upgrade-system-tests-41:processResources NO-SOURCE -> Task :streams:upgrade-system-tests-39:classes UP-TO-DATE -> Task :streams:upgrade-system-tests-33:jar UP-TO-DATE -> Task :streams:upgrade-system-tests-41:classes UP-TO-DATE -> Task :streams:upgrade-system-tests-38:compileJava NO-SOURCE -> Task :streams:upgrade-system-tests-36:jar UP-TO-DATE -> Task :streams:upgrade-system-tests-40:compileJava NO-SOURCE -> Task :streams:upgrade-system-tests-39:jar UP-TO-DATE -> Task :streams:upgrade-system-tests-38:processResources NO-SOURCE -> Task :streams:upgrade-system-tests-37:jar UP-TO-DATE -> Task :streams:upgrade-system-tests-40:processResources NO-SOURCE -> Task :streams:upgrade-system-tests-40:classes UP-TO-DATE -> Task :streams:upgrade-system-tests-38:classes UP-TO-DATE -> Task :streams:upgrade-system-tests-41:jar UP-TO-DATE -> Task :streams:upgrade-system-tests-38:jar UP-TO-DATE -> Task :streams:upgrade-system-tests-40:jar UP-TO-DATE -> Task :streams:upgrade-system-tests-32:compileTestJava UP-TO-DATE -> Task :streams:upgrade-system-tests-34:compileTestJava UP-TO-DATE -> Task :streams:upgrade-system-tests-37:compileTestJava UP-TO-DATE -> Task :streams:upgrade-system-tests-36:compileTestJava UP-TO-DATE -> Task :streams:upgrade-system-tests-32:processTestResources NO-SOURCE -> Task :streams:upgrade-system-tests-35:compileTestJava UP-TO-DATE -> Task :streams:upgrade-system-tests-36:processTestResources NO-SOURCE -> Task :streams:upgrade-system-tests-33:compileTestJava UP-TO-DATE -> Task :streams:upgrade-system-tests-34:processTestResources NO-SOURCE -> Task :streams:upgrade-system-tests-32:testClasses UP-TO-DATE -> Task :streams:upgrade-system-tests-34:testClasses UP-TO-DATE -> Task :streams:upgrade-system-tests-31:compileTestJava UP-TO-DATE -> Task :streams:upgrade-system-tests-37:processTestResources NO-SOURCE -> Task :streams:upgrade-system-tests-36:testClasses UP-TO-DATE -> Task :streams:upgrade-system-tests-38:compileTestJava UP-TO-DATE -> Task :streams:upgrade-system-tests-41:compileTestJava UP-TO-DATE -> Task :streams:upgrade-system-tests-39:compileTestJava UP-TO-DATE -> Task :streams:upgrade-system-tests-33:processTestResources NO-SOURCE -> Task :streams:upgrade-system-tests-33:testClasses UP-TO-DATE -> Task :streams:upgrade-system-tests-35:processTestResources NO-SOURCE -> Task :streams:upgrade-system-tests-40:compileTestJava UP-TO-DATE -> Task :streams:upgrade-system-tests-35:testClasses UP-TO-DATE -> Task :streams:upgrade-system-tests-39:processTestResources NO-SOURCE -> Task :streams:upgrade-system-tests-37:testClasses UP-TO-DATE -> Task :streams:upgrade-system-tests-38:processTestResources NO-SOURCE -> Task :streams:upgrade-system-tests-31:processTestResources NO-SOURCE -> Task :streams:upgrade-system-tests-41:processTestResources NO-SOURCE -> Task :streams:upgrade-system-tests-31:testClasses UP-TO-DATE -> Task :streams:upgrade-system-tests-40:processTestResources NO-SOURCE -> Task :streams:upgrade-system-tests-36:testJar UP-TO-DATE -> Task :streams:upgrade-system-tests-41:testClasses UP-TO-DATE -> Task :streams:upgrade-system-tests-33:testJar UP-TO-DATE -> Task :streams:upgrade-system-tests-39:testClasses UP-TO-DATE -> Task :streams:upgrade-system-tests-33:systemTestLibs UP-TO-DATE -> Task :streams:upgrade-system-tests-38:testClasses UP-TO-DATE -> Task :streams:upgrade-system-tests-32:testJar UP-TO-DATE -> Task :streams:upgrade-system-tests-34:testJar UP-TO-DATE -> Task :clients:compileJava UP-TO-DATE -> Task :streams:upgrade-system-tests-31:testJar UP-TO-DATE -> Task :streams:upgrade-system-tests-35:testJar UP-TO-DATE -> Task :streams:upgrade-system-tests-32:systemTestLibs UP-TO-DATE -> Task :streams:upgrade-system-tests-37:testJar UP-TO-DATE -> Task :streams:upgrade-system-tests-34:systemTestLibs UP-TO-DATE -> Task :streams:upgrade-system-tests-36:systemTestLibs UP-TO-DATE -> Task :streams:upgrade-system-tests-31:systemTestLibs UP-TO-DATE -> Task :streams:upgrade-system-tests-37:systemTestLibs UP-TO-DATE -> Task :streams:upgrade-system-tests-38:testJar UP-TO-DATE -> Task :streams:upgrade-system-tests-39:testJar UP-TO-DATE -> Task :streams:upgrade-system-tests-38:systemTestLibs UP-TO-DATE -> Task :streams:upgrade-system-tests-39:systemTestLibs UP-TO-DATE -> Task :clients:classes UP-TO-DATE -> Task :streams:upgrade-system-tests-40:testClasses UP-TO-DATE -> Task :streams:upgrade-system-tests-41:testJar UP-TO-DATE -> Task :streams:upgrade-system-tests-35:systemTestLibs UP-TO-DATE -> Task :streams:upgrade-system-tests-41:systemTestLibs UP-TO-DATE -> Task :connect:mirror-client:compileJava UP-TO-DATE -> Task :connect:mirror-client:classes UP-TO-DATE -> Task :tools:tools-api:compileJava UP-TO-DATE -> Task :examples:compileJava UP-TO-DATE -> Task :tools:tools-api:classes UP-TO-DATE -> Task :group-coordinator:group-coordinator-api:compileJava UP-TO-DATE -> Task :examples:classes UP-TO-DATE -> Task :group-coordinator:group-coordinator-api:classes UP-TO-DATE -> Task :streams:upgrade-system-tests-40:testJar UP-TO-DATE -> Task :streams:upgrade-system-tests-40:systemTestLibs UP-TO-DATE -> Task :examples:jar UP-TO-DATE -> Task :examples:systemTestLibs UP-TO-DATE -> Task :connect:api:compileJava UP-TO-DATE -> Task :group-coordinator:group-coordinator-api:jar UP-TO-DATE -> Task :tools:tools-api:jar UP-TO-DATE -> Task :connect:api:classes UP-TO-DATE -> Task :server-common:compileJava UP-TO-DATE -> Task :tools:tools-api:systemTestLibs UP-TO-DATE -> Task :connect:file:compileJava UP-TO-DATE -> Task :connect:json:compileJava UP-TO-DATE -> Task :connect:basic-auth-extension:compileJava UP-TO-DATE -> Task :group-coordinator:group-coordinator-api:systemTestLibs UP-TO-DATE -> Task :connect:transforms:compileJava UP-TO-DATE -> Task :connect:basic-auth-extension:classes UP-TO-DATE -> Task :connect:json:classes UP-TO-DATE -> Task :connect:transforms:classes UP-TO-DATE -> Task :server-common:classes UP-TO-DATE -> Task :storage:storage-api:compileJava UP-TO-DATE -> Task :connect:file:classes UP-TO-DATE -> Task :connect:test-plugins:compileJava UP-TO-DATE -> Task :test-common:test-common-internal-api:compileJava UP-TO-DATE -> Task :storage:storage-api:classes UP-TO-DATE -> Task :test-common:test-common-internal-api:classes UP-TO-DATE -> Task :connect:test-plugins:classes UP-TO-DATE -> Task :server-common:jar UP-TO-DATE -> Task :server-common:systemTestLibs UP-TO-DATE -> Task :clients:shadowJar UP-TO-DATE -> Task :connect:test-plugins:jar UP-TO-DATE -> Task :clients:jar SKIPPED -> Task :test-common:test-common-internal-api:jar UP-TO-DATE -> Task :clients:systemTestLibs UP-TO-DATE -> Task :connect:test-plugins:systemTestLibs UP-TO-DATE -> Task :test-common:test-common-internal-api:systemTestLibs UP-TO-DATE -> Task :storage:storage-api:jar UP-TO-DATE -> Task :storage:storage-api:systemTestLibs UP-TO-DATE -> Task :clients:processTestMessages UP-TO-DATE -> Task :connect:mirror-client:copyDependantLibs UP-TO-DATE -> Task :connect:api:copyDependantLibs UP-TO-DATE -> Task :storage:compileJava UP-TO-DATE -> Task :storage:classes UP-TO-DATE -> Task :connect:mirror-client:jar UP-TO-DATE -> Task :connect:api:jar UP-TO-DATE -> Task :connect:mirror-client:systemTestLibs UP-TO-DATE -> Task :connect:api:systemTestLibs UP-TO-DATE -> Task :connect:transforms:copyDependantLibs UP-TO-DATE -> Task :connect:file:copyDependantLibs UP-TO-DATE -> Task :storage:jar UP-TO-DATE -> Task :raft:compileJava UP-TO-DATE -> Task :connect:transforms:jar UP-TO-DATE -> Task :connect:basic-auth-extension:copyDependantLibs UP-TO-DATE -> Task :storage:systemTestLibs UP-TO-DATE -> Task :connect:json:copyDependantLibs UP-TO-DATE -> Task :raft:classes UP-TO-DATE -> Task :connect:file:jar UP-TO-DATE -> Task :connect:file:systemTestLibs UP-TO-DATE -> Task :connect:transforms:systemTestLibs UP-TO-DATE -> Task :connect:basic-auth-extension:jar UP-TO-DATE -> Task :connect:json:jar UP-TO-DATE -> Task :connect:basic-auth-extension:systemTestLibs UP-TO-DATE -> Task :connect:json:systemTestLibs UP-TO-DATE -> Task :raft:jar UP-TO-DATE -> Task :raft:systemTestLibs UP-TO-DATE -> Task :metadata:compileJava UP-TO-DATE -> Task :metadata:classes UP-TO-DATE -> Task :metadata:jar UP-TO-DATE -> Task :metadata:systemTestLibs UP-TO-DATE -> Task :streams:compileJava UP-TO-DATE -> Task :streams:classes UP-TO-DATE -> Task :streams:streams-scala:compileJava NO-SOURCE -> Task :streams:test-utils:compileJava UP-TO-DATE -> Task :streams:test-utils:classes UP-TO-DATE -> Task :streams:examples:compileJava UP-TO-DATE -> Task :streams:copyDependantLibs UP-TO-DATE -> Task :streams:examples:classes UP-TO-DATE -> Task :streams:jar UP-TO-DATE -> Task :coordinator-common:compileJava UP-TO-DATE -> Task :coordinator-common:classes UP-TO-DATE -> Task :coordinator-common:jar UP-TO-DATE -> Task :coordinator-common:systemTestLibs UP-TO-DATE -> Task :streams:examples:copyDependantLibs UP-TO-DATE -> Task :streams:test-utils:copyDependantLibs UP-TO-DATE -> Task :share-coordinator:compileJava UP-TO-DATE -> Task :transaction-coordinator:compileJava UP-TO-DATE -> Task :share-coordinator:classes UP-TO-DATE -> Task :streams:test-utils:jar UP-TO-DATE -> Task :transaction-coordinator:classes UP-TO-DATE -> Task :streams:examples:jar UP-TO-DATE -> Task :streams:test-utils:systemTestLibs UP-TO-DATE -> Task :streams:examples:systemTestLibs UP-TO-DATE -> Task :share-coordinator:jar UP-TO-DATE -> Task :transaction-coordinator:jar UP-TO-DATE -> Task :transaction-coordinator:systemTestLibs UP-TO-DATE -> Task :share-coordinator:systemTestLibs UP-TO-DATE -> Task :group-coordinator:compileJava UP-TO-DATE -> Task :group-coordinator:classes UP-TO-DATE -> Task :clients:clients-integration-tests:compileJava NO-SOURCE -> Task :clients:clients-integration-tests:classes UP-TO-DATE -> Task :group-coordinator:jar UP-TO-DATE -> Task :group-coordinator:systemTestLibs UP-TO-DATE -> Task :clients:clients-integration-tests:jar UP-TO-DATE -> Task :clients:clients-integration-tests:systemTestLibs UP-TO-DATE -> Task :server:compileJava UP-TO-DATE -> Task :server:classes UP-TO-DATE -> Task :core:compileJava NO-SOURCE -> Task :server:jar UP-TO-DATE -> Task :server:systemTestLibs UP-TO-DATE -> Task :trogdor:compileJava UP-TO-DATE -> Task :trogdor:classes UP-TO-DATE -> Task :connect:runtime:compileJava UP-TO-DATE -> Task :connect:runtime:classes UP-TO-DATE -> Task :tools:compileJava UP-TO-DATE -> Task :tools:classes UP-TO-DATE -> Task :connect:mirror:compileJava UP-TO-DATE -> Task :connect:mirror:classes UP-TO-DATE -> Task :connect:runtime:copyDependantLibs UP-TO-DATE -> Task :trogdor:copyDependantLibs UP-TO-DATE -> Task :connect:runtime:jar UP-TO-DATE -> Task :connect:runtime:systemTestLibs UP-TO-DATE -> Task :trogdor:jar UP-TO-DATE -> Task :trogdor:systemTestLibs UP-TO-DATE -> Task :connect:mirror:copyDependantLibs UP-TO-DATE -> Task :connect:mirror:jar UP-TO-DATE -> Task :connect:mirror:systemTestLibs UP-TO-DATE -> Task :tools:copyDependantLibs UP-TO-DATE -> Task :tools:jar UP-TO-DATE -> Task :tools:systemTestLibs UP-TO-DATE -> Task :streams:streams-scala:compileScala UP-TO-DATE -> Task :core:compileScala UP-TO-DATE -> Task :streams:streams-scala:classes UP-TO-DATE -> Task :core:classes UP-TO-DATE -> Task :streams:streams-scala:copyDependantLibs UP-TO-DATE -> Task :streams:streams-scala:jar UP-TO-DATE -> Task :streams:streams-scala:systemTestLibs UP-TO-DATE -> Task :test-common:test-common-runtime:compileJava UP-TO-DATE -> Task :test-common:test-common-runtime:classes UP-TO-DATE -> Task :core:copyDependantLibs UP-TO-DATE -> Task :test-common:test-common-runtime:jar UP-TO-DATE -> Task :shell:compileJava UP-TO-DATE -> Task :shell:classes UP-TO-DATE -> Task :test-common:test-common-runtime:systemTestLibs UP-TO-DATE -> Task :core:jar UP-TO-DATE -> Task :clients:compileTestJava UP-TO-DATE -> Task :clients:testClasses UP-TO-DATE -> Task :clients:testJar UP-TO-DATE -> Task :storage:storage-api:compileTestJava UP-TO-DATE -> Task :storage:storage-api:testClasses UP-TO-DATE -> Task :server-common:compileTestJava UP-TO-DATE -> Task :shell:copyDependantLibs UP-TO-DATE -> Task :server-common:testClasses UP-TO-DATE -> Task :shell:jar UP-TO-DATE -> Task :shell:systemTestLibs UP-TO-DATE -> Task :coordinator-common:compileTestJava UP-TO-DATE -> Task :streams:compileTestJava UP-TO-DATE -> Task :coordinator-common:testClasses UP-TO-DATE -> Task :streams:testClasses UP-TO-DATE -> Task :raft:compileTestJava UP-TO-DATE -> Task :raft:testClasses UP-TO-DATE -> Task :server:compileTestJava UP-TO-DATE -> Task :server:testClasses UP-TO-DATE -> Task :streams:testJar UP-TO-DATE -> Task :streams:systemTestLibs UP-TO-DATE -> Task :share-coordinator:compileTestJava UP-TO-DATE -> Task :share-coordinator:testClasses UP-TO-DATE -> Task :group-coordinator:compileTestJava UP-TO-DATE -> Task :group-coordinator:testClasses UP-TO-DATE -> Task :metadata:compileTestJava UP-TO-DATE -> Task :metadata:testClasses UP-TO-DATE -> Task :jmh-benchmarks:compileJava UP-TO-DATE -> Task :jmh-benchmarks:classes UP-TO-DATE -> Task :jmh-benchmarks:jar UP-TO-DATE -> Task :jmh-benchmarks:systemTestLibs UP-TO-DATE -> Task :core:copyDependantTestLibs UP-TO-DATE -> Task :core:compileTestJava NO-SOURCE -> Task :core:compileTestScala UP-TO-DATE -> Task :core:testClasses UP-TO-DATE -> Task :core:testJar UP-TO-DATE -> Task :core:systemTestLibs UP-TO-DATE - -[Incubating] Problems report is available at: file:///Users/gabriellafu/kafka/build/reports/problems/problems-report.html - -Deprecated Gradle features were used in this build, making it incompatible with Gradle 10. - -You can use '--warning-mode all' to show the individual deprecation warnings and determine if they come from your own scripts or plugins. - -For more on this, please refer to https://docs.gradle.org/9.4.1/userguide/command_line_interface.html#sec:command_line_warnings in the Gradle documentation. - -BUILD SUCCESSFUL in 1s -226 actionable tasks: 226 up-to-date -Consider enabling configuration cache to speed up this build: https://docs.gradle.org/9.4.1/userguide/configuration_cache_enabling.html -docker exec ducker01 bash -c "cd /opt/kafka-dev && ducktape --cluster-file /opt/kafka-dev/tests/docker/build/cluster.json ./tests/kafkatest/ " -[WARNING:2026-04-20 14:23:12,089]: No tests loaded for /opt/kafka-dev/tests/kafkatest/tests - verifiable_share_consumer_test.py - - - None -[WARNING:2026-04-20 14:23:12,089]: Didn't find any tests in /opt/kafka-dev/tests/kafkatest/tests/verifiable_share_consumer_test.py -[WARNING:2026-04-20 14:23:12,090]: No tests loaded for /opt/kafka-dev/tests/kafkatest/tests - kafka_test.py - - - None -[WARNING:2026-04-20 14:23:12,090]: Didn't find any tests in /opt/kafka-dev/tests/kafkatest/tests/kafka_test.py -[WARNING:2026-04-20 14:23:12,091]: No tests loaded for /opt/kafka-dev/tests/kafkatest/tests - verifiable_consumer_test.py - - - None -[WARNING:2026-04-20 14:23:12,091]: Didn't find any tests in /opt/kafka-dev/tests/kafkatest/tests/verifiable_consumer_test.py -[WARNING:2026-04-20 14:23:12,130]: No tests loaded for /opt/kafka-dev/tests/kafkatest/tests/streams - base_streams_test.py - - - None -[WARNING:2026-04-20 14:23:12,130]: Didn't find any tests in /opt/kafka-dev/tests/kafkatest/tests/streams/base_streams_test.py -[INFO:2026-04-20 14:23:12,217]: Discovered 1116 tests to run -[INFO:2026-04-20 14:23:12,243]: starting test run with session id 2026-04-20--002... -[INFO:2026-04-20 14:23:12,243]: running 1116 tests... -[INFO:2026-04-20 14:23:12,243]: Triggering test 1 of 1116... -[INFO:2026-04-20 14:23:12,253]: RunnerClient: Loading test {'directory': '/opt/kafka-dev/tests/kafkatest/tests/core', 'file_name': 'replica_scale_test.py', 'cls_name': 'ReplicaScaleTest', 'method_name': 'test_clean_bounce', 'injected_args': {'topic_count': 50, 'partition_count': 34, 'replication_factor': 3, 'metadata_quorum': 'ISOLATED_KRAFT'}} -[INFO:2026-04-20 14:23:12,256]: RunnerClient: kafkatest.tests.core.replica_scale_test.ReplicaScaleTest.test_clean_bounce.topic_count=50.partition_count=34.replication_factor=3.metadata_quorum=ISOLATED_KRAFT: on run 1/1 -[INFO:2026-04-20 14:23:12,257]: RunnerClient: kafkatest.tests.core.replica_scale_test.ReplicaScaleTest.test_clean_bounce.topic_count=50.partition_count=34.replication_factor=3.metadata_quorum=ISOLATED_KRAFT: Setting up... -Creating topic topic-0000 -[INFO:2026-04-20 14:23:35,226]: RunnerClient: kafkatest.tests.core.replica_scale_test.ReplicaScaleTest.test_clean_bounce.topic_count=50.partition_count=34.replication_factor=3.metadata_quorum=ISOLATED_KRAFT: Running... -Creating topic topic-0001 -Creating topic topic-0002 -Creating topic topic-0003 -Creating topic topic-0004 -Creating topic topic-0005 -Creating topic topic-0006 -Creating topic topic-0007 -Creating topic topic-0008 -Creating topic topic-0009 -Creating topic topic-0010 -Creating topic topic-0011 -Creating topic topic-0012 -Creating topic topic-0013 -Creating topic topic-0014 -Creating topic topic-0015 -Creating topic topic-0016 -Creating topic topic-0017 -Creating topic topic-0018 -Creating topic topic-0019 -Creating topic topic-0020 -Creating topic topic-0021 -Creating topic topic-0022 -Creating topic topic-0023 -Creating topic topic-0024 -Creating topic topic-0025 -Creating topic topic-0026 -Creating topic topic-0027 -Creating topic topic-0028 -Creating topic topic-0029 -Creating topic topic-0030 -Creating topic topic-0031 -Creating topic topic-0032 -Creating topic topic-0033 -Creating topic topic-0034 -Creating topic topic-0035 -Creating topic topic-0036 -Creating topic topic-0037 -Creating topic topic-0038 -Creating topic topic-0039 -Creating topic topic-0040 -Creating topic topic-0041 -Creating topic topic-0042 -Creating topic topic-0043 -Creating topic topic-0044 -Creating topic topic-0045 -Creating topic topic-0046 -Creating topic topic-0047 -Creating topic topic-0048 -Creating topic topic-0049 -ducker-ak test failed From 4d61fb07baf92a320d4964d7195ab61eed59412a Mon Sep 17 00:00:00 2001 From: Gabriella Fu <2234862823@qq.com> Date: Tue, 21 Apr 2026 23:27:51 -0400 Subject: [PATCH 11/23] use the latest timestamp as the time to restore --- .../internals/StoreChangelogReader.java | 65 +++++++++++++++---- .../internals/StoreChangelogReaderTest.java | 22 ++++++- 2 files changed, 71 insertions(+), 16 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java index 5ee58f393d11e..d3b1925874b7f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java @@ -26,7 +26,6 @@ import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.InvalidOffsetException; import org.apache.kafka.clients.consumer.OffsetAndTimestamp; -import org.apache.kafka.clients.admin.ListOffsetsResult.ListOffsetsResultInfo; import org.apache.kafka.common.IsolationLevel; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; @@ -1054,40 +1053,78 @@ private Map computeTimestampSeekFromStreamTime( final Map result = new HashMap<>(); try { - final ListOffsetsOptions options = new ListOffsetsOptions(IsolationLevel.READ_UNCOMMITTED); - final Map offsetSpecs = partitionsWithRetentionPeriod.keySet().stream() - .collect(Collectors.toMap(Function.identity(), tp -> OffsetSpec.maxTimestamp())); - final Map maxTimestamps = - adminClient.listOffsets(offsetSpecs, options).all().get(); + // Save pause state so we can restore it after reading the last records + final Set allAssigned = restoreConsumer.assignment(); + final Set previouslyPaused = new HashSet<>(restoreConsumer.paused()); + + // Pause all partitions, then resume only the target partitions + restoreConsumer.pause(allAssigned); + final Set targetPartitions = partitionsWithRetentionPeriod.keySet(); + restoreConsumer.resume(targetPartitions); + + // Seek each target partition to near the end to read the last records + for (final TopicPartition partition : targetPartitions) { + final ChangelogMetadata changelogMeta = changelogs.get(partition); + final long endOffset = changelogMeta.restoreEndOffset; + if (endOffset > 0) { + restoreConsumer.seek(partition, Math.max(0, endOffset - 10)); + } else { + fallbackPartitions.add(partition); + } + } + // Poll to get the last records — these are actual data records (consumer filters out + // transaction markers), so their timestamps reflect stream time, not wall-clock time + final ConsumerRecords records = restoreConsumer.poll(pollTime); + + // Find the max timestamp per partition from actual data records + final Map maxTimestamps = new HashMap<>(); + for (final TopicPartition partition : targetPartitions) { + final ChangelogMetadata changelogMeta = changelogs.get(partition); + for (final ConsumerRecord record : records.records(partition)) { + if (record.offset() < changelogMeta.restoreEndOffset) { + maxTimestamps.merge(partition, record.timestamp(), Math::max); + } + } + } + + // Compute seek timestamps based on actual stream-time timestamps for (final Map.Entry entry : partitionsWithRetentionPeriod.entrySet()) { final TopicPartition partition = entry.getKey(); final long retentionPeriod = entry.getValue(); - final ListOffsetsResultInfo info = maxTimestamps.get(partition); + final Long maxTimestamp = maxTimestamps.get(partition); - if (info != null && info.timestamp() > 0) { - final long seekTimestamp = info.timestamp() - retentionPeriod; + if (maxTimestamp != null && maxTimestamp > 0) { + final long seekTimestamp = maxTimestamp - retentionPeriod; if (seekTimestamp > 0) { result.put(partition, seekTimestamp); log.debug("Start restoring windowed changelog partition {} from stream-time-based timestamp {} " + - "(maxStreamTime={}, retention={}).", partition, seekTimestamp, info.timestamp(), retentionPeriod); + "(maxStreamTime={}, retention={}).", partition, seekTimestamp, maxTimestamp, retentionPeriod); } else { log.debug("Start restoring changelog partition {} from the beginning (computed seek timestamp " + "is non-positive).", partition); fallbackPartitions.add(partition); } } else { - log.debug("Start restoring changelog partition {} from the beginning (no max timestamp available).", + log.debug("Start restoring changelog partition {} from the beginning (no records found near end).", partition); fallbackPartitions.add(partition); } } - } catch (final TimeoutException | InterruptedException | ExecutionException e) { - log.debug("Could not fetch max timestamps for {}, falling back to seek-to-beginning", + + // Restore the original pause state + restoreConsumer.pause(allAssigned); + final Set toResume = new HashSet<>(allAssigned); + toResume.removeAll(previouslyPaused); + if (!toResume.isEmpty()) { + restoreConsumer.resume(toResume); + } + } catch (final TimeoutException e) { + log.debug("Could not read last records from changelog for {}, falling back to seek-to-beginning", partitionsWithRetentionPeriod.keySet(), e); fallbackPartitions.addAll(partitionsWithRetentionPeriod.keySet()); } catch (final KafkaException e) { - log.warn("Failed to fetch max timestamps for {}, falling back to seek-to-beginning", + log.warn("Failed to read last records from changelog for {}, falling back to seek-to-beginning", partitionsWithRetentionPeriod.keySet(), e); fallbackPartitions.addAll(partitionsWithRetentionPeriod.keySet()); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java index 292d166687125..caa1fd55ecc8c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java @@ -62,10 +62,14 @@ import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.Optional; import java.util.Properties; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.kafka.common.header.internals.RecordHeaders; +import org.apache.kafka.common.record.TimestampType; + import static java.util.Collections.singletonMap; import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; @@ -1464,7 +1468,14 @@ public synchronized Map offsetsForTimes(fina timestampConsumer.updateBeginningOffsets(Collections.singletonMap(tp, 0L)); adminClient.updateEndOffsets(Collections.singletonMap(tp, 100L)); - adminClient.updateMaxTimestamps(Collections.singletonMap(tp, time.milliseconds())); + + // Temporarily assign, add a record near the end with an explicit stream-time + // timestamp, then clear the assignment (the reader will re-assign during restore) + timestampConsumer.assign(Collections.singletonList(tp)); + timestampConsumer.addRecord(new ConsumerRecord<>(topicName, 0, 95, time.milliseconds(), + TimestampType.CREATE_TIME, 0, 0, new byte[0], new byte[0], + new RecordHeaders(), Optional.empty())); + timestampConsumer.assign(Collections.emptyList()); final StoreChangelogReader reader = new StoreChangelogReader(time, config, logContext, adminClient, timestampConsumer, callback, standbyListener); @@ -1505,7 +1516,14 @@ public synchronized Map offsetsForTimes(fina timestampConsumer.updateBeginningOffsets(Collections.singletonMap(tp, 0L)); adminClient.updateEndOffsets(Collections.singletonMap(tp, 100L)); - adminClient.updateMaxTimestamps(Collections.singletonMap(tp, time.milliseconds())); + + // Temporarily assign, add a record near the end with an explicit timestamp, + // then clear the assignment + timestampConsumer.assign(Collections.singletonList(tp)); + timestampConsumer.addRecord(new ConsumerRecord<>(topicName, 0, 95, time.milliseconds(), + TimestampType.CREATE_TIME, 0, 0, new byte[0], new byte[0], + new RecordHeaders(), Optional.empty())); + timestampConsumer.assign(Collections.emptyList()); final StoreChangelogReader reader = new StoreChangelogReader(time, config, logContext, adminClient, timestampConsumer, callback, standbyListener); From ee9aee22216680a905a44bd094fb091e50c962da Mon Sep 17 00:00:00 2001 From: Gabriella Fu <2234862823@qq.com> Date: Wed, 22 Apr 2026 00:12:29 -0400 Subject: [PATCH 12/23] cleanup some code --- .../kafka/clients/admin/MockAdminClient.java | 11 +- .../internals/StoreChangelogReader.java | 169 ++++++++---------- .../state/internals/InMemorySessionStore.java | 2 +- .../state/internals/WithRetentionPeriod.java | 2 +- .../internals/StoreChangelogReaderTest.java | 4 +- 5 files changed, 77 insertions(+), 111 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java index ab249b2a90998..2d43b135dfb23 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java @@ -89,7 +89,6 @@ public class MockAdminClient extends AdminClient { private final Map beginningOffsets; private final Map endOffsets; private final Map committedOffsets; - private final Map maxTimestamps; private final boolean usingRaftController; private final Map featureLevels; private final Map minSupportedFeatureLevels; @@ -270,7 +269,6 @@ private MockAdminClient( this.beginningOffsets = new HashMap<>(); this.endOffsets = new HashMap<>(); this.committedOffsets = new HashMap<>(); - this.maxTimestamps = new HashMap<>(); this.usingRaftController = usingRaftController; this.featureLevels = new HashMap<>(featureLevels); this.minSupportedFeatureLevels = new HashMap<>(minSupportedFeatureLevels); @@ -1233,11 +1231,7 @@ public synchronized ListOffsetsResult listOffsets(Map newOff endOffsets.putAll(newOffsets); } - public synchronized void updateMaxTimestamps(final Map newTimestamps) { - maxTimestamps.putAll(newTimestamps); - } public synchronized void updateConsumerGroupOffsets(final Map newOffsets) { committedOffsets.putAll(newOffsets); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java index d3b1925874b7f..56df22520173e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java @@ -999,10 +999,7 @@ private void prepareChangelogs(final Map tasks, } } - final Map newPartitionsWithTimestampSeek = - computeTimestampSeekFromStreamTime(newPartitionsWithRetentionPeriod, newPartitionsWithoutStartOffset); - - seekToTimestampOrBeginning(newPartitionsWithTimestampSeek, newPartitionsWithoutStartOffset); + seekByRetentionOrBeginning(newPartitionsWithRetentionPeriod, newPartitionsWithoutStartOffset); for (final ChangelogMetadata changelogMetadata : newPartitionsToRestore) { final StateStoreMetadata storeMetadata = changelogMetadata.storeMetadata; @@ -1044,112 +1041,90 @@ private void prepareChangelogs(final Map tasks, } } - private Map computeTimestampSeekFromStreamTime( - final Map partitionsWithRetentionPeriod, - final Set fallbackPartitions) { - if (partitionsWithRetentionPeriod.isEmpty()) { - return Collections.emptyMap(); - } - - final Map result = new HashMap<>(); - try { - // Save pause state so we can restore it after reading the last records + private void seekByRetentionOrBeginning(final Map partitionsWithRetentionPeriod, + final Set partitionsWithoutStartOffset) { + if (!partitionsWithRetentionPeriod.isEmpty()) { final Set allAssigned = restoreConsumer.assignment(); final Set previouslyPaused = new HashSet<>(restoreConsumer.paused()); - // Pause all partitions, then resume only the target partitions - restoreConsumer.pause(allAssigned); - final Set targetPartitions = partitionsWithRetentionPeriod.keySet(); - restoreConsumer.resume(targetPartitions); - - // Seek each target partition to near the end to read the last records - for (final TopicPartition partition : targetPartitions) { - final ChangelogMetadata changelogMeta = changelogs.get(partition); - final long endOffset = changelogMeta.restoreEndOffset; - if (endOffset > 0) { - restoreConsumer.seek(partition, Math.max(0, endOffset - 10)); - } else { - fallbackPartitions.add(partition); - } - } - - // Poll to get the last records — these are actual data records (consumer filters out - // transaction markers), so their timestamps reflect stream time, not wall-clock time - final ConsumerRecords records = restoreConsumer.poll(pollTime); - - // Find the max timestamp per partition from actual data records - final Map maxTimestamps = new HashMap<>(); - for (final TopicPartition partition : targetPartitions) { - final ChangelogMetadata changelogMeta = changelogs.get(partition); - for (final ConsumerRecord record : records.records(partition)) { - if (record.offset() < changelogMeta.restoreEndOffset) { - maxTimestamps.merge(partition, record.timestamp(), Math::max); + try { + // pause all partitions, then resume only the target partitions so that + // poll() only fetches from the changelog partitions we need to inspect + restoreConsumer.pause(allAssigned); + final Set targetPartitions = partitionsWithRetentionPeriod.keySet(); + restoreConsumer.resume(targetPartitions); + + for (final TopicPartition partition : targetPartitions) { + final long endOffset = changelogs.get(partition).restoreEndOffset; + if (endOffset > 0) { + restoreConsumer.seek(partition, endOffset - 1); + } else { + partitionsWithoutStartOffset.add(partition); } } - } - // Compute seek timestamps based on actual stream-time timestamps - for (final Map.Entry entry : partitionsWithRetentionPeriod.entrySet()) { - final TopicPartition partition = entry.getKey(); - final long retentionPeriod = entry.getValue(); - final Long maxTimestamp = maxTimestamps.get(partition); - - if (maxTimestamp != null && maxTimestamp > 0) { - final long seekTimestamp = maxTimestamp - retentionPeriod; - if (seekTimestamp > 0) { - result.put(partition, seekTimestamp); - log.debug("Start restoring windowed changelog partition {} from stream-time-based timestamp {} " + - "(maxStreamTime={}, retention={}).", partition, seekTimestamp, maxTimestamp, retentionPeriod); - } else { - log.debug("Start restoring changelog partition {} from the beginning (computed seek timestamp " + - "is non-positive).", partition); - fallbackPartitions.add(partition); + // poll to get the last records; the consumer filters out transaction control records + // (commit/abort markers), so the timestamps here reflect actual stream time + final ConsumerRecords polledRecords = restoreConsumer.poll(pollTime); + + final Map seekTimestamps = new HashMap<>(); + for (final Map.Entry entry : partitionsWithRetentionPeriod.entrySet()) { + final TopicPartition partition = entry.getKey(); + final long retentionPeriod = entry.getValue(); + final long endOffset = changelogs.get(partition).restoreEndOffset; + + long maxTimestamp = -1; + for (final ConsumerRecord record : polledRecords.records(partition)) { + if (record.offset() < endOffset && record.timestamp() > maxTimestamp) { + maxTimestamp = record.timestamp(); + } } - } else { - log.debug("Start restoring changelog partition {} from the beginning (no records found near end).", - partition); - fallbackPartitions.add(partition); - } - } - // Restore the original pause state - restoreConsumer.pause(allAssigned); - final Set toResume = new HashSet<>(allAssigned); - toResume.removeAll(previouslyPaused); - if (!toResume.isEmpty()) { - restoreConsumer.resume(toResume); - } - } catch (final TimeoutException e) { - log.debug("Could not read last records from changelog for {}, falling back to seek-to-beginning", - partitionsWithRetentionPeriod.keySet(), e); - fallbackPartitions.addAll(partitionsWithRetentionPeriod.keySet()); - } catch (final KafkaException e) { - log.warn("Failed to read last records from changelog for {}, falling back to seek-to-beginning", - partitionsWithRetentionPeriod.keySet(), e); - fallbackPartitions.addAll(partitionsWithRetentionPeriod.keySet()); - } + if (maxTimestamp > 0) { + final long seekTimestamp = maxTimestamp - retentionPeriod; + if (seekTimestamp > 0) { + seekTimestamps.put(partition, seekTimestamp); + log.debug("Start restoring windowed changelog partition {} from stream-time-based timestamp {} " + + "(maxStreamTime={}, retention={}).", partition, seekTimestamp, maxTimestamp, retentionPeriod); + continue; + } + } - return result; - } + log.debug("Start restoring changelog partition {} from the beginning.", partition); + partitionsWithoutStartOffset.add(partition); + } - private void seekToTimestampOrBeginning(final Map partitionsWithTimestampSeek, - final Set partitionsWithoutStartOffset) { - // optimization: seek windowed stores by timestamp to skip expired data - if (!partitionsWithTimestampSeek.isEmpty()) { - final Map offsetsByTimestamp = - restoreConsumer.offsetsForTimes(partitionsWithTimestampSeek); - offsetsByTimestamp.forEach((key, value) -> { - if (value != null) { - restoreConsumer.seek(key, value.offset()); - } else { - // no offset found for the timestamp, fall back to seeking to the beginning - partitionsWithoutStartOffset.add(key); + // translate seek timestamps to offsets and seek + if (!seekTimestamps.isEmpty()) { + final Map offsetsByTimestamp = + restoreConsumer.offsetsForTimes(seekTimestamps); + offsetsByTimestamp.forEach((partition, offsetAndTimestamp) -> { + if (offsetAndTimestamp != null) { + restoreConsumer.seek(partition, offsetAndTimestamp.offset()); + } else { + partitionsWithoutStartOffset.add(partition); + } + }); + } + } catch (final TimeoutException e) { + log.debug("Could not read last records from changelog for {}, falling back to seek-to-beginning", + partitionsWithRetentionPeriod.keySet(), e); + partitionsWithoutStartOffset.addAll(partitionsWithRetentionPeriod.keySet()); + } catch (final KafkaException e) { + log.warn("Failed to read last records from changelog for {}, falling back to seek-to-beginning", + partitionsWithRetentionPeriod.keySet(), e); + partitionsWithoutStartOffset.addAll(partitionsWithRetentionPeriod.keySet()); + } finally { + // restore the original pause state regardless of success or failure + restoreConsumer.pause(allAssigned); + final Set toResume = new HashSet<>(allAssigned); + toResume.removeAll(previouslyPaused); + if (!toResume.isEmpty()) { + restoreConsumer.resume(toResume); } - }); + } } - // optimization: batch all seek-to-beginning offsets in a single request - // seek is not a blocking call so there's nothing to capture if (!partitionsWithoutStartOffset.isEmpty()) { restoreConsumer.seekToBeginning(partitionsWithoutStartOffset); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java index 78b161a90f57d..dd3f083aa10ea 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java @@ -597,4 +597,4 @@ private void getNextIterators() { } } -} \ No newline at end of file +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WithRetentionPeriod.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WithRetentionPeriod.java index 0ffa460e8b151..44dee28eafaac 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/WithRetentionPeriod.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WithRetentionPeriod.java @@ -18,4 +18,4 @@ public interface WithRetentionPeriod { long retentionPeriod(); -} \ No newline at end of file +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java index caa1fd55ecc8c..219a8e8fedfa0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java @@ -1472,7 +1472,7 @@ public synchronized Map offsetsForTimes(fina // Temporarily assign, add a record near the end with an explicit stream-time // timestamp, then clear the assignment (the reader will re-assign during restore) timestampConsumer.assign(Collections.singletonList(tp)); - timestampConsumer.addRecord(new ConsumerRecord<>(topicName, 0, 95, time.milliseconds(), + timestampConsumer.addRecord(new ConsumerRecord<>(topicName, 0, 99, time.milliseconds(), TimestampType.CREATE_TIME, 0, 0, new byte[0], new byte[0], new RecordHeaders(), Optional.empty())); timestampConsumer.assign(Collections.emptyList()); @@ -1520,7 +1520,7 @@ public synchronized Map offsetsForTimes(fina // Temporarily assign, add a record near the end with an explicit timestamp, // then clear the assignment timestampConsumer.assign(Collections.singletonList(tp)); - timestampConsumer.addRecord(new ConsumerRecord<>(topicName, 0, 95, time.milliseconds(), + timestampConsumer.addRecord(new ConsumerRecord<>(topicName, 0, 99, time.milliseconds(), TimestampType.CREATE_TIME, 0, 0, new byte[0], new byte[0], new RecordHeaders(), Optional.empty())); timestampConsumer.assign(Collections.emptyList()); From 55535e357d060b6d0dfe7581a421a3895351838c Mon Sep 17 00:00:00 2001 From: Gabriella Fu <2234862823@qq.com> Date: Wed, 22 Apr 2026 00:15:38 -0400 Subject: [PATCH 13/23] one space --- .../java/org/apache/kafka/clients/admin/MockAdminClient.java | 1 - 1 file changed, 1 deletion(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java index 2d43b135dfb23..bb460855df01a 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java @@ -1491,7 +1491,6 @@ public synchronized void updateEndOffsets(final Map newOff endOffsets.putAll(newOffsets); } - public synchronized void updateConsumerGroupOffsets(final Map newOffsets) { committedOffsets.putAll(newOffsets); } From 045b0f25cc2973d2b7d3f58132cd864d17bbe933 Mon Sep 17 00:00:00 2001 From: Gabriella Fu <2234862823@qq.com> Date: Wed, 22 Apr 2026 00:22:16 -0400 Subject: [PATCH 14/23] simply the way of getting the max_timestamp --- .../internals/StoreChangelogReader.java | 16 +++++----------- 1 file changed, 5 insertions(+), 11 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java index 56df22520173e..3cde57a5a8dbb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java @@ -1071,21 +1071,15 @@ private void seekByRetentionOrBeginning(final Map partitio for (final Map.Entry entry : partitionsWithRetentionPeriod.entrySet()) { final TopicPartition partition = entry.getKey(); final long retentionPeriod = entry.getValue(); - final long endOffset = changelogs.get(partition).restoreEndOffset; - - long maxTimestamp = -1; - for (final ConsumerRecord record : polledRecords.records(partition)) { - if (record.offset() < endOffset && record.timestamp() > maxTimestamp) { - maxTimestamp = record.timestamp(); - } - } - if (maxTimestamp > 0) { - final long seekTimestamp = maxTimestamp - retentionPeriod; + final List> records = polledRecords.records(partition); + if (!records.isEmpty()) { + final long latestTimestamp = records.get(records.size() - 1).timestamp(); + final long seekTimestamp = latestTimestamp - retentionPeriod; if (seekTimestamp > 0) { seekTimestamps.put(partition, seekTimestamp); log.debug("Start restoring windowed changelog partition {} from stream-time-based timestamp {} " + - "(maxStreamTime={}, retention={}).", partition, seekTimestamp, maxTimestamp, retentionPeriod); + "(maxStreamTime={}, retention={}).", partition, seekTimestamp, latestTimestamp, retentionPeriod); continue; } } From c0bf24e5a776f8f52b0e279808034f1dc6670dd4 Mon Sep 17 00:00:00 2001 From: Gabriella Fu <2234862823@qq.com> Date: Wed, 22 Apr 2026 12:40:12 -0400 Subject: [PATCH 15/23] format --- .../processor/internals/StoreChangelogReaderTest.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java index 219a8e8fedfa0..f8d8cf8f7fe12 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java @@ -33,6 +33,8 @@ import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.header.internals.RecordHeaders; +import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.LogCaptureAppender; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; @@ -67,9 +69,6 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.kafka.common.header.internals.RecordHeaders; -import org.apache.kafka.common.record.TimestampType; - import static java.util.Collections.singletonMap; import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; From 5dc78fa086bde16878eaa4822b27ef29e9f99aa9 Mon Sep 17 00:00:00 2001 From: Gabriella Fu <2234862823@qq.com> Date: Wed, 29 Apr 2026 10:45:11 -0400 Subject: [PATCH 16/23] fix new standby task npe --- .../streams/processor/internals/StoreChangelogReader.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java index 3cde57a5a8dbb..684031eaec4a0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java @@ -1055,8 +1055,8 @@ private void seekByRetentionOrBeginning(final Map partitio restoreConsumer.resume(targetPartitions); for (final TopicPartition partition : targetPartitions) { - final long endOffset = changelogs.get(partition).restoreEndOffset; - if (endOffset > 0) { + final Long endOffset = changelogs.get(partition).restoreEndOffset; + if (endOffset != null && endOffset > 0) { restoreConsumer.seek(partition, endOffset - 1); } else { partitionsWithoutStartOffset.add(partition); From 3c1f8d175e67c6b0ac473af4e8ec6f7f38d29a81 Mon Sep 17 00:00:00 2001 From: Gabriella Fu <2234862823@qq.com> Date: Fri, 1 May 2026 12:34:26 -0400 Subject: [PATCH 17/23] filter out standby task --- .../kafka/streams/processor/internals/StoreChangelogReader.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java index 684031eaec4a0..8a64f29a360aa 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java @@ -989,7 +989,7 @@ private void prepareChangelogs(final Map tasks, partition, currentOffset, recordEndOffset(endOffset)); } else { final long retentionPeriod = storeMetadata.retentionPeriod(); - if (retentionPeriod > 0 && retentionPeriod != Long.MAX_VALUE) { + if (retentionPeriod > 0 && retentionPeriod != Long.MAX_VALUE && endOffset != null && endOffset > 0) { newPartitionsWithRetentionPeriod.put(partition, retentionPeriod); } else { log.debug("Start restoring changelog partition {} from the beginning offset to end offset {} " + From 519a2d410c8f1ad49d760ae3a8985f70a2aa0c58 Mon Sep 17 00:00:00 2001 From: Gabriella Fu <2234862823@qq.com> Date: Fri, 1 May 2026 16:55:27 -0400 Subject: [PATCH 18/23] restoreConsumer.endOffsets() + poll for standby --- .../internals/StoreChangelogReader.java | 95 ++++++++++++------- .../internals/StoreChangelogReaderTest.java | 41 ++++---- 2 files changed, 83 insertions(+), 53 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java index 8a64f29a360aa..e09a33f5e237a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java @@ -989,7 +989,7 @@ private void prepareChangelogs(final Map tasks, partition, currentOffset, recordEndOffset(endOffset)); } else { final long retentionPeriod = storeMetadata.retentionPeriod(); - if (retentionPeriod > 0 && retentionPeriod != Long.MAX_VALUE && endOffset != null && endOffset > 0) { + if (retentionPeriod > 0 && retentionPeriod != Long.MAX_VALUE) { newPartitionsWithRetentionPeriod.put(partition, retentionPeriod); } else { log.debug("Start restoring changelog partition {} from the beginning offset to end offset {} " + @@ -1041,6 +1041,62 @@ private void prepareChangelogs(final Map tasks, } } + private Map resolveEndOffsets(final Set partitions) { + final Map endOffsets = new HashMap<>(); + final Set needEndOffset = new HashSet<>(); + + for (final TopicPartition partition : partitions) { + final Long restoreEndOffset = changelogs.get(partition).restoreEndOffset; + if (restoreEndOffset != null && restoreEndOffset > 0) { + endOffsets.put(partition, restoreEndOffset); + } else { + needEndOffset.add(partition); + } + } + + if (!needEndOffset.isEmpty()) { + final Map fetchedEndOffsets = restoreConsumer.endOffsets(needEndOffset); + for (final Map.Entry entry : fetchedEndOffsets.entrySet()) { + if (entry.getValue() != null && entry.getValue() > 0) { + endOffsets.put(entry.getKey(), entry.getValue()); + } + } + } + + return endOffsets; + } + + private Map computeSeekTimestamps(final Map partitionsWithRetentionPeriod, + final Map endOffsets, + final Set partitionsWithoutStartOffset) { + for (final Map.Entry entry : endOffsets.entrySet()) { + restoreConsumer.seek(entry.getKey(), entry.getValue() - 1); + } + + final ConsumerRecords polledRecords = restoreConsumer.poll(pollTime); + + final Map seekTimestamps = new HashMap<>(); + for (final TopicPartition partition : endOffsets.keySet()) { + final long retentionPeriod = partitionsWithRetentionPeriod.get(partition); + final List> records = polledRecords.records(partition); + if (!records.isEmpty()) { + final long latestTimestamp = records.get(records.size() - 1).timestamp(); + final long seekTimestamp = latestTimestamp - retentionPeriod; + if (seekTimestamp > 0) { + seekTimestamps.put(partition, seekTimestamp); + log.debug("Start restoring windowed changelog partition {} from stream-time-based timestamp {} " + + "(maxStreamTime={}, retention={}).", partition, seekTimestamp, latestTimestamp, retentionPeriod); + continue; + } + } + + log.debug("Start restoring changelog partition {} from the beginning.", partition); + partitionsWithoutStartOffset.add(partition); + } + + return seekTimestamps; + } + private void seekByRetentionOrBeginning(final Map partitionsWithRetentionPeriod, final Set partitionsWithoutStartOffset) { if (!partitionsWithRetentionPeriod.isEmpty()) { @@ -1048,47 +1104,21 @@ private void seekByRetentionOrBeginning(final Map partitio final Set previouslyPaused = new HashSet<>(restoreConsumer.paused()); try { - // pause all partitions, then resume only the target partitions so that - // poll() only fetches from the changelog partitions we need to inspect restoreConsumer.pause(allAssigned); final Set targetPartitions = partitionsWithRetentionPeriod.keySet(); restoreConsumer.resume(targetPartitions); + final Map endOffsets = resolveEndOffsets(targetPartitions); + for (final TopicPartition partition : targetPartitions) { - final Long endOffset = changelogs.get(partition).restoreEndOffset; - if (endOffset != null && endOffset > 0) { - restoreConsumer.seek(partition, endOffset - 1); - } else { + if (!endOffsets.containsKey(partition)) { partitionsWithoutStartOffset.add(partition); } } - // poll to get the last records; the consumer filters out transaction control records - // (commit/abort markers), so the timestamps here reflect actual stream time - final ConsumerRecords polledRecords = restoreConsumer.poll(pollTime); - - final Map seekTimestamps = new HashMap<>(); - for (final Map.Entry entry : partitionsWithRetentionPeriod.entrySet()) { - final TopicPartition partition = entry.getKey(); - final long retentionPeriod = entry.getValue(); - - final List> records = polledRecords.records(partition); - if (!records.isEmpty()) { - final long latestTimestamp = records.get(records.size() - 1).timestamp(); - final long seekTimestamp = latestTimestamp - retentionPeriod; - if (seekTimestamp > 0) { - seekTimestamps.put(partition, seekTimestamp); - log.debug("Start restoring windowed changelog partition {} from stream-time-based timestamp {} " + - "(maxStreamTime={}, retention={}).", partition, seekTimestamp, latestTimestamp, retentionPeriod); - continue; - } - } - - log.debug("Start restoring changelog partition {} from the beginning.", partition); - partitionsWithoutStartOffset.add(partition); - } + final Map seekTimestamps = + computeSeekTimestamps(partitionsWithRetentionPeriod, endOffsets, partitionsWithoutStartOffset); - // translate seek timestamps to offsets and seek if (!seekTimestamps.isEmpty()) { final Map offsetsByTimestamp = restoreConsumer.offsetsForTimes(seekTimestamps); @@ -1109,7 +1139,6 @@ private void seekByRetentionOrBeginning(final Map partitio partitionsWithRetentionPeriod.keySet(), e); partitionsWithoutStartOffset.addAll(partitionsWithRetentionPeriod.keySet()); } finally { - // restore the original pause state regardless of success or failure restoreConsumer.pause(allAssigned); final Set toResume = new HashSet<>(allAssigned); toResume.removeAll(previouslyPaused); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java index f8d8cf8f7fe12..e3276cefefbac 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java @@ -1439,8 +1439,9 @@ private void addRecords(final long messages, final TopicPartition topicPartition public void shouldSeekByTimestampForWindowedStoreWithoutCheckpoint() { final long retentionMs = Duration.ofHours(2).toMillis(); final long offsetForTimestamp = 42L; + final long latestRecordTimestamp = 10_000_000L; + final long endOffset = 100L; - // Use a MockConsumer subclass that supports offsetsForTimes final MockConsumer timestampConsumer = new MockConsumer<>(AutoOffsetResetStrategy.EARLIEST.name()) { @Override public synchronized Map offsetsForTimes(final Map timestampsToSearch) { @@ -1450,7 +1451,6 @@ public synchronized Map offsetsForTimes(fina } }; - // Set up mocks - storeMetadata returns null offset (no checkpoint) and positive retentionPeriod final StateStoreMetadata windowStoreMetadata = mock(StateStoreMetadata.class); final ProcessorStateManager windowStateManager = mock(ProcessorStateManager.class); final StateStore windowStore = mock(StateStore.class); @@ -1466,15 +1466,15 @@ public synchronized Map offsetsForTimes(fina when(windowStateManager.taskId()).thenReturn(taskId); timestampConsumer.updateBeginningOffsets(Collections.singletonMap(tp, 0L)); - adminClient.updateEndOffsets(Collections.singletonMap(tp, 100L)); + timestampConsumer.updateEndOffsets(Collections.singletonMap(tp, endOffset)); + adminClient.updateEndOffsets(Collections.singletonMap(tp, endOffset)); - // Temporarily assign, add a record near the end with an explicit stream-time - // timestamp, then clear the assignment (the reader will re-assign during restore) - timestampConsumer.assign(Collections.singletonList(tp)); - timestampConsumer.addRecord(new ConsumerRecord<>(topicName, 0, 99, time.milliseconds(), - TimestampType.CREATE_TIME, 0, 0, new byte[0], new byte[0], - new RecordHeaders(), Optional.empty())); - timestampConsumer.assign(Collections.emptyList()); + // schedule adding the record during poll, after the partition is assigned + timestampConsumer.schedulePollTask(() -> timestampConsumer.addRecord(new ConsumerRecord<>( + tp.topic(), tp.partition(), endOffset - 1, + latestRecordTimestamp, TimestampType.CREATE_TIME, + 0, 0, new byte[0], new byte[0], + new RecordHeaders(), Optional.empty()))); final StoreChangelogReader reader = new StoreChangelogReader(time, config, logContext, adminClient, timestampConsumer, callback, standbyListener); @@ -1488,8 +1488,9 @@ public synchronized Map offsetsForTimes(fina @Test public void shouldSeekToBeginningWhenBrokerReturnsNullForOffsetsForTimes() { final long retentionMs = Duration.ofHours(2).toMillis(); + final long latestRecordTimestamp = 10_000_000L; + final long endOffset = 100L; - // Use a MockConsumer subclass that returns null for offsetsForTimes final MockConsumer timestampConsumer = new MockConsumer<>(AutoOffsetResetStrategy.EARLIEST.name()) { @Override public synchronized Map offsetsForTimes(final Map timestampsToSearch) { @@ -1514,15 +1515,15 @@ public synchronized Map offsetsForTimes(fina when(windowStateManager.taskId()).thenReturn(taskId); timestampConsumer.updateBeginningOffsets(Collections.singletonMap(tp, 0L)); - adminClient.updateEndOffsets(Collections.singletonMap(tp, 100L)); - - // Temporarily assign, add a record near the end with an explicit timestamp, - // then clear the assignment - timestampConsumer.assign(Collections.singletonList(tp)); - timestampConsumer.addRecord(new ConsumerRecord<>(topicName, 0, 99, time.milliseconds(), - TimestampType.CREATE_TIME, 0, 0, new byte[0], new byte[0], - new RecordHeaders(), Optional.empty())); - timestampConsumer.assign(Collections.emptyList()); + timestampConsumer.updateEndOffsets(Collections.singletonMap(tp, endOffset)); + adminClient.updateEndOffsets(Collections.singletonMap(tp, endOffset)); + + // schedule adding the record during poll, after the partition is assigned + timestampConsumer.schedulePollTask(() -> timestampConsumer.addRecord(new ConsumerRecord<>( + tp.topic(), tp.partition(), endOffset - 1, + latestRecordTimestamp, TimestampType.CREATE_TIME, + 0, 0, new byte[0], new byte[0], + new RecordHeaders(), Optional.empty()))); final StoreChangelogReader reader = new StoreChangelogReader(time, config, logContext, adminClient, timestampConsumer, callback, standbyListener); From 02a61e6216929076b26ee1f36530d1dfa51da230 Mon Sep 17 00:00:00 2001 From: Gabriella Fu <2234862823@qq.com> Date: Fri, 1 May 2026 19:29:08 -0400 Subject: [PATCH 19/23] use seek to end to get the last offset --- .../internals/StoreChangelogReader.java | 59 ++++++------------- 1 file changed, 18 insertions(+), 41 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java index e09a33f5e237a..26688eeea71dd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java @@ -1041,46 +1041,32 @@ private void prepareChangelogs(final Map tasks, } } - private Map resolveEndOffsets(final Set partitions) { - final Map endOffsets = new HashMap<>(); - final Set needEndOffset = new HashSet<>(); + private Map pollLatestTimestamps(final Map partitionsWithRetentionPeriod, + final Set partitionsWithoutStartOffset) { + final Set targetPartitions = partitionsWithRetentionPeriod.keySet(); + restoreConsumer.seekToEnd(targetPartitions); - for (final TopicPartition partition : partitions) { - final Long restoreEndOffset = changelogs.get(partition).restoreEndOffset; - if (restoreEndOffset != null && restoreEndOffset > 0) { - endOffsets.put(partition, restoreEndOffset); + for (final TopicPartition partition : targetPartitions) { + final long endPosition = restoreConsumer.position(partition); + if (endPosition > 0) { + restoreConsumer.seek(partition, endPosition - 1); } else { - needEndOffset.add(partition); - } - } - - if (!needEndOffset.isEmpty()) { - final Map fetchedEndOffsets = restoreConsumer.endOffsets(needEndOffset); - for (final Map.Entry entry : fetchedEndOffsets.entrySet()) { - if (entry.getValue() != null && entry.getValue() > 0) { - endOffsets.put(entry.getKey(), entry.getValue()); - } + partitionsWithoutStartOffset.add(partition); } } - return endOffsets; - } - - private Map computeSeekTimestamps(final Map partitionsWithRetentionPeriod, - final Map endOffsets, - final Set partitionsWithoutStartOffset) { - for (final Map.Entry entry : endOffsets.entrySet()) { - restoreConsumer.seek(entry.getKey(), entry.getValue() - 1); - } - final ConsumerRecords polledRecords = restoreConsumer.poll(pollTime); final Map seekTimestamps = new HashMap<>(); - for (final TopicPartition partition : endOffsets.keySet()) { - final long retentionPeriod = partitionsWithRetentionPeriod.get(partition); + for (final Map.Entry entry : partitionsWithRetentionPeriod.entrySet()) { + final TopicPartition partition = entry.getKey(); + if (partitionsWithoutStartOffset.contains(partition)) { + continue; + } + final long retentionPeriod = entry.getValue(); final List> records = polledRecords.records(partition); if (!records.isEmpty()) { - final long latestTimestamp = records.get(records.size() - 1).timestamp(); + final long latestTimestamp = records.get(0).timestamp(); final long seekTimestamp = latestTimestamp - retentionPeriod; if (seekTimestamp > 0) { seekTimestamps.put(partition, seekTimestamp); @@ -1105,19 +1091,10 @@ private void seekByRetentionOrBeginning(final Map partitio try { restoreConsumer.pause(allAssigned); - final Set targetPartitions = partitionsWithRetentionPeriod.keySet(); - restoreConsumer.resume(targetPartitions); - - final Map endOffsets = resolveEndOffsets(targetPartitions); - - for (final TopicPartition partition : targetPartitions) { - if (!endOffsets.containsKey(partition)) { - partitionsWithoutStartOffset.add(partition); - } - } + restoreConsumer.resume(partitionsWithRetentionPeriod.keySet()); final Map seekTimestamps = - computeSeekTimestamps(partitionsWithRetentionPeriod, endOffsets, partitionsWithoutStartOffset); + pollLatestTimestamps(partitionsWithRetentionPeriod, partitionsWithoutStartOffset); if (!seekTimestamps.isEmpty()) { final Map offsetsByTimestamp = From 1e8d5c4efeac5bcef1308c452e57e0f9449c4a90 Mon Sep 17 00:00:00 2001 From: Gabriella Fu <2234862823@qq.com> Date: Fri, 1 May 2026 20:45:16 -0400 Subject: [PATCH 20/23] use restoreConsumer.endOffsets(partitions) --- .../streams/processor/internals/StoreChangelogReader.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java index 26688eeea71dd..0cf640bd4f7b9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java @@ -1044,12 +1044,12 @@ private void prepareChangelogs(final Map tasks, private Map pollLatestTimestamps(final Map partitionsWithRetentionPeriod, final Set partitionsWithoutStartOffset) { final Set targetPartitions = partitionsWithRetentionPeriod.keySet(); - restoreConsumer.seekToEnd(targetPartitions); + final Map endOffsets = restoreConsumer.endOffsets(targetPartitions); for (final TopicPartition partition : targetPartitions) { - final long endPosition = restoreConsumer.position(partition); - if (endPosition > 0) { - restoreConsumer.seek(partition, endPosition - 1); + final Long endOffset = endOffsets.get(partition); + if (endOffset != null && endOffset > 0) { + restoreConsumer.seek(partition, endOffset - 1); } else { partitionsWithoutStartOffset.add(partition); } From 2ba2da2ef4fb6e3bc90fe55dd41f814a6e74d61b Mon Sep 17 00:00:00 2001 From: Gabriella Fu <2234862823@qq.com> Date: Fri, 1 May 2026 21:22:33 -0400 Subject: [PATCH 21/23] add seek to beginning --- .../streams/processor/internals/StoreChangelogReader.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java index 0cf640bd4f7b9..54b7f0516b095 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java @@ -1055,6 +1055,10 @@ private Map pollLatestTimestamps(final Map polledRecords = restoreConsumer.poll(pollTime); final Map seekTimestamps = new HashMap<>(); From 0b0a8667c1cf601e538f829e2fd88cba3fd29e0b Mon Sep 17 00:00:00 2001 From: Gabriella Fu <2234862823@qq.com> Date: Fri, 1 May 2026 22:12:05 -0400 Subject: [PATCH 22/23] change the name to make it less confusing --- .../internals/StoreChangelogReader.java | 145 +++++++++--------- 1 file changed, 76 insertions(+), 69 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java index 54b7f0516b095..abc2d686d2288 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java @@ -970,8 +970,8 @@ private void resumeChangelogsFromRestoreConsumer(final Collection tasks, final Set newPartitionsToRestore) { // separate those who do not have the current offset loaded from checkpoint - final Set newPartitionsWithoutStartOffset = new HashSet<>(); - final Map newPartitionsWithRetentionPeriod = new HashMap<>(); + final Set newSeekToBeginningPartitions = new HashSet<>(); + final Map newWindowedPartitionsRetention = new HashMap<>(); for (final ChangelogMetadata changelogMetadata : newPartitionsToRestore) { final StateStoreMetadata storeMetadata = changelogMetadata.storeMetadata; @@ -990,16 +990,16 @@ private void prepareChangelogs(final Map tasks, } else { final long retentionPeriod = storeMetadata.retentionPeriod(); if (retentionPeriod > 0 && retentionPeriod != Long.MAX_VALUE) { - newPartitionsWithRetentionPeriod.put(partition, retentionPeriod); + newWindowedPartitionsRetention.put(partition, retentionPeriod); } else { log.debug("Start restoring changelog partition {} from the beginning offset to end offset {} " + "since we cannot find current offset.", partition, recordEndOffset(endOffset)); - newPartitionsWithoutStartOffset.add(partition); + newSeekToBeginningPartitions.add(partition); } } } - seekByRetentionOrBeginning(newPartitionsWithRetentionPeriod, newPartitionsWithoutStartOffset); + seekNewPartitions(newWindowedPartitionsRetention, newSeekToBeginningPartitions); for (final ChangelogMetadata changelogMetadata : newPartitionsToRestore) { final StateStoreMetadata storeMetadata = changelogMetadata.storeMetadata; @@ -1041,30 +1041,75 @@ private void prepareChangelogs(final Map tasks, } } - private Map pollLatestTimestamps(final Map partitionsWithRetentionPeriod, - final Set partitionsWithoutStartOffset) { - final Set targetPartitions = partitionsWithRetentionPeriod.keySet(); - final Map endOffsets = restoreConsumer.endOffsets(targetPartitions); + private void seekNewPartitions(final Map windowedPartitionsRetention, + final Set seekToBeginningPartitions) { + // Seek non-windowed partitions to beginning. + if (!seekToBeginningPartitions.isEmpty()) { + restoreConsumer.seekToBeginning(seekToBeginningPartitions); + } - for (final TopicPartition partition : targetPartitions) { - final Long endOffset = endOffsets.get(partition); - if (endOffset != null && endOffset > 0) { - restoreConsumer.seek(partition, endOffset - 1); - } else { - partitionsWithoutStartOffset.add(partition); + // Try to optimize windowed partitions by seeking past expired data. + if (!windowedPartitionsRetention.isEmpty()) { + final Set allAssigned = restoreConsumer.assignment(); + final Set previouslyPaused = new HashSet<>(restoreConsumer.paused()); + + try { + restoreConsumer.pause(allAssigned); + restoreConsumer.resume(windowedPartitionsRetention.keySet()); + + // Give windowed partitions a fallback position. The restore consumer uses + // auto.offset.reset=none, so poll() throws NoOffsetForPartitionException for + // any assigned partition without a position — even paused ones. + // seek(endOffset-1) below overrides this for non-empty partitions. + restoreConsumer.seekToBeginning(windowedPartitionsRetention.keySet()); + + final Map endOffsets = + restoreConsumer.endOffsets(windowedPartitionsRetention.keySet()); + + for (final TopicPartition partition : windowedPartitionsRetention.keySet()) { + final Long endOffset = endOffsets.get(partition); + if (endOffset != null && endOffset > 0) { + restoreConsumer.seek(partition, endOffset - 1); + } else { + seekToBeginningPartitions.add(partition); + } + } + + final ConsumerRecords polledRecords = restoreConsumer.poll(pollTime); + + seekByRetentionFromPolledRecords(polledRecords, windowedPartitionsRetention, seekToBeginningPartitions); + } catch (final TimeoutException e) { + log.debug("Could not seek by timestamp for changelog partitions {}, falling back to seek-to-beginning", + windowedPartitionsRetention.keySet(), e); + seekToBeginningPartitions.addAll(windowedPartitionsRetention.keySet()); + } catch (final KafkaException e) { + log.warn("Failed to seek by timestamp for changelog partitions {}, falling back to seek-to-beginning", + windowedPartitionsRetention.keySet(), e); + seekToBeginningPartitions.addAll(windowedPartitionsRetention.keySet()); + } finally { + restoreConsumer.pause(allAssigned); + final Set toResume = new HashSet<>(allAssigned); + toResume.removeAll(previouslyPaused); + if (!toResume.isEmpty()) { + restoreConsumer.resume(toResume); + } } } - if (!partitionsWithoutStartOffset.isEmpty()) { - restoreConsumer.seekToBeginning(partitionsWithoutStartOffset); + // Seek any windowed partitions that failed during the optimization back to the beginning. + // Their position was moved by seek+poll above. + if (!seekToBeginningPartitions.isEmpty()) { + restoreConsumer.seekToBeginning(seekToBeginningPartitions); } + } - final ConsumerRecords polledRecords = restoreConsumer.poll(pollTime); - + private void seekByRetentionFromPolledRecords(final ConsumerRecords polledRecords, + final Map windowedPartitionsRetention, + final Set seekToBeginningPartitions) { final Map seekTimestamps = new HashMap<>(); - for (final Map.Entry entry : partitionsWithRetentionPeriod.entrySet()) { + for (final Map.Entry entry : windowedPartitionsRetention.entrySet()) { final TopicPartition partition = entry.getKey(); - if (partitionsWithoutStartOffset.contains(partition)) { + if (seekToBeginningPartitions.contains(partition)) { continue; } final long retentionPeriod = entry.getValue(); @@ -1079,58 +1124,20 @@ private Map pollLatestTimestamps(final Map partitionsWithRetentionPeriod, - final Set partitionsWithoutStartOffset) { - if (!partitionsWithRetentionPeriod.isEmpty()) { - final Set allAssigned = restoreConsumer.assignment(); - final Set previouslyPaused = new HashSet<>(restoreConsumer.paused()); - - try { - restoreConsumer.pause(allAssigned); - restoreConsumer.resume(partitionsWithRetentionPeriod.keySet()); - - final Map seekTimestamps = - pollLatestTimestamps(partitionsWithRetentionPeriod, partitionsWithoutStartOffset); - - if (!seekTimestamps.isEmpty()) { - final Map offsetsByTimestamp = - restoreConsumer.offsetsForTimes(seekTimestamps); - offsetsByTimestamp.forEach((partition, offsetAndTimestamp) -> { - if (offsetAndTimestamp != null) { - restoreConsumer.seek(partition, offsetAndTimestamp.offset()); - } else { - partitionsWithoutStartOffset.add(partition); - } - }); - } - } catch (final TimeoutException e) { - log.debug("Could not read last records from changelog for {}, falling back to seek-to-beginning", - partitionsWithRetentionPeriod.keySet(), e); - partitionsWithoutStartOffset.addAll(partitionsWithRetentionPeriod.keySet()); - } catch (final KafkaException e) { - log.warn("Failed to read last records from changelog for {}, falling back to seek-to-beginning", - partitionsWithRetentionPeriod.keySet(), e); - partitionsWithoutStartOffset.addAll(partitionsWithRetentionPeriod.keySet()); - } finally { - restoreConsumer.pause(allAssigned); - final Set toResume = new HashSet<>(allAssigned); - toResume.removeAll(previouslyPaused); - if (!toResume.isEmpty()) { - restoreConsumer.resume(toResume); + if (!seekTimestamps.isEmpty()) { + final Map offsetsByTimestamp = + restoreConsumer.offsetsForTimes(seekTimestamps); + offsetsByTimestamp.forEach((partition, offsetAndTimestamp) -> { + if (offsetAndTimestamp != null) { + restoreConsumer.seek(partition, offsetAndTimestamp.offset()); + } else { + seekToBeginningPartitions.add(partition); } - } - } - - if (!partitionsWithoutStartOffset.isEmpty()) { - restoreConsumer.seekToBeginning(partitionsWithoutStartOffset); + }); } } From c2ec3be913106340fee6e1bac967bbbb92edf67e Mon Sep 17 00:00:00 2001 From: Gabriella Fu <2234862823@qq.com> Date: Fri, 1 May 2026 22:23:06 -0400 Subject: [PATCH 23/23] fix the bug --- .../processor/internals/StoreChangelogReader.java | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java index abc2d686d2288..51287a9df55ea 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java @@ -1057,12 +1057,6 @@ private void seekNewPartitions(final Map windowedPartition restoreConsumer.pause(allAssigned); restoreConsumer.resume(windowedPartitionsRetention.keySet()); - // Give windowed partitions a fallback position. The restore consumer uses - // auto.offset.reset=none, so poll() throws NoOffsetForPartitionException for - // any assigned partition without a position — even paused ones. - // seek(endOffset-1) below overrides this for non-empty partitions. - restoreConsumer.seekToBeginning(windowedPartitionsRetention.keySet()); - final Map endOffsets = restoreConsumer.endOffsets(windowedPartitionsRetention.keySet()); @@ -1071,9 +1065,11 @@ private void seekNewPartitions(final Map windowedPartition if (endOffset != null && endOffset > 0) { restoreConsumer.seek(partition, endOffset - 1); } else { + restoreConsumer.seekToBeginning(Collections.singleton(partition)); seekToBeginningPartitions.add(partition); } } + windowedPartitionsRetention.keySet().removeAll(seekToBeginningPartitions); final ConsumerRecords polledRecords = restoreConsumer.poll(pollTime); @@ -1109,9 +1105,6 @@ private void seekByRetentionFromPolledRecords(final ConsumerRecords seekTimestamps = new HashMap<>(); for (final Map.Entry entry : windowedPartitionsRetention.entrySet()) { final TopicPartition partition = entry.getKey(); - if (seekToBeginningPartitions.contains(partition)) { - continue; - } final long retentionPeriod = entry.getValue(); final List> records = polledRecords.records(partition); if (!records.isEmpty()) {