diff --git a/core/src/main/java/kafka/server/share/DelayedShareFetch.java b/core/src/main/java/kafka/server/share/DelayedShareFetch.java index 2d01fb959cb10..aa619ee91a8ac 100644 --- a/core/src/main/java/kafka/server/share/DelayedShareFetch.java +++ b/core/src/main/java/kafka/server/share/DelayedShareFetch.java @@ -512,11 +512,11 @@ private LogOffsetMetadata endOffsetMetadataForTopicPartition(TopicIdPartition to // extend it to support other FetchIsolation types. FetchIsolation isolationType = shareFetch.fetchParams().isolation; if (isolationType == FetchIsolation.LOG_END) - return offsetSnapshot.logEndOffset; + return offsetSnapshot.logEndOffset(); else if (isolationType == FetchIsolation.HIGH_WATERMARK) - return offsetSnapshot.highWatermark; + return offsetSnapshot.highWatermark(); else - return offsetSnapshot.lastStableOffset; + return offsetSnapshot.lastStableOffset(); } @@ -835,8 +835,8 @@ private void completeRemoteStorageShareFetchRequest() { for (RemoteFetch remoteFetch : pendingRemoteFetchesOpt.get().remoteFetches()) { if (remoteFetch.remoteFetchResult().isDone()) { RemoteLogReadResult remoteLogReadResult = remoteFetch.remoteFetchResult().get(); - if (remoteLogReadResult.error.isPresent()) { - Throwable error = remoteLogReadResult.error.get(); + if (remoteLogReadResult.error().isPresent()) { + Throwable error = remoteLogReadResult.error().get(); // If there is any error for the remote fetch topic partition, we populate the error accordingly. shareFetchPartitionData.add( new ShareFetchPartitionData( @@ -846,7 +846,7 @@ private void completeRemoteStorageShareFetchRequest() { ) ); } else { - FetchDataInfo info = remoteLogReadResult.fetchDataInfo.get(); + FetchDataInfo info = remoteLogReadResult.fetchDataInfo().get(); TopicIdPartition topicIdPartition = remoteFetch.topicIdPartition(); LogReadResult logReadResult = remoteFetch.logReadResult(); shareFetchPartitionData.add( diff --git a/server/src/test/java/org/apache/kafka/server/util/SchedulerTest.java b/server/src/test/java/org/apache/kafka/server/util/SchedulerTest.java index 933a14b77667e..99fda4f9762d0 100644 --- a/server/src/test/java/org/apache/kafka/server/util/SchedulerTest.java +++ b/server/src/test/java/org/apache/kafka/server/util/SchedulerTest.java @@ -177,9 +177,9 @@ void testUnscheduleProducerTask() throws IOException { leaderEpochCache, producerStateManager, new ConcurrentHashMap<>(), false).load(); - LocalLog localLog = new LocalLog(logDir, logConfig, segments, offsets.recoveryPoint, - offsets.nextOffsetMetadata, scheduler, mockTime, topicPartition, logDirFailureChannel); - UnifiedLog log = new UnifiedLog(offsets.logStartOffset, + LocalLog localLog = new LocalLog(logDir, logConfig, segments, offsets.recoveryPoint(), + offsets.nextOffsetMetadata(), scheduler, mockTime, topicPartition, logDirFailureChannel); + UnifiedLog log = new UnifiedLog(offsets.logStartOffset(), localLog, brokerTopicStats, producerIdExpirationCheckIntervalMs, diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/quota/RLMQuotaManagerConfig.java b/storage/src/main/java/org/apache/kafka/server/log/remote/quota/RLMQuotaManagerConfig.java index 0b97b464740e1..4609fc5924bb5 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/quota/RLMQuotaManagerConfig.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/quota/RLMQuotaManagerConfig.java @@ -16,44 +16,13 @@ */ package org.apache.kafka.server.log.remote.quota; -public class RLMQuotaManagerConfig { +/** + * Configuration settings for quota management + * + * @param quotaBytesPerSecond The quota in bytes per second + * @param numQuotaSamples The number of samples to retain in memory + * @param quotaWindowSizeSeconds The time span of each sample + */ +public record RLMQuotaManagerConfig(long quotaBytesPerSecond, int numQuotaSamples, int quotaWindowSizeSeconds) { public static final int INACTIVE_SENSOR_EXPIRATION_TIME_SECONDS = 3600; - - private final long quotaBytesPerSecond; - private final int numQuotaSamples; - private final int quotaWindowSizeSeconds; - - /** - * Configuration settings for quota management - * - * @param quotaBytesPerSecond The quota in bytes per second - * @param numQuotaSamples The number of samples to retain in memory - * @param quotaWindowSizeSeconds The time span of each sample - */ - public RLMQuotaManagerConfig(long quotaBytesPerSecond, int numQuotaSamples, int quotaWindowSizeSeconds) { - this.quotaBytesPerSecond = quotaBytesPerSecond; - this.numQuotaSamples = numQuotaSamples; - this.quotaWindowSizeSeconds = quotaWindowSizeSeconds; - } - - public long quotaBytesPerSecond() { - return quotaBytesPerSecond; - } - - public int numQuotaSamples() { - return numQuotaSamples; - } - - public int quotaWindowSizeSeconds() { - return quotaWindowSizeSeconds; - } - - @Override - public String toString() { - return "RLMQuotaManagerConfig{" + - "quotaBytesPerSecond=" + quotaBytesPerSecond + - ", numQuotaSamples=" + numQuotaSamples + - ", quotaWindowSizeSeconds=" + quotaWindowSizeSeconds + - '}'; - } } diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java index 852e1d4d1fbe2..4b36d2a7edd1f 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java @@ -188,10 +188,11 @@ public class RemoteLogManager implements Closeable, AsyncOffsetReader { private final KafkaMetricsGroup metricsGroup = new KafkaMetricsGroup("kafka.log.remote", "RemoteLogManager"); // The endpoint for remote log metadata manager to connect to - private Optional endpoint = Optional.empty(); + private final Optional endpoint; + private final Timer remoteReadTimer; + private boolean closed = false; - private final Timer remoteReadTimer; private volatile DelayedOperationPurgatory delayedRemoteListOffsetsPurgatory; /** @@ -1010,7 +1011,7 @@ private void copyLogSegment(UnifiedLog log, LogSegment segment, RemoteLogSegment List epochEntries = getLeaderEpochEntries(log, segment.baseOffset(), nextSegmentBaseOffset); Map segmentLeaderEpochs = new HashMap<>(epochEntries.size()); - epochEntries.forEach(entry -> segmentLeaderEpochs.put(entry.epoch, entry.startOffset)); + epochEntries.forEach(entry -> segmentLeaderEpochs.put(entry.epoch(), entry.startOffset())); boolean isTxnIdxEmpty = segment.txnIndex().isEmpty(); RemoteLogSegmentMetadata copySegmentStartedRlsm = new RemoteLogSegmentMetadata(segmentId, segment.baseOffset(), endOffset, @@ -1071,7 +1072,7 @@ private void copyLogSegment(UnifiedLog log, LogSegment segment, RemoteLogSegment // `epochEntries` cannot be empty, there is a pre-condition validation in RemoteLogSegmentMetadata // constructor - int lastEpochInSegment = epochEntries.get(epochEntries.size() - 1).epoch; + int lastEpochInSegment = epochEntries.get(epochEntries.size() - 1).epoch(); copiedOffsetOption = Optional.of(new OffsetAndEpoch(endOffset, lastEpochInSegment)); // Update the highest offset in remote storage for this partition's log so that the local log segments // are not deleted before they are copied to remote storage. @@ -1206,7 +1207,7 @@ private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earl RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException { boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, - ignored -> metadata.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch)); + ignored -> metadata.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch())); if (isSegmentDeleted) { logger.info("Deleted remote log segment {} due to leader-epoch-cache truncation. " + "Current earliest-epoch-entry: {}, segment-end-offset: {} and segment-epochs: {}", @@ -1375,7 +1376,7 @@ void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionE if (earliestEpochEntryOptional.isPresent()) { EpochEntry earliestEpochEntry = earliestEpochEntryOptional.get(); Iterator epochsToClean = remoteLeaderEpochs.stream() - .filter(remoteEpoch -> remoteEpoch < earliestEpochEntry.epoch) + .filter(remoteEpoch -> remoteEpoch < earliestEpochEntry.epoch()) .iterator(); List listOfSegmentsToBeCleaned = new ArrayList<>(); @@ -1658,11 +1659,11 @@ static NavigableMap buildFilteredLeaderEpochMap(NavigableMap maxBytes) { + if (!remoteStorageFetchInfo.minOneMessage() && firstBatchSize > maxBytes) { LOGGER.debug("Returning empty record for offset {} in partition {} because the first batch size {} " + "is greater than max fetch bytes {}", offset, tp, firstBatchSize, maxBytes); return new FetchDataInfo(new LogOffsetMetadata(offset), MemoryRecords.EMPTY); } int updatedFetchSize = - remoteStorageFetchInfo.minOneMessage && firstBatchSize > maxBytes ? firstBatchSize : maxBytes; + remoteStorageFetchInfo.minOneMessage() && firstBatchSize > maxBytes ? firstBatchSize : maxBytes; ByteBuffer buffer = ByteBuffer.allocate(updatedFetchSize); int remainingBytes = updatedFetchSize; @@ -1770,7 +1771,7 @@ private FetchDataInfo addAbortedTransactions(long startOffset, OffsetIndex offsetIndex = indexCache.getIndexEntry(segmentMetadata).offsetIndex(); long upperBoundOffset = offsetIndex.fetchUpperBoundOffset(startOffsetPosition, fetchSize) - .map(position -> position.offset).orElse(segmentMetadata.endOffset() + 1); + .map(OffsetPosition::offset).orElse(segmentMetadata.endOffset() + 1); final Set abortedTransactions = new HashSet<>(); @@ -1815,8 +1816,8 @@ private void collectAbortedTransactions(long startOffset, if (txnIndexOpt.isPresent()) { TransactionIndex txnIndex = txnIndexOpt.get(); TxnIndexSearchResult searchResult = txnIndex.collectAbortedTxns(startOffset, upperBoundOffset); - accumulator.accept(searchResult.abortedTransactions); - isSearchComplete = searchResult.isComplete; + accumulator.accept(searchResult.abortedTransactions()); + isSearchComplete = searchResult.isComplete(); } if (!isSearchComplete) { currentMetadataOpt = findNextSegmentWithTxnIndex(tp, currentMetadata.endOffset() + 1, leaderEpochCache); @@ -1844,8 +1845,8 @@ private void collectAbortedTransactionInLocalSegments(long startOffset, TransactionIndex txnIndex = localLogSegments.next().txnIndex(); if (txnIndex != null) { TxnIndexSearchResult searchResult = txnIndex.collectAbortedTxns(startOffset, upperBoundOffset); - accumulator.accept(searchResult.abortedTransactions); - if (searchResult.isComplete) { + accumulator.accept(searchResult.abortedTransactions()); + if (searchResult.isComplete()) { return; } } @@ -1886,9 +1887,9 @@ Optional findNextSegmentWithTxnIndex(TopicPartition tp } int initialEpoch = initialEpochOpt.getAsInt(); for (EpochEntry epochEntry : leaderEpochCache.epochEntries()) { - if (epochEntry.epoch >= initialEpoch) { - long startOffset = Math.max(epochEntry.startOffset, offset); - Optional metadataOpt = fetchNextSegmentWithTxnIndex(tp, epochEntry.epoch, startOffset); + if (epochEntry.epoch() >= initialEpoch) { + long startOffset = Math.max(epochEntry.startOffset(), offset); + Optional metadataOpt = fetchNextSegmentWithTxnIndex(tp, epochEntry.epoch(), startOffset); if (metadataOpt.isPresent()) { return metadataOpt; } @@ -1917,7 +1918,7 @@ OffsetAndEpoch findHighestRemoteOffset(TopicIdPartition topicIdPartition, Unifie LeaderEpochFileCache leaderEpochCache = log.leaderEpochCache(); Optional maybeEpochEntry = leaderEpochCache.latestEntry(); while (offsetAndEpoch == null && maybeEpochEntry.isPresent()) { - int epoch = maybeEpochEntry.get().epoch; + int epoch = maybeEpochEntry.get().epoch(); Optional highestRemoteOffsetOpt = remoteLogMetadataManagerPlugin.get().highestOffsetForEpoch(topicIdPartition, epoch); if (highestRemoteOffsetOpt.isPresent()) { @@ -1946,7 +1947,7 @@ long findLogStartOffset(TopicIdPartition topicIdPartition, UnifiedLog log) throw Optional logStartOffset = Optional.empty(); LeaderEpochFileCache leaderEpochCache = log.leaderEpochCache(); OptionalInt earliestEpochOpt = leaderEpochCache.earliestEntry() - .map(epochEntry -> OptionalInt.of(epochEntry.epoch)) + .map(epochEntry -> OptionalInt.of(epochEntry.epoch())) .orElseGet(OptionalInt::empty); while (logStartOffset.isEmpty() && earliestEpochOpt.isPresent()) { Iterator iterator = diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogReader.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogReader.java index 898ff52760c1d..afcba812e1ca2 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogReader.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogReader.java @@ -51,7 +51,7 @@ public RemoteLogReader(RemoteStorageFetchInfo fetchInfo, this.rlm = rlm; this.brokerTopicStats = brokerTopicStats; this.callback = callback; - this.brokerTopicStats.topicStats(fetchInfo.topicIdPartition.topic()).remoteFetchRequestRate().mark(); + this.brokerTopicStats.topicStats(fetchInfo.topicIdPartition().topic()).remoteFetchRequestRate().mark(); this.brokerTopicStats.allTopicsStats().remoteFetchRequestRate().mark(); this.quotaManager = quotaManager; this.remoteReadTimer = remoteReadTimer; @@ -61,21 +61,21 @@ public RemoteLogReader(RemoteStorageFetchInfo fetchInfo, public Void call() { RemoteLogReadResult result; try { - LOGGER.debug("Reading records from remote storage for topic partition {}", fetchInfo.topicIdPartition); + LOGGER.debug("Reading records from remote storage for topic partition {}", fetchInfo.topicIdPartition()); FetchDataInfo fetchDataInfo = remoteReadTimer.time(() -> rlm.read(fetchInfo)); - brokerTopicStats.topicStats(fetchInfo.topicIdPartition.topic()).remoteFetchBytesRate().mark(fetchDataInfo.records.sizeInBytes()); + brokerTopicStats.topicStats(fetchInfo.topicIdPartition().topic()).remoteFetchBytesRate().mark(fetchDataInfo.records.sizeInBytes()); brokerTopicStats.allTopicsStats().remoteFetchBytesRate().mark(fetchDataInfo.records.sizeInBytes()); result = new RemoteLogReadResult(Optional.of(fetchDataInfo), Optional.empty()); } catch (OffsetOutOfRangeException e) { result = new RemoteLogReadResult(Optional.empty(), Optional.of(e)); } catch (Exception e) { - brokerTopicStats.topicStats(fetchInfo.topicIdPartition.topic()).failedRemoteFetchRequestRate().mark(); + brokerTopicStats.topicStats(fetchInfo.topicIdPartition().topic()).failedRemoteFetchRequestRate().mark(); brokerTopicStats.allTopicsStats().failedRemoteFetchRequestRate().mark(); - LOGGER.error("Error occurred while reading the remote data for {}", fetchInfo.topicIdPartition, e); + LOGGER.error("Error occurred while reading the remote data for {}", fetchInfo.topicIdPartition(), e); result = new RemoteLogReadResult(Optional.empty(), Optional.of(e)); } - LOGGER.debug("Finished reading records from remote storage for topic partition {}", fetchInfo.topicIdPartition); - quotaManager.record(result.fetchDataInfo.map(fetchDataInfo -> fetchDataInfo.records.sizeInBytes()).orElse(0)); + LOGGER.debug("Finished reading records from remote storage for topic partition {}", fetchInfo.topicIdPartition()); + quotaManager.record(result.fetchDataInfo().map(fetchDataInfo -> fetchDataInfo.records.sizeInBytes()).orElse(0)); callback.accept(result); return null; } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/checkpoint/CleanShutdownFileHandler.java b/storage/src/main/java/org/apache/kafka/storage/internals/checkpoint/CleanShutdownFileHandler.java index 42c41226783a4..4f476f3a0557b 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/checkpoint/CleanShutdownFileHandler.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/checkpoint/CleanShutdownFileHandler.java @@ -94,7 +94,7 @@ public OptionalLong read() { Content content = Json.parseStringAs(text, Content.class); return OptionalLong.of(content.brokerEpoch); } catch (Exception e) { - logger.debug("Fail to read the clean shutdown file in " + cleanShutdownFile.toPath() + ":" + e); + logger.debug("Fail to read the clean shutdown file in {}", cleanShutdownFile.toPath(), e); return OptionalLong.empty(); } } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/checkpoint/LeaderEpochCheckpointFile.java b/storage/src/main/java/org/apache/kafka/storage/internals/checkpoint/LeaderEpochCheckpointFile.java index 33b4a1d5bbdee..17de83d4d20aa 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/checkpoint/LeaderEpochCheckpointFile.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/checkpoint/LeaderEpochCheckpointFile.java @@ -74,7 +74,7 @@ public static File newFile(File dir) { private static class Formatter implements EntryFormatter { public String toString(EpochEntry entry) { - return entry.epoch + " " + entry.startOffset; + return entry.epoch() + " " + entry.startOffset(); } public Optional fromString(String line) { diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/checkpoint/OffsetCheckpointFile.java b/storage/src/main/java/org/apache/kafka/storage/internals/checkpoint/OffsetCheckpointFile.java index 3ad6c0f0c370e..5225d6503d15a 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/checkpoint/OffsetCheckpointFile.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/checkpoint/OffsetCheckpointFile.java @@ -92,15 +92,7 @@ public Optional fromString(String line) { } } - static class TopicPartitionOffset { - - final TopicPartition tp; - final long offset; - - TopicPartitionOffset(TopicPartition tp, long offset) { - this.tp = tp; - this.offset = offset; - } + record TopicPartitionOffset(TopicPartition tp, long offset) { } } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/checkpoint/PartitionMetadata.java b/storage/src/main/java/org/apache/kafka/storage/internals/checkpoint/PartitionMetadata.java index 1928780900d8e..13fae37bde06a 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/checkpoint/PartitionMetadata.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/checkpoint/PartitionMetadata.java @@ -19,22 +19,7 @@ import org.apache.kafka.common.Uuid; -public class PartitionMetadata { - private final int version; - private final Uuid topicId; - - public PartitionMetadata(int version, Uuid topicId) { - this.version = version; - this.topicId = topicId; - } - - public int version() { - return version; - } - - public Uuid topicId() { - return topicId; - } +public record PartitionMetadata(int version, Uuid topicId) { public String encode() { return "version: " + version + "\ntopic_id: " + topicId; diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/epoch/LeaderEpochFileCache.java b/storage/src/main/java/org/apache/kafka/storage/internals/epoch/LeaderEpochFileCache.java index bc8c03a69033e..568dc05c1e6a8 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/epoch/LeaderEpochFileCache.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/epoch/LeaderEpochFileCache.java @@ -92,7 +92,7 @@ private LeaderEpochFileCache(List epochEntries, LogContext logContext = new LogContext("[LeaderEpochCache " + topicPartition + "] "); log = logContext.logger(LeaderEpochFileCache.class); for (EpochEntry entry : epochEntries) { - epochs.put(entry.epoch, entry); + epochs.put(entry.epoch(), entry); } } @@ -118,11 +118,11 @@ public void assign(List entries) { } private boolean isUpdateNeeded(EpochEntry entry) { - return latestEntry().map(epochEntry -> entry.epoch != epochEntry.epoch || entry.startOffset < epochEntry.startOffset).orElse(true); + return latestEntry().map(epochEntry -> entry.epoch() != epochEntry.epoch() || entry.startOffset() < epochEntry.startOffset()).orElse(true); } private boolean assign(EpochEntry entry) { - if (entry.epoch < 0 || entry.startOffset < 0) { + if (entry.epoch() < 0 || entry.startOffset() < 0) { throw new IllegalArgumentException("Received invalid partition leader epoch entry " + entry); } @@ -134,7 +134,7 @@ private boolean assign(EpochEntry entry) { try { if (isUpdateNeeded(entry)) { maybeTruncateNonMonotonicEntries(entry); - epochs.put(entry.epoch, entry); + epochs.put(entry.epoch(), entry); return true; } else { return false; @@ -150,9 +150,9 @@ private boolean assign(EpochEntry entry) { private void maybeTruncateNonMonotonicEntries(EpochEntry newEntry) { List removedEpochs = removeWhileMatching( epochs.descendingMap().entrySet().iterator(), - entry -> entry.epoch >= newEntry.epoch || entry.startOffset >= newEntry.startOffset); + entry -> entry.epoch() >= newEntry.epoch() || entry.startOffset() >= newEntry.startOffset()); - if (removedEpochs.size() > 1 || (!removedEpochs.isEmpty() && removedEpochs.get(0).startOffset != newEntry.startOffset)) { + if (removedEpochs.size() > 1 || (!removedEpochs.isEmpty() && removedEpochs.get(0).startOffset() != newEntry.startOffset())) { // Only log a warning if there were non-trivial removals. If the start offset of the new entry // matches the start offset of the removed epoch, then no data has been written and the truncation @@ -200,13 +200,13 @@ public Optional latestEntry() { * which has messages assigned to it. */ public Optional latestEpoch() { - return latestEntry().map(epochEntry -> epochEntry.epoch); + return latestEntry().map(epochEntry -> epochEntry.epoch()); } public OptionalInt previousEpoch() { lock.readLock().lock(); try { - return latestEntry().flatMap(entry -> Optional.ofNullable(epochs.lowerEntry(entry.epoch))) + return latestEntry().flatMap(entry -> Optional.ofNullable(epochs.lowerEntry(entry.epoch()))) .map(integerEpochEntryEntry -> OptionalInt.of(integerEpochEntryEntry.getKey())).orElseGet(OptionalInt::empty); } finally { lock.readLock().unlock(); @@ -309,11 +309,11 @@ public Map.Entry endOffsetFor(int requestedEpoch, long logEndOffs // epochs in between, but the point is that the data has already been removed from the log // and we want to ensure that the follower can replicate correctly beginning from the leader's // start offset. - epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, higherEntry.getValue().startOffset); + epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(requestedEpoch, higherEntry.getValue().startOffset()); } else { // We have at least one previous epoch and one subsequent epoch. The result is the first // prior epoch and the starting offset of the first subsequent epoch. - epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(floorEntry.getValue().epoch, higherEntry.getValue().startOffset); + epochAndOffset = new AbstractMap.SimpleImmutableEntry<>(floorEntry.getValue().epoch(), higherEntry.getValue().startOffset()); } } } @@ -389,12 +389,12 @@ public void truncateFromStartAsyncFlush(long startOffset) { private static List truncateFromStart(TreeMap epochs, long startOffset) { List removedEntries = removeWhileMatching( - epochs.entrySet().iterator(), entry -> entry.startOffset <= startOffset); + epochs.entrySet().iterator(), entry -> entry.startOffset() <= startOffset); if (!removedEntries.isEmpty()) { EpochEntry firstBeforeStartOffset = removedEntries.get(removedEntries.size() - 1); - EpochEntry updatedFirstEntry = new EpochEntry(firstBeforeStartOffset.epoch, startOffset); - epochs.put(updatedFirstEntry.epoch, updatedFirstEntry); + EpochEntry updatedFirstEntry = new EpochEntry(firstBeforeStartOffset.epoch(), startOffset); + epochs.put(updatedFirstEntry.epoch(), updatedFirstEntry); } return removedEntries; @@ -402,8 +402,8 @@ private static List truncateFromStart(TreeMap e private static List truncateFromEnd(TreeMap epochs, long endOffset) { Optional epochEntry = Optional.ofNullable(epochs.lastEntry()).map(Entry::getValue); - if (endOffset >= 0 && epochEntry.isPresent() && epochEntry.get().startOffset >= endOffset) { - return removeWhileMatching(epochs.descendingMap().entrySet().iterator(), x -> x.startOffset >= endOffset); + if (endOffset >= 0 && epochEntry.isPresent() && epochEntry.get().startOffset() >= endOffset) { + return removeWhileMatching(epochs.descendingMap().entrySet().iterator(), x -> x.startOffset() >= endOffset); } return List.of(); } @@ -413,8 +413,8 @@ public OptionalInt epochForOffset(long offset) { try { OptionalInt previousEpoch = OptionalInt.empty(); for (EpochEntry epochEntry : epochs.values()) { - int epoch = epochEntry.epoch; - long startOffset = epochEntry.startOffset; + int epoch = epochEntry.epoch(); + long startOffset = epochEntry.startOffset(); // Found the exact offset, return the respective epoch. if (startOffset == offset) return OptionalInt.of(epoch); @@ -505,7 +505,7 @@ public NavigableMap epochWithOffsets() { try { NavigableMap epochWithOffsets = new TreeMap<>(); for (EpochEntry epochEntry : epochs.values()) { - epochWithOffsets.put(epochEntry.epoch, epochEntry.startOffset); + epochWithOffsets.put(epochEntry.epoch(), epochEntry.startOffset()); } return epochWithOffsets; } finally { diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/AbortedTxn.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/AbortedTxn.java index cce6867c31751..05217d97ba132 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/AbortedTxn.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/AbortedTxn.java @@ -44,7 +44,7 @@ public class AbortedTxn { } public AbortedTxn(CompletedTxn completedTxn, long lastStableOffset) { - this(completedTxn.producerId, completedTxn.firstOffset, completedTxn.lastOffset, lastStableOffset); + this(completedTxn.producerId(), completedTxn.firstOffset(), completedTxn.lastOffset(), lastStableOffset); } public AbortedTxn(long producerId, long firstOffset, long lastOffset, long lastStableOffset) { diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/AbstractIndex.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/AbstractIndex.java index 12e99b34739e9..b1475884e3a50 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/AbstractIndex.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/AbstractIndex.java @@ -503,12 +503,10 @@ private int indexSlotRangeFor(ByteBuffer idx, long target, IndexSearchType searc // check if the target offset is smaller than the least offset if (compareIndexEntry(parseEntry(idx, 0), target, searchEntity) > 0) { - switch (searchResultType) { - case LARGEST_LOWER_BOUND: - return -1; - case SMALLEST_UPPER_BOUND: - return 0; - } + return switch (searchResultType) { + case LARGEST_LOWER_BOUND -> -1; + case SMALLEST_UPPER_BOUND -> 0; + }; } return binarySearch(idx, target, searchEntity, searchResultType, 0, firstHotEntry); @@ -544,18 +542,10 @@ else if (compareResult < 0) } private int compareIndexEntry(IndexEntry indexEntry, long target, IndexSearchType searchEntity) { - int result; - switch (searchEntity) { - case KEY: - result = Long.compare(indexEntry.indexKey(), target); - break; - case VALUE: - result = Long.compare(indexEntry.indexValue(), target); - break; - default: - throw new IllegalStateException("Unexpected IndexSearchType: " + searchEntity); - } - return result; + return switch (searchEntity) { + case KEY -> Long.compare(indexEntry.indexKey(), target); + case VALUE -> Long.compare(indexEntry.indexValue(), target); + }; } private OptionalInt toRelative(long offset) { diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/BatchMetadata.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/BatchMetadata.java index e6b9cb6fef49e..9052fd1c33577 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/BatchMetadata.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/BatchMetadata.java @@ -18,23 +18,7 @@ import org.apache.kafka.common.record.DefaultRecordBatch; -public class BatchMetadata { - - public final int lastSeq; - public final long lastOffset; - public final int offsetDelta; - public final long timestamp; - - public BatchMetadata( - int lastSeq, - long lastOffset, - int offsetDelta, - long timestamp) { - this.lastSeq = lastSeq; - this.lastOffset = lastOffset; - this.offsetDelta = offsetDelta; - this.timestamp = timestamp; - } +public record BatchMetadata(int lastSeq, long lastOffset, int offsetDelta, long timestamp) { public int firstSeq() { return DefaultRecordBatch.decrementSequence(lastSeq, offsetDelta); @@ -44,28 +28,6 @@ public long firstOffset() { return lastOffset - offsetDelta; } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - BatchMetadata that = (BatchMetadata) o; - - return lastSeq == that.lastSeq && - lastOffset == that.lastOffset && - offsetDelta == that.offsetDelta && - timestamp == that.timestamp; - } - - @Override - public int hashCode() { - int result = lastSeq; - result = 31 * result + Long.hashCode(lastOffset); - result = 31 * result + offsetDelta; - result = 31 * result + Long.hashCode(timestamp); - return result; - } - @Override public String toString() { return "BatchMetadata(" + diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java index 54ad6a4e79a97..e17809d988ece 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java @@ -338,10 +338,10 @@ public BatchRetentionResult checkBatchRetention(RecordBatch batch) { // last producer epoch, which is needed to ensure fencing. boolean isBatchLastRecordOfProducer = Optional.ofNullable(lastRecordsOfActiveProducers.get(batch.producerId())) .map(lastRecord -> { - if (lastRecord.lastDataOffset.isPresent()) { - return batch.lastOffset() == lastRecord.lastDataOffset.getAsLong(); + if (lastRecord.lastDataOffset().isPresent()) { + return batch.lastOffset() == lastRecord.lastDataOffset().getAsLong(); } else { - return batch.isControlBatch() && batch.producerEpoch() == lastRecord.producerEpoch; + return batch.isControlBatch() && batch.producerEpoch() == lastRecord.producerEpoch(); } }) .orElse(false); @@ -702,7 +702,7 @@ private boolean buildOffsetMapForSegment(TopicPartition topicPartition, int maxLogMessageSize, CleanedTransactionMetadata transactionMetadata, CleanerStats stats) throws IOException, DigestException { - int position = segment.offsetIndex().lookup(startOffset).position; + int position = segment.offsetIndex().lookup(startOffset).position(); int maxDesiredMapSize = (int) (map.slots() * dupBufferLoadFactor); while (position < segment.log().sizeInBytes()) { diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/CompletedTxn.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/CompletedTxn.java index e8f70ccf5b4f6..7e162502e226f 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/CompletedTxn.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/CompletedTxn.java @@ -19,57 +19,12 @@ /** * A class used to hold useful metadata about a completed transaction. This is used to build * the transaction index after appending to the log. + * + * @param producerId The ID of the producer + * @param firstOffset The first offset (inclusive) of the transaction + * @param lastOffset The last offset (inclusive) of the transaction. This is always the offset of the + * COMMIT/ABORT control record which indicates the transaction's completion. + * @param isAborted Whether the transaction was aborted */ -public class CompletedTxn { - public final long producerId; - public final long firstOffset; - public final long lastOffset; - public final boolean isAborted; - - /** - * Create an instance of this class. - * - * @param producerId The ID of the producer - * @param firstOffset The first offset (inclusive) of the transaction - * @param lastOffset The last offset (inclusive) of the transaction. This is always the offset of the - * COMMIT/ABORT control record which indicates the transaction's completion. - * @param isAborted Whether the transaction was aborted - */ - public CompletedTxn(long producerId, long firstOffset, long lastOffset, boolean isAborted) { - this.producerId = producerId; - this.firstOffset = firstOffset; - this.lastOffset = lastOffset; - this.isAborted = isAborted; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - CompletedTxn that = (CompletedTxn) o; - - return producerId == that.producerId - && firstOffset == that.firstOffset - && lastOffset == that.lastOffset - && isAborted == that.isAborted; - } - - @Override - public int hashCode() { - int result = Long.hashCode(producerId); - result = 31 * result + Long.hashCode(firstOffset); - result = 31 * result + Long.hashCode(lastOffset); - result = 31 * result + Boolean.hashCode(isAborted); - return result; - } - - @Override - public String toString() { - return "CompletedTxn(producerId=" + producerId + - ", firstOffset=" + firstOffset + - ", lastOffset=" + lastOffset + - ", isAborted=" + isAborted + - ')'; - } +public record CompletedTxn(long producerId, long firstOffset, long lastOffset, boolean isAborted) { } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/EpochEntry.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/EpochEntry.java index 3b4666ed0b2f9..a024919b665d8 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/EpochEntry.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/EpochEntry.java @@ -17,29 +17,7 @@ package org.apache.kafka.storage.internals.log; // Mapping of epoch to the first offset of the subsequent epoch -public class EpochEntry { - public final int epoch; - public final long startOffset; - - public EpochEntry(int epoch, long startOffset) { - this.epoch = epoch; - this.startOffset = startOffset; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - EpochEntry that = (EpochEntry) o; - return epoch == that.epoch && startOffset == that.startOffset; - } - - @Override - public int hashCode() { - int result = epoch; - result = 31 * result + Long.hashCode(startOffset); - return result; - } +public record EpochEntry(int epoch, long startOffset) { @Override public String toString() { diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LastRecord.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LastRecord.java index 6886079cbf974..e000c77864df0 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LastRecord.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LastRecord.java @@ -23,30 +23,9 @@ * The last written record for a given producer. The last data offset may be undefined * if the only log entry for a producer is a transaction marker. */ -public final class LastRecord { - public final OptionalLong lastDataOffset; - public final short producerEpoch; - - public LastRecord(OptionalLong lastDataOffset, short producerEpoch) { +public record LastRecord(OptionalLong lastDataOffset, short producerEpoch) { + public LastRecord { Objects.requireNonNull(lastDataOffset, "lastDataOffset must be non null"); - this.lastDataOffset = lastDataOffset; - this.producerEpoch = producerEpoch; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - LastRecord that = (LastRecord) o; - - return producerEpoch == that.producerEpoch && - lastDataOffset.equals(that.lastDataOffset); - } - - @Override - public int hashCode() { - return 31 * lastDataOffset.hashCode() + producerEpoch; } @Override diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LazyIndex.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LazyIndex.java index c78075595aec8..afc1c679c29c5 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LazyIndex.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LazyIndex.java @@ -235,14 +235,10 @@ public void closeHandler() { @SuppressWarnings("unchecked") private T loadIndex(File file) throws IOException { - switch (indexType) { - case OFFSET: - return (T) new OffsetIndex(file, baseOffset, maxIndexSize, true); - case TIME: - return (T) new TimeIndex(file, baseOffset, maxIndexSize, true); - default: - throw new IllegalStateException("Unexpected indexType " + indexType); - } + return switch (indexType) { + case OFFSET -> (T) new OffsetIndex(file, baseOffset, maxIndexSize, true); + case TIME -> (T) new TimeIndex(file, baseOffset, maxIndexSize, true); + }; } } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LoadedLogOffsets.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LoadedLogOffsets.java index a025ea86d95f5..30bcf31258856 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LoadedLogOffsets.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LoadedLogOffsets.java @@ -18,11 +18,7 @@ import java.util.Objects; -public class LoadedLogOffsets { - public final long logStartOffset; - public final long recoveryPoint; - public final LogOffsetMetadata nextOffsetMetadata; - +public record LoadedLogOffsets(long logStartOffset, long recoveryPoint, LogOffsetMetadata nextOffsetMetadata) { public LoadedLogOffsets(final long logStartOffset, final long recoveryPoint, final LogOffsetMetadata nextOffsetMetadata) { @@ -31,29 +27,6 @@ public LoadedLogOffsets(final long logStartOffset, this.nextOffsetMetadata = Objects.requireNonNull(nextOffsetMetadata, "nextOffsetMetadata should not be null"); } - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - final LoadedLogOffsets that = (LoadedLogOffsets) o; - return logStartOffset == that.logStartOffset - && recoveryPoint == that.recoveryPoint - && nextOffsetMetadata.equals(that.nextOffsetMetadata); - } - - @Override - public int hashCode() { - int result = Long.hashCode(logStartOffset); - result = 31 * result + Long.hashCode(recoveryPoint); - result = 31 * result + nextOffsetMetadata.hashCode(); - return result; - } - @Override public String toString() { return "LoadedLogOffsets(" + diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LocalLog.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LocalLog.java index 2c4938aa96d33..cf0716ee23427 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LocalLog.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LocalLog.java @@ -557,8 +557,8 @@ private void collectAbortedTransactions(long startOffset, long upperBoundOffset, while (segmentEntryOpt.isPresent()) { LogSegment segment = segmentEntryOpt.get(); TxnIndexSearchResult searchResult = segment.collectAbortedTxns(startOffset, upperBoundOffset); - accumulator.accept(searchResult.abortedTransactions); - if (searchResult.isComplete) return; + accumulator.accept(searchResult.abortedTransactions()); + if (searchResult.isComplete()) return; segmentEntryOpt = nextItem(higherSegments); } } @@ -752,7 +752,7 @@ public static TopicPartition parseTopicPartitionName(File dir) throws IOExceptio throw new KafkaException("dir should not be null"); } String dirName = dir.getName(); - if (dirName.isEmpty() || !dirName.contains("-")) { + if (!dirName.contains("-")) { throw exception(dir); } if (dirName.endsWith(DELETE_DIR_SUFFIX) && !DELETE_DIR_PATTERN.matcher(dirName).matches() || @@ -1055,20 +1055,12 @@ public static List replaceSegments(LogSegments existingSegments, return deletedNotReplaced; } - public static class SplitSegmentResult { - - public final List deletedSegments; - public final List newSegments; - - /** - * Holds the result of splitting a segment into one or more segments, see LocalLog.splitOverflowedSegment(). - * - * @param deletedSegments segments deleted when splitting a segment - * @param newSegments new segments created when splitting a segment - */ - public SplitSegmentResult(List deletedSegments, List newSegments) { - this.deletedSegments = deletedSegments; - this.newSegments = newSegments; - } + /** + * Holds the result of splitting a segment into one or more segments, see LocalLog.splitOverflowedSegment(). + * + * @param deletedSegments segments deleted when splitting a segment + * @param newSegments new segments created when splitting a segment + */ + public record SplitSegmentResult(List deletedSegments, List newSegments) { } } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index eb7d0eaa5d4d5..363fb06c2de28 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -339,28 +339,20 @@ public LogConfig(Map props, Set overriddenConfigs) { } private Optional getCompression() { - switch (compressionType) { - case GZIP: - return Optional.of(Compression.gzip() - .level(getInt(TopicConfig.COMPRESSION_GZIP_LEVEL_CONFIG)) - .build()); - case LZ4: - return Optional.of(Compression.lz4() - .level(getInt(TopicConfig.COMPRESSION_LZ4_LEVEL_CONFIG)) - .build()); - case ZSTD: - return Optional.of(Compression.zstd() - .level(getInt(TopicConfig.COMPRESSION_ZSTD_LEVEL_CONFIG)) - .build()); - case SNAPPY: - return Optional.of(Compression.snappy().build()); - case UNCOMPRESSED: - return Optional.of(Compression.NONE); - case PRODUCER: - return Optional.empty(); - default: - throw new IllegalArgumentException("Invalid value for " + TopicConfig.COMPRESSION_TYPE_CONFIG); - } + return switch (compressionType) { + case GZIP -> Optional.of(Compression.gzip() + .level(getInt(TopicConfig.COMPRESSION_GZIP_LEVEL_CONFIG)) + .build()); + case LZ4 -> Optional.of(Compression.lz4() + .level(getInt(TopicConfig.COMPRESSION_LZ4_LEVEL_CONFIG)) + .build()); + case ZSTD -> Optional.of(Compression.zstd() + .level(getInt(TopicConfig.COMPRESSION_ZSTD_LEVEL_CONFIG)) + .build()); + case SNAPPY -> Optional.of(Compression.snappy().build()); + case UNCOMPRESSED -> Optional.of(Compression.NONE); + case PRODUCER -> Optional.empty(); + }; } public int segmentSize() { diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogLoader.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogLoader.java index eaab1881d0851..dca7e298ceb08 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogLoader.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogLoader.java @@ -338,7 +338,7 @@ private T retryOnOffsetOverflow(StorageAction function) thro scheduler, logDirFailureChannel, logPrefix); - deleteProducerSnapshotsAsync(result.deletedSegments); + deleteProducerSnapshotsAsync(result.deletedSegments()); } } } @@ -438,15 +438,7 @@ private Optional deleteSegmentsIfLogStartGreaterThanLogEnd() throws IOExce return Optional.empty(); } - static class RecoveryOffsets { - - final long newRecoveryPoint; - final long nextOffset; - - RecoveryOffsets(long newRecoveryPoint, long nextOffset) { - this.newRecoveryPoint = newRecoveryPoint; - this.nextOffset = nextOffset; - } + record RecoveryOffsets(long newRecoveryPoint, long nextOffset) { } /** diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogOffsetSnapshot.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogOffsetSnapshot.java index 44b40943db0e0..b96f68ba8dd3a 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogOffsetSnapshot.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogOffsetSnapshot.java @@ -16,52 +16,13 @@ */ package org.apache.kafka.storage.internals.log; -import java.util.Objects; - /** * Container class which represents a snapshot of the significant offsets for a partition. This allows fetching * of these offsets atomically without the possibility of a leader change affecting their consistency relative * to each other. See {@link UnifiedLog#fetchOffsetSnapshot()}. */ -public class LogOffsetSnapshot { - - public final long logStartOffset; - public final LogOffsetMetadata logEndOffset; - public final LogOffsetMetadata highWatermark; - public final LogOffsetMetadata lastStableOffset; - - public LogOffsetSnapshot(long logStartOffset, - LogOffsetMetadata logEndOffset, - LogOffsetMetadata highWatermark, - LogOffsetMetadata lastStableOffset) { - - this.logStartOffset = logStartOffset; - this.logEndOffset = logEndOffset; - this.highWatermark = highWatermark; - this.lastStableOffset = lastStableOffset; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - LogOffsetSnapshot that = (LogOffsetSnapshot) o; - - return logStartOffset == that.logStartOffset && - Objects.equals(logEndOffset, that.logEndOffset) && - Objects.equals(highWatermark, that.highWatermark) && - Objects.equals(lastStableOffset, that.lastStableOffset); - } - - @Override - public int hashCode() { - int result = (int) (logStartOffset ^ (logStartOffset >>> 32)); - result = 31 * result + (logEndOffset != null ? logEndOffset.hashCode() : 0); - result = 31 * result + (highWatermark != null ? highWatermark.hashCode() : 0); - result = 31 * result + (lastStableOffset != null ? lastStableOffset.hashCode() : 0); - return result; - } +public record LogOffsetSnapshot(long logStartOffset, LogOffsetMetadata logEndOffset, LogOffsetMetadata highWatermark, + LogOffsetMetadata lastStableOffset) { @Override public String toString() { diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogReadInfo.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogReadInfo.java index 4c3a42698c809..d50f00ddba192 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogReadInfo.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogReadInfo.java @@ -23,28 +23,8 @@ /** * Structure used for lower level reads using {@link kafka.cluster.Partition#fetchRecords()}. */ -public class LogReadInfo { - - public final FetchDataInfo fetchedData; - public final Optional divergingEpoch; - public final long highWatermark; - public final long logStartOffset; - public final long logEndOffset; - public final long lastStableOffset; - - public LogReadInfo(FetchDataInfo fetchedData, - Optional divergingEpoch, - long highWatermark, - long logStartOffset, - long logEndOffset, - long lastStableOffset) { - this.fetchedData = fetchedData; - this.divergingEpoch = divergingEpoch; - this.highWatermark = highWatermark; - this.logStartOffset = logStartOffset; - this.logEndOffset = logEndOffset; - this.lastStableOffset = lastStableOffset; - } +public record LogReadInfo(FetchDataInfo fetchedData, Optional divergingEpoch, + long highWatermark, long logStartOffset, long logEndOffset, long lastStableOffset) { @Override public String toString() { diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegment.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegment.java index de5e0f50433ed..e33adb354dd02 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegment.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegment.java @@ -165,11 +165,11 @@ public TransactionIndex txnIndex() { } public boolean shouldRoll(RollParams rollParams) throws IOException { - boolean reachedRollMs = timeWaitedForRoll(rollParams.now, rollParams.maxTimestampInMessages) > rollParams.maxSegmentMs - rollJitterMs; + boolean reachedRollMs = timeWaitedForRoll(rollParams.now(), rollParams.maxTimestampInMessages()) > rollParams.maxSegmentMs() - rollJitterMs; int size = size(); - return size > rollParams.maxSegmentBytes - rollParams.messagesSize || + return size > rollParams.maxSegmentBytes() - rollParams.messagesSize() || (size > 0 && reachedRollMs) || - offsetIndex().isFull() || timeIndex().isFull() || !canConvertToRelativeOffset(rollParams.maxOffsetInMessages); + offsetIndex().isFull() || timeIndex().isFull() || !canConvertToRelativeOffset(rollParams.maxOffsetInMessages()); } public void resizeIndexes(int size) throws IOException { @@ -215,14 +215,14 @@ public TimestampOffset readMaxTimestampAndOffsetSoFar() throws IOException { * Note that this may result in time index materialization. */ public long maxTimestampSoFar() throws IOException { - return readMaxTimestampAndOffsetSoFar().timestamp; + return readMaxTimestampAndOffsetSoFar().timestamp(); } /** * Note that this may result in time index materialization. */ private long shallowOffsetOfMaxTimestampSoFar() throws IOException { - return readMaxTimestampAndOffsetSoFar().offset; + return readMaxTimestampAndOffsetSoFar().offset(); } /* Return the size in bytes of this log segment */ @@ -346,7 +346,7 @@ public int appendFromFile(FileRecords records, int start) throws IOException { /* not thread safe */ public void updateTxnIndex(CompletedTxn completedTxn, long lastStableOffset) throws IOException { - if (completedTxn.isAborted) { + if (completedTxn.isAborted()) { LOGGER.trace("Writing aborted transaction {} to transaction index, last stable offset is {}", completedTxn, lastStableOffset); txnIndex.append(new AbortedTxn(completedTxn, lastStableOffset)); } @@ -393,7 +393,7 @@ public LogOffsetPosition translateOffset(long offset) throws IOException { */ LogOffsetPosition translateOffset(long offset, int startingFilePosition) throws IOException { OffsetPosition mapping = offsetIndex().lookup(offset); - return log.searchForOffsetFromPosition(offset, Math.max(mapping.position, startingFilePosition)); + return log.searchForOffsetFromPosition(offset, Math.max(mapping.position(), startingFilePosition)); } /** @@ -460,7 +460,7 @@ public FetchDataInfo read(long startOffset, int maxSize, Optional maxPosit public OptionalLong fetchUpperBoundOffset(OffsetPosition startOffsetPosition, int fetchSize) throws IOException { return offsetIndex().fetchUpperBoundOffset(startOffsetPosition, fetchSize) - .map(offsetPosition -> OptionalLong.of(offsetPosition.offset)).orElseGet(OptionalLong::empty); + .map(offsetPosition -> OptionalLong.of(offsetPosition.offset())).orElseGet(OptionalLong::empty); } /** @@ -541,7 +541,7 @@ public String toString() { return "LogSegment(baseOffset=" + baseOffset + ", size=" + size() + ", lastModifiedTime=" + lastModified() + - ", largestRecordTimestamp=" + maxTimestampAndOffsetSoFar.timestamp + + ", largestRecordTimestamp=" + maxTimestampAndOffsetSoFar.timestamp() + ")"; } @@ -590,11 +590,11 @@ public int truncateTo(long offset) throws IOException { private TimestampOffset readLargestTimestamp() throws IOException { // Get the last time index entry. If the time index is empty, it will return (-1, baseOffset) TimestampOffset lastTimeIndexEntry = timeIndex().lastEntry(); - OffsetPosition offsetPosition = offsetIndex().lookup(lastTimeIndexEntry.offset); + OffsetPosition offsetPosition = offsetIndex().lookup(lastTimeIndexEntry.offset()); // Scan the rest of the messages to see if there is a larger timestamp after the last time index entry. - FileRecords.TimestampAndOffset maxTimestampOffsetAfterLastEntry = log.largestTimestampAfter(offsetPosition.position); - if (maxTimestampOffsetAfterLastEntry.timestamp > lastTimeIndexEntry.timestamp) + FileRecords.TimestampAndOffset maxTimestampOffsetAfterLastEntry = log.largestTimestampAfter(offsetPosition.position()); + if (maxTimestampOffsetAfterLastEntry.timestamp > lastTimeIndexEntry.timestamp()) return new TimestampOffset(maxTimestampOffsetAfterLastEntry.timestamp, maxTimestampOffsetAfterLastEntry.offset); return lastTimeIndexEntry; @@ -612,7 +612,7 @@ public long readNextOffset() throws IOException { return baseOffset; else return fetchData.records.lastBatch() - .map(batch -> batch.nextOffset()) + .map(RecordBatch::nextOffset) .orElse(baseOffset); } @@ -750,7 +750,7 @@ public long getFirstBatchTimestamp() { public Optional findOffsetByTimestamp(long timestampMs, long startingOffset) throws IOException { // Get the index entry with a timestamp less than or equal to the target timestamp TimestampOffset timestampOffset = timeIndex().lookup(timestampMs); - int position = offsetIndex().lookup(Math.max(timestampOffset.offset, startingOffset)).position; + int position = offsetIndex().lookup(Math.max(timestampOffset.offset(), startingOffset)).position(); // Search the timestamp return Optional.ofNullable(log.searchForTimestamp(timestampMs, position, startingOffset)); @@ -770,9 +770,9 @@ public void close() throws IOException { * Close file handlers used by the log segment but don't write to disk. This is used when the disk may have failed */ void closeHandlers() { - Utils.swallow(LOGGER, Level.WARN, "offsetIndex", () -> lazyOffsetIndex.closeHandler()); - Utils.swallow(LOGGER, Level.WARN, "timeIndex", () -> lazyTimeIndex.closeHandler()); - Utils.swallow(LOGGER, Level.WARN, "log", () -> log.closeHandlers()); + Utils.swallow(LOGGER, Level.WARN, "offsetIndex", lazyOffsetIndex::closeHandler); + Utils.swallow(LOGGER, Level.WARN, "timeIndex", lazyTimeIndex::closeHandler); + Utils.swallow(LOGGER, Level.WARN, "log", log::closeHandlers); Utils.closeQuietly(txnIndex, "txnIndex", LOGGER); } @@ -782,10 +782,10 @@ void closeHandlers() { public void deleteIfExists() throws IOException { try { Utils.tryAll(List.of( - () -> deleteTypeIfExists(() -> log.deleteIfExists(), "log", log.file(), true), - () -> deleteTypeIfExists(() -> lazyOffsetIndex.deleteIfExists(), "offset index", offsetIndexFile(), true), - () -> deleteTypeIfExists(() -> lazyTimeIndex.deleteIfExists(), "time index", timeIndexFile(), true), - () -> deleteTypeIfExists(() -> txnIndex.deleteIfExists(), "transaction index", txnIndex.file(), false))); + () -> deleteTypeIfExists(log::deleteIfExists, "log", log.file(), true), + () -> deleteTypeIfExists(lazyOffsetIndex::deleteIfExists, "offset index", offsetIndexFile(), true), + () -> deleteTypeIfExists(lazyTimeIndex::deleteIfExists, "time index", timeIndexFile(), true), + () -> deleteTypeIfExists(txnIndex::deleteIfExists, "transaction index", txnIndex.file(), false))); } catch (Throwable t) { if (t instanceof IOException) throw (IOException) t; diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogValidator.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogValidator.java index 8cb5680068109..a6137ffc68131 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogValidator.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogValidator.java @@ -60,32 +60,11 @@ public interface MetricsRecorder { void recordNoKeyCompactedTopic(); } - public static class ValidationResult { - public final long logAppendTimeMs; - public final MemoryRecords validatedRecords; - public final long maxTimestampMs; - public final boolean messageSizeMaybeChanged; - public final RecordValidationStats recordValidationStats; - - public ValidationResult(long logAppendTimeMs, MemoryRecords validatedRecords, long maxTimestampMs, - boolean messageSizeMaybeChanged, - RecordValidationStats recordValidationStats) { - this.logAppendTimeMs = logAppendTimeMs; - this.validatedRecords = validatedRecords; - this.maxTimestampMs = maxTimestampMs; - this.messageSizeMaybeChanged = messageSizeMaybeChanged; - this.recordValidationStats = recordValidationStats; - } + public record ValidationResult(long logAppendTimeMs, MemoryRecords validatedRecords, long maxTimestampMs, + boolean messageSizeMaybeChanged, RecordValidationStats recordValidationStats) { } - private static class ApiRecordError { - final Errors apiError; - final RecordError recordError; - - private ApiRecordError(Errors apiError, RecordError recordError) { - this.apiError = apiError; - this.recordError = recordError; - } + private record ApiRecordError(Errors apiError, RecordError recordError) { } private final MemoryRecords records; @@ -204,7 +183,7 @@ private ValidationResult convertAndAssignOffsetsNonCompressed(LongRef offsetCoun Optional recordError = validateRecord(batch, topicPartition, record, recordIndex, now, timestampType, timestampBeforeMaxMs, timestampAfterMaxMs, compactedTopic, metricsRecorder); - recordError.ifPresent(e -> recordErrors.add(e)); + recordError.ifPresent(recordErrors::add); // we fail the batch if any record fails, so we stop appending if any record fails if (recordErrors.isEmpty()) builder.appendWithOffset(offsetCounter.value++, record); diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/OffsetIndex.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/OffsetIndex.java index 7d20edf37d3ec..f92a160432fbe 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/OffsetIndex.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/OffsetIndex.java @@ -30,25 +30,25 @@ * An index that maps offsets to physical file locations for a particular log segment. This index may be sparse: * that is it may not hold an entry for all messages in the log. * - * The index is stored in a file that is pre-allocated to hold a fixed maximum number of 8-byte entries. + *

The index is stored in a file that is pre-allocated to hold a fixed maximum number of 8-byte entries. * - * The index supports lookups against a memory-map of this file. These lookups are done using a simple binary search variant + *

The index supports lookups against a memory-map of this file. These lookups are done using a simple binary search variant * to locate the offset/location pair for the greatest offset less than or equal to the target offset. * - * Index files can be opened in two ways: either as an empty, mutable index that allows appends or + *

Index files can be opened in two ways: either as an empty, mutable index that allows appends or * an immutable read-only index file that has previously been populated. The makeReadOnly method will turn a mutable file into an * immutable one and truncate off any extra bytes. This is done when the index file is rolled over. * - * No attempt is made to checksum the contents of this file, in the event of a crash it is rebuilt. + *

No attempt is made to checksum the contents of this file, in the event of a crash it is rebuilt. * - * The file format is a series of entries. The physical format is a 4 byte "relative" offset and a 4 byte file location for the + *

The file format is a series of entries. The physical format is a 4 byte "relative" offset and a 4 byte file location for the * message with that offset. The offset stored is relative to the base offset of the index file. So, for example, * if the base offset was 50, then the offset 55 would be stored as 5. Using relative offsets in this way let's us use * only 4 bytes for the offset. * - * The frequency of entries is up to the user of this class. + *

The frequency of entries is up to the user of this class. * - * All external APIs translate from relative offsets to full offsets, so users of this class do not interact with the internal + *

All external APIs translate from relative offsets to full offsets, so users of this class do not interact with the internal * storage format. */ public final class OffsetIndex extends AbstractIndex { @@ -69,7 +69,7 @@ public OffsetIndex(File file, long baseOffset, int maxIndexSize) throws IOExcept public OffsetIndex(File file, long baseOffset, int maxIndexSize, boolean writable) throws IOException { super(file, baseOffset, maxIndexSize, writable); - lastOffset = lastEntry().offset; + lastOffset = lastEntry().offset(); log.debug("Loaded index file {} with maxEntries = {}, maxIndexSize = {}, entries = {}, lastOffset = {}, file position = {}", file.getAbsolutePath(), maxEntries(), maxIndexSize, entries(), lastOffset, mmap().position()); @@ -127,7 +127,7 @@ public OffsetPosition entry(int n) { public Optional fetchUpperBoundOffset(OffsetPosition fetchOffset, int fetchSize) { return inRemapReadLock(() -> { ByteBuffer idx = mmap().duplicate(); - int slot = smallestUpperBoundSlotFor(idx, fetchOffset.position + fetchSize, IndexSearchType.VALUE); + int slot = smallestUpperBoundSlotFor(idx, fetchOffset.position() + fetchSize, IndexSearchType.VALUE); if (slot == -1) return Optional.empty(); else @@ -214,7 +214,7 @@ private int physical(ByteBuffer buffer, int n) { private void truncateToEntries(int entries) { inLock(() -> { super.truncateToEntries0(entries); - this.lastOffset = lastEntry().offset; + this.lastOffset = lastEntry().offset(); log.debug("Truncated index {} to {} entries; position is now {} and last offset is now {}", file().getAbsolutePath(), entries, mmap().position(), lastOffset); }); diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/OffsetPosition.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/OffsetPosition.java index b32b069aa1f47..31ca01832763c 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/OffsetPosition.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/OffsetPosition.java @@ -21,14 +21,7 @@ * in some log file of the beginning of the message set entry with the * given offset. */ -public final class OffsetPosition implements IndexEntry { - public final long offset; - public final int position; - - public OffsetPosition(long offset, int position) { - this.offset = offset; - this.position = position; - } +public record OffsetPosition(long offset, int position) implements IndexEntry { @Override public long indexKey() { @@ -40,26 +33,6 @@ public long indexValue() { return position; } - @Override - public boolean equals(Object o) { - if (this == o) - return true; - if (o == null || getClass() != o.getClass()) - return false; - - OffsetPosition that = (OffsetPosition) o; - - return offset == that.offset - && position == that.position; - } - - @Override - public int hashCode() { - int result = Long.hashCode(offset); - result = 31 * result + position; - return result; - } - @Override public String toString() { return "OffsetPosition(" + diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/OffsetResultHolder.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/OffsetResultHolder.java index f682b975ef3be..8abe680018228 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/OffsetResultHolder.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/OffsetResultHolder.java @@ -25,7 +25,7 @@ public class OffsetResultHolder { private Optional timestampAndOffsetOpt; - private Optional> futureHolderOpt; + private final Optional> futureHolderOpt; private Optional maybeOffsetsError = Optional.empty(); private Optional lastFetchableOffset = Optional.empty(); @@ -95,51 +95,20 @@ public int hashCode() { return result; } - public static class FileRecordsOrError { - private Optional exception; - private Optional timestampAndOffset; - - public FileRecordsOrError( - Optional exception, - Optional timestampAndOffset - ) { + public record FileRecordsOrError(Optional exception, Optional timestampAndOffset) { + public FileRecordsOrError { if (exception.isPresent() && timestampAndOffset.isPresent()) { throw new IllegalArgumentException("Either exception or timestampAndOffset should be present, but not both"); } - this.exception = exception; - this.timestampAndOffset = timestampAndOffset; - } - - public Optional exception() { - return exception; } - public Optional timestampAndOffset() { - return timestampAndOffset; - } - public boolean hasException() { return exception.isPresent(); } - + public boolean hasTimestampAndOffset() { return timestampAndOffset.isPresent(); } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - FileRecordsOrError that = (FileRecordsOrError) o; - return Objects.equals(exception, that.exception) && Objects.equals(timestampAndOffset, that.timestampAndOffset); - } - - @Override - public int hashCode() { - int result = Objects.hashCode(exception); - result = 31 * result + Objects.hashCode(timestampAndOffset); - return result; - } } } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateEntry.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateEntry.java index a3248998af23e..b813aa89158ab 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateEntry.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateEntry.java @@ -60,7 +60,7 @@ public int firstSeq() { } public int lastSeq() { - return isEmpty() ? RecordBatch.NO_SEQUENCE : batchMetadata.getLast().lastSeq; + return isEmpty() ? RecordBatch.NO_SEQUENCE : batchMetadata.getLast().lastSeq(); } public long firstDataOffset() { @@ -68,11 +68,11 @@ public long firstDataOffset() { } public long lastDataOffset() { - return isEmpty() ? -1L : batchMetadata.getLast().lastOffset; + return isEmpty() ? -1L : batchMetadata.getLast().lastOffset(); } public int lastOffsetDelta() { - return isEmpty() ? 0 : batchMetadata.getLast().offsetDelta; + return isEmpty() ? 0 : batchMetadata.getLast().offsetDelta(); } public boolean isEmpty() { @@ -138,7 +138,7 @@ public Optional findDuplicateBatch(RecordBatch batch) { // Return the batch metadata of the cached batch having the exact sequence range, if any. Optional batchWithSequenceRange(int firstSeq, int lastSeq) { - Stream duplicate = batchMetadata.stream().filter(metadata -> firstSeq == metadata.firstSeq() && lastSeq == metadata.lastSeq); + Stream duplicate = batchMetadata.stream().filter(metadata -> firstSeq == metadata.firstSeq() && lastSeq == metadata.lastSeq()); return duplicate.findFirst(); } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java index 77826359ee88b..0cb344f5ef0a0 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java @@ -524,9 +524,9 @@ public void truncateFullyAndStartAt(long offset) throws IOException { * transaction index, but the completion must be done only after successfully appending to the index. */ public long lastStableOffset(CompletedTxn completedTxn) { - return findNextIncompleteTxn(completedTxn.producerId) + return findNextIncompleteTxn(completedTxn.producerId()) .map(x -> x.firstOffset.messageOffset) - .orElse(completedTxn.lastOffset + 1); + .orElse(completedTxn.lastOffset() + 1); } private Optional findNextIncompleteTxn(long producerId) { @@ -543,13 +543,13 @@ private Optional findNextIncompleteTxn(long producerId) { * advancing the first unstable offset. */ public void completeTxn(CompletedTxn completedTxn) { - TxnMetadata txnMetadata = ongoingTxns.remove(completedTxn.firstOffset); + TxnMetadata txnMetadata = ongoingTxns.remove(completedTxn.firstOffset()); if (txnMetadata == null) throw new IllegalArgumentException("Attempted to complete transaction " + completedTxn + " on partition " + topicPartition + " which was not started"); - txnMetadata.lastOffset = OptionalLong.of(completedTxn.lastOffset); - unreplicatedTxns.put(completedTxn.firstOffset, txnMetadata); + txnMetadata.lastOffset = OptionalLong.of(completedTxn.lastOffset()); + unreplicatedTxns.put(completedTxn.firstOffset(), txnMetadata); updateOldestTxnTimestamp(); } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java index db90f494b9174..325080fb0a894 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java @@ -423,7 +423,7 @@ private RemoteIndexCache.Entry createCacheEntry(RemoteLogSegmentMetadata remoteL public int lookupOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) { lock.readLock().lock(); try { - return getIndexEntry(remoteLogSegmentMetadata).lookupOffset(offset).position; + return getIndexEntry(remoteLogSegmentMetadata).lookupOffset(offset).position(); } finally { lock.readLock().unlock(); } @@ -432,7 +432,7 @@ public int lookupOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long public int lookupTimestamp(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long timestamp, long startingOffset) { lock.readLock().lock(); try { - return getIndexEntry(remoteLogSegmentMetadata).lookupTimestamp(timestamp, startingOffset).position; + return getIndexEntry(remoteLogSegmentMetadata).lookupTimestamp(timestamp, startingOffset).position(); } finally { lock.readLock().unlock(); } @@ -542,7 +542,7 @@ public OffsetPosition lookupTimestamp(long timestamp, long startingOffset) { if (cleanStarted) throw new IllegalStateException("This entry is marked for cleanup"); TimestampOffset timestampOffset = timeIndex.lookup(timestamp); - return offsetIndex.lookup(Math.max(startingOffset, timestampOffset.offset)); + return offsetIndex.lookup(Math.max(startingOffset, timestampOffset.offset())); } finally { entryLock.readLock().unlock(); } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteLogReadResult.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteLogReadResult.java index b44b4a0aaa4a9..30b718ab0fed1 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteLogReadResult.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteLogReadResult.java @@ -18,12 +18,5 @@ import java.util.Optional; -public class RemoteLogReadResult { - public final Optional fetchDataInfo; - public final Optional error; - - public RemoteLogReadResult(Optional fetchDataInfo, Optional error) { - this.fetchDataInfo = fetchDataInfo; - this.error = error; - } +public record RemoteLogReadResult(Optional fetchDataInfo, Optional error) { } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteStorageFetchInfo.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteStorageFetchInfo.java index e02fea8c0efcb..3e70f8c7120cf 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteStorageFetchInfo.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteStorageFetchInfo.java @@ -20,22 +20,8 @@ import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.server.storage.log.FetchIsolation; -public class RemoteStorageFetchInfo { - - public final int fetchMaxBytes; - public final boolean minOneMessage; - public final TopicIdPartition topicIdPartition; - public final FetchRequest.PartitionData fetchInfo; - public final FetchIsolation fetchIsolation; - - public RemoteStorageFetchInfo(int fetchMaxBytes, boolean minOneMessage, TopicIdPartition topicIdPartition, - FetchRequest.PartitionData fetchInfo, FetchIsolation fetchIsolation) { - this.fetchMaxBytes = fetchMaxBytes; - this.minOneMessage = minOneMessage; - this.topicIdPartition = topicIdPartition; - this.fetchInfo = fetchInfo; - this.fetchIsolation = fetchIsolation; - } +public record RemoteStorageFetchInfo(int fetchMaxBytes, boolean minOneMessage, TopicIdPartition topicIdPartition, + FetchRequest.PartitionData fetchInfo, FetchIsolation fetchIsolation) { @Override public String toString() { diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/RollParams.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/RollParams.java index 29d385cd9762b..cbb934a725c64 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/RollParams.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/RollParams.java @@ -19,29 +19,8 @@ /** * A class used to hold params required to decide to rotate a log segment or not. */ -public class RollParams { - - public final long maxSegmentMs; - public final int maxSegmentBytes; - public final long maxTimestampInMessages; - public final long maxOffsetInMessages; - public final int messagesSize; - public final long now; - - public RollParams(long maxSegmentMs, - int maxSegmentBytes, - long maxTimestampInMessages, - long maxOffsetInMessages, - int messagesSize, - long now) { - - this.maxSegmentMs = maxSegmentMs; - this.maxSegmentBytes = maxSegmentBytes; - this.maxTimestampInMessages = maxTimestampInMessages; - this.maxOffsetInMessages = maxOffsetInMessages; - this.messagesSize = messagesSize; - this.now = now; - } +public record RollParams(long maxSegmentMs, int maxSegmentBytes, long maxTimestampInMessages, long maxOffsetInMessages, + int messagesSize, long now) { @Override public String toString() { diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/TimeIndex.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/TimeIndex.java index 3043c17cf8ac3..f9f426e426c1e 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/TimeIndex.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/TimeIndex.java @@ -68,14 +68,14 @@ public TimeIndex(File file, long baseOffset, int maxIndexSize, boolean writable) this.lastEntry = lastEntryFromIndexFile(); log.debug("Loaded index file {} with maxEntries = {}, maxIndexSize = {}, entries = {}, lastOffset = {}, file position = {}", - file.getAbsolutePath(), maxEntries(), maxIndexSize, entries(), lastEntry.offset, mmap().position()); + file.getAbsolutePath(), maxEntries(), maxIndexSize, entries(), lastEntry.offset(), mmap().position()); } @Override public void sanityCheck() { TimestampOffset entry = lastEntry(); - long lastTimestamp = entry.timestamp; - long lastOffset = entry.offset; + long lastTimestamp = entry.timestamp(); + long lastOffset = entry.offset(); inRemapReadLock(() -> { if (entries() != 0 && lastTimestamp < timestamp(mmap(), 0)) throw new CorruptIndexException("Corrupt time index found, time index file (" + file().getAbsolutePath() + ") has " @@ -190,17 +190,17 @@ public void maybeAppend(long timestamp, long offset, boolean skipFullCheck) { // because that could happen in the following two scenarios: // 1. A log segment is closed. // 2. LogSegment.onBecomeInactiveSegment() is called when an active log segment is rolled. - if (entries() != 0 && offset < lastEntry.offset) + if (entries() != 0 && offset < lastEntry.offset()) throw new InvalidOffsetException("Attempt to append an offset (" + offset + ") to slot " + entries() - + " no larger than the last offset appended (" + lastEntry.offset + ") to " + file().getAbsolutePath()); - if (entries() != 0 && timestamp < lastEntry.timestamp) + + " no larger than the last offset appended (" + lastEntry.offset() + ") to " + file().getAbsolutePath()); + if (entries() != 0 && timestamp < lastEntry.timestamp()) throw new IllegalStateException("Attempt to append a timestamp (" + timestamp + ") to slot " + entries() - + " no larger than the last timestamp appended (" + lastEntry.timestamp + ") to " + file().getAbsolutePath()); + + " no larger than the last timestamp appended (" + lastEntry.timestamp() + ") to " + file().getAbsolutePath()); // We only append to the time index when the timestamp is greater than the last inserted timestamp. // If all the messages are in message format v0, the timestamp will always be NoTimestamp. In that case, the time // index will be empty. - if (timestamp > lastEntry.timestamp) { + if (timestamp > lastEntry.timestamp()) { log.trace("Adding index entry {} => {} to {}.", timestamp, offset, file().getAbsolutePath()); MappedByteBuffer mmap = mmap(); mmap.putLong(timestamp); @@ -269,7 +269,7 @@ private void truncateToEntries(int entries) { super.truncateToEntries0(entries); this.lastEntry = lastEntryFromIndexFile(); log.debug("Truncated index {} to {} entries; position is now {} and last entry is now {}", - file().getAbsolutePath(), entries, mmap().position(), lastEntry.offset); + file().getAbsolutePath(), entries, mmap().position(), lastEntry.offset()); }); } } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/TimestampOffset.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/TimestampOffset.java index 9dc35ae32d4b4..fecf458ed467f 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/TimestampOffset.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/TimestampOffset.java @@ -19,25 +19,14 @@ /** * The mapping between a timestamp to a message offset. The entry means that any message whose timestamp is greater * than that timestamp must be at or after that offset. + * + * @param timestamp The max timestamp before the given offset. + * @param offset The message offset. */ -public class TimestampOffset implements IndexEntry { +public record TimestampOffset(long timestamp, long offset) implements IndexEntry { public static final TimestampOffset UNKNOWN = new TimestampOffset(-1, -1); - public final long timestamp; - public final long offset; - - /** - * Create a TimestampOffset with the provided parameters. - * - * @param timestamp The max timestamp before the given offset. - * @param offset The message offset. - */ - public TimestampOffset(long timestamp, long offset) { - this.timestamp = timestamp; - this.offset = offset; - } - @Override public long indexKey() { return timestamp; @@ -48,26 +37,6 @@ public long indexValue() { return offset; } - @Override - public boolean equals(Object o) { - if (this == o) - return true; - if (o == null || getClass() != o.getClass()) - return false; - - TimestampOffset that = (TimestampOffset) o; - - return timestamp == that.timestamp - && offset == that.offset; - } - - @Override - public int hashCode() { - int result = Long.hashCode(timestamp); - result = 31 * result + Long.hashCode(offset); - return result; - } - @Override public String toString() { return String.format("TimestampOffset(offset = %d, timestamp = %d)", diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/TransactionIndex.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/TransactionIndex.java index 076dfa0627cd8..6283012c165fd 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/TransactionIndex.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/TransactionIndex.java @@ -47,13 +47,7 @@ */ public class TransactionIndex implements Closeable { - private static class AbortedTxnWithPosition { - final AbortedTxn txn; - final int position; - AbortedTxnWithPosition(AbortedTxn txn, int position) { - this.txn = txn; - this.position = position; - } + private record AbortedTxnWithPosition(AbortedTxn txn, int position) { } private final long startOffset; @@ -232,7 +226,7 @@ private Iterable iterable(Supplier allocate) PrimitiveRef.IntRef position = PrimitiveRef.ofInt(0); - return () -> new Iterator() { + return () -> new Iterator<>() { @Override public boolean hasNext() { diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/TxnIndexSearchResult.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/TxnIndexSearchResult.java index b6c704610bcda..6f9a425ea1ab9 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/TxnIndexSearchResult.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/TxnIndexSearchResult.java @@ -19,10 +19,7 @@ import java.util.Collections; import java.util.List; -public class TxnIndexSearchResult { - public final List abortedTransactions; - public final boolean isComplete; - +public record TxnIndexSearchResult(List abortedTransactions, boolean isComplete) { public TxnIndexSearchResult(List abortedTransactions, boolean isComplete) { this.abortedTransactions = Collections.unmodifiableList(abortedTransactions); this.isComplete = isComplete; diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java index cb642bd1e4b59..ca32e4f086a61 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java @@ -362,13 +362,13 @@ public static UnifiedLog create(File dir, dir, config, segments, - offsets.recoveryPoint, - offsets.nextOffsetMetadata, + offsets.recoveryPoint(), + offsets.nextOffsetMetadata(), scheduler, time, topicPartition, logDirFailureChannel); - return new UnifiedLog(offsets.logStartOffset, + return new UnifiedLog(offsets.logStartOffset(), localLog, brokerTopicStats, producerIdExpirationCheckIntervalMs, @@ -1128,17 +1128,17 @@ private LogAppendInfo append(MemoryRecords records, requestLocal.orElseThrow(() -> new IllegalArgumentException( "requestLocal should be defined if assignOffsets is true")).bufferSupplier()); - validRecords = validateAndOffsetAssignResult.validatedRecords; - appendInfo.setMaxTimestamp(validateAndOffsetAssignResult.maxTimestampMs); + validRecords = validateAndOffsetAssignResult.validatedRecords(); + appendInfo.setMaxTimestamp(validateAndOffsetAssignResult.maxTimestampMs()); appendInfo.setLastOffset(offset.value - 1); - appendInfo.setRecordValidationStats(validateAndOffsetAssignResult.recordValidationStats); + appendInfo.setRecordValidationStats(validateAndOffsetAssignResult.recordValidationStats()); if (config().messageTimestampType == TimestampType.LOG_APPEND_TIME) { - appendInfo.setLogAppendTime(validateAndOffsetAssignResult.logAppendTimeMs); + appendInfo.setLogAppendTime(validateAndOffsetAssignResult.logAppendTimeMs()); } // re-validate message sizes if there's a possibility that they have changed (due to re-compression or message // format conversion) - if (!ignoreRecordSize && validateAndOffsetAssignResult.messageSizeMaybeChanged) { + if (!ignoreRecordSize && validateAndOffsetAssignResult.messageSizeMaybeChanged()) { validRecords.batches().forEach(batch -> { if (batch.sizeInBytes() > config().maxMessageSize()) { // we record the original message set size instead of the trimmed size @@ -1211,8 +1211,8 @@ private LogAppendInfo append(MemoryRecords records, if (result.maybeDuplicate.isPresent()) { BatchMetadata duplicate = result.maybeDuplicate.get(); appendInfo.setFirstOffset(duplicate.firstOffset()); - appendInfo.setLastOffset(duplicate.lastOffset); - appendInfo.setLogAppendTime(duplicate.timestamp); + appendInfo.setLastOffset(duplicate.lastOffset()); + appendInfo.setLogAppendTime(duplicate.timestamp()); appendInfo.setLogStartOffset(logStartOffset); } else { // Append the records, and increment the local log end offset immediately after the append because a @@ -1638,8 +1638,8 @@ public OffsetResultHolder fetchOffsetByTimestamp(long targetTimestamp, Optional< // it may not be true following a message format version bump as the epoch will not be available for // log entries written in the older format. Optional earliestEpochEntry = leaderEpochCache.earliestEntry(); - Optional epochOpt = (earliestEpochEntry.isPresent() && earliestEpochEntry.get().startOffset <= logStartOffset) - ? Optional.of(earliestEpochEntry.get().epoch) + Optional epochOpt = (earliestEpochEntry.isPresent() && earliestEpochEntry.get().startOffset() <= logStartOffset) + ? Optional.of(earliestEpochEntry.get().epoch()) : Optional.empty(); return new OffsetResultHolder(new FileRecords.TimestampAndOffset(RecordBatch.NO_TIMESTAMP, logStartOffset, epochOpt)); @@ -1681,10 +1681,10 @@ public OffsetResultHolder fetchOffsetByTimestamp(long targetTimestamp, Optional< // cache the timestamp and offset TimestampOffset maxTimestampSoFar = latestTimestampSegment.readMaxTimestampAndOffsetSoFar(); // lookup the position of batch to avoid extra I/O - OffsetPosition position = latestTimestampSegment.offsetIndex().lookup(maxTimestampSoFar.offset); + OffsetPosition position = latestTimestampSegment.offsetIndex().lookup(maxTimestampSoFar.offset()); Optional timestampAndOffsetOpt = findFirst( - latestTimestampSegment.log().batchesFrom(position.position), - item -> item.maxTimestamp() == maxTimestampSoFar.timestamp) + latestTimestampSegment.log().batchesFrom(position.position()), + item -> item.maxTimestamp() == maxTimestampSoFar.timestamp()) .flatMap(batch -> batch.offsetOfMaxTimestamp() .map(offset -> new FileRecords.TimestampAndOffset( batch.maxTimestamp(), @@ -2397,8 +2397,8 @@ private T maybeHandleIOException(Supplier msg, StorageAction splitOverflowedSegment(LogSegment segment) throws IOException { synchronized (lock) { LocalLog.SplitSegmentResult result = LocalLog.splitOverflowedSegment(segment, localLog.segments(), dir(), topicPartition(), config(), scheduler(), logDirFailureChannel(), logIdent); - deleteProducerSnapshots(result.deletedSegments, true); - return result.newSegments; + deleteProducerSnapshots(result.deletedSegments(), true); + return result.newSegments(); } } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/VerificationStateEntry.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/VerificationStateEntry.java index ef7f046cf1833..999e43d094fbd 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/VerificationStateEntry.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/VerificationStateEntry.java @@ -31,9 +31,9 @@ public class VerificationStateEntry { private final long timestamp; private final VerificationGuard verificationGuard; + private final boolean supportsEpochBump; private int lowestSequence; private short epoch; - private boolean supportsEpochBump; public VerificationStateEntry(long timestamp, int sequence, short epoch, boolean supportsEpochBump) { this.timestamp = timestamp; diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogSegmentLifecycleTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogSegmentLifecycleTest.java index 1e91625e2bddf..4b0de8f69ba02 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogSegmentLifecycleTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogSegmentLifecycleTest.java @@ -160,7 +160,7 @@ public void testRemoteLogSegmentLifeCycle() throws Exception { for (Map.Entry entry : expectedEpochEntryToMetadata.entrySet()) { EpochEntry epochEntry = entry.getKey(); Optional actualMetadataOpt = metadataManager - .remoteLogSegmentMetadata(topicIdPartition, epochEntry.epoch, epochEntry.startOffset); + .remoteLogSegmentMetadata(topicIdPartition, epochEntry.epoch(), epochEntry.startOffset()); RemoteLogSegmentMetadata expectedSegmentMetadata = entry.getValue(); if (expectedSegmentMetadata != null) { assertEquals(Optional.of(expectedSegmentMetadata), actualMetadataOpt); diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/LocalTieredStorage.java b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/LocalTieredStorage.java index 1f082a9217986..69cca4d63e00c 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/LocalTieredStorage.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/LocalTieredStorage.java @@ -537,35 +537,23 @@ private static void checkArgument(final boolean valid, final String message, fin } private EventType getEventTypeForFetch(IndexType indexType) { - switch (indexType) { - case OFFSET: - return FETCH_OFFSET_INDEX; - case TIMESTAMP: - return FETCH_TIME_INDEX; - case PRODUCER_SNAPSHOT: - return FETCH_PRODUCER_SNAPSHOT; - case TRANSACTION: - return FETCH_TRANSACTION_INDEX; - case LEADER_EPOCH: - return FETCH_LEADER_EPOCH_CHECKPOINT; - } - return FETCH_SEGMENT; + return switch (indexType) { + case OFFSET -> FETCH_OFFSET_INDEX; + case TIMESTAMP -> FETCH_TIME_INDEX; + case PRODUCER_SNAPSHOT -> FETCH_PRODUCER_SNAPSHOT; + case TRANSACTION -> FETCH_TRANSACTION_INDEX; + case LEADER_EPOCH -> FETCH_LEADER_EPOCH_CHECKPOINT; + }; } private RemoteLogSegmentFileset.RemoteLogSegmentFileType getLogSegmentFileType(IndexType indexType) { - switch (indexType) { - case OFFSET: - return OFFSET_INDEX; - case TIMESTAMP: - return TIME_INDEX; - case PRODUCER_SNAPSHOT: - return PRODUCER_SNAPSHOT; - case TRANSACTION: - return TRANSACTION_INDEX; - case LEADER_EPOCH: - return LEADER_EPOCH_CHECKPOINT; - } - return SEGMENT; + return switch (indexType) { + case OFFSET -> OFFSET_INDEX; + case TIMESTAMP -> TIME_INDEX; + case PRODUCER_SNAPSHOT -> PRODUCER_SNAPSHOT; + case TRANSACTION -> TRANSACTION_INDEX; + case LEADER_EPOCH -> LEADER_EPOCH_CHECKPOINT; + }; } public int brokerId() { diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java index 182fda9abb926..695c47717ff73 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java @@ -555,8 +555,8 @@ private void assertCopyExpectedLogSegmentsToRemote(long oldSegmentStartOffset, verify(remoteLogMetadataManager).addRemoteLogSegmentMetadata(remoteLogSegmentMetadataArg.capture()); // The old segment should only contain leader epoch [0->0, 1->100] since its offset range is [0, 149] Map expectedLeaderEpochs = new TreeMap<>(); - expectedLeaderEpochs.put(epochEntry0.epoch, epochEntry0.startOffset); - expectedLeaderEpochs.put(epochEntry1.epoch, epochEntry1.startOffset); + expectedLeaderEpochs.put(epochEntry0.epoch(), epochEntry0.startOffset()); + expectedLeaderEpochs.put(epochEntry1.epoch(), epochEntry1.startOffset()); verifyRemoteLogSegmentMetadata(remoteLogSegmentMetadataArg.getValue(), oldSegmentStartOffset, oldSegmentEndOffset, expectedLeaderEpochs); // verify copyLogSegmentData is passing the RemoteLogSegmentMetadata we created above @@ -1620,15 +1620,12 @@ private void doTestFindOffsetByTimestamp(long ts, long startOffset, int targetLe metadata.startOffset(), maxEntries * 8); TimeIndex timeIdx = new TimeIndex(new File(tpDir, metadata.startOffset() + UnifiedLog.TIME_INDEX_FILE_SUFFIX), metadata.startOffset(), maxEntries * 12); - switch (indexType) { - case OFFSET: - return Files.newInputStream(offsetIdx.file().toPath()); - case TIMESTAMP: - return Files.newInputStream(timeIdx.file().toPath()); - case TRANSACTION: - return Files.newInputStream(txnIdxFile.toPath()); - } - return null; + return switch (indexType) { + case OFFSET -> Files.newInputStream(offsetIdx.file().toPath()); + case TIMESTAMP -> Files.newInputStream(timeIdx.file().toPath()); + case TRANSACTION -> Files.newInputStream(txnIdxFile.toPath()); + default -> null; + }; }); when(remoteLogMetadataManager.listRemoteLogSegments(eq(leaderTopicIdPartition), anyInt())) @@ -3078,7 +3075,7 @@ private Map truncateAndGetLeaderEpochs(List entries, LeaderEpochFileCache cache = new LeaderEpochFileCache(null, myCheckpoint, scheduler); cache.truncateFromStartAsyncFlush(startOffset); cache.truncateFromEndAsyncFlush(endOffset); - return myCheckpoint.read().stream().collect(Collectors.toMap(e -> e.epoch, e -> e.startOffset)); + return myCheckpoint.read().stream().collect(Collectors.toMap(EpochEntry::epoch, EpochEntry::startOffset)); } @Test diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogReaderTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogReaderTest.java index efb6bb76e0598..ce80183d58d65 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogReaderTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogReaderTest.java @@ -79,9 +79,9 @@ public void testRemoteLogReaderWithoutError() throws RemoteStorageException, IOE ArgumentCaptor remoteLogReadResultArg = ArgumentCaptor.forClass(RemoteLogReadResult.class); verify(callback, times(1)).accept(remoteLogReadResultArg.capture()); RemoteLogReadResult actualRemoteLogReadResult = remoteLogReadResultArg.getValue(); - assertFalse(actualRemoteLogReadResult.error.isPresent()); - assertTrue(actualRemoteLogReadResult.fetchDataInfo.isPresent()); - assertEquals(fetchDataInfo, actualRemoteLogReadResult.fetchDataInfo.get()); + assertFalse(actualRemoteLogReadResult.error().isPresent()); + assertTrue(actualRemoteLogReadResult.fetchDataInfo().isPresent()); + assertEquals(fetchDataInfo, actualRemoteLogReadResult.fetchDataInfo().get()); // verify the record method on quota manager was called with the expected value ArgumentCaptor recordedArg = ArgumentCaptor.forClass(Double.class); @@ -112,8 +112,8 @@ public void testRemoteLogReaderWithError() throws RemoteStorageException, IOExce ArgumentCaptor remoteLogReadResultArg = ArgumentCaptor.forClass(RemoteLogReadResult.class); verify(callback, times(1)).accept(remoteLogReadResultArg.capture()); RemoteLogReadResult actualRemoteLogReadResult = remoteLogReadResultArg.getValue(); - assertTrue(actualRemoteLogReadResult.error.isPresent()); - assertFalse(actualRemoteLogReadResult.fetchDataInfo.isPresent()); + assertTrue(actualRemoteLogReadResult.error().isPresent()); + assertFalse(actualRemoteLogReadResult.fetchDataInfo().isPresent()); // verify the record method on quota manager was called with the expected value ArgumentCaptor recordedArg = ArgumentCaptor.forClass(Double.class); diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentFileset.java b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentFileset.java index 15cff539575ea..01f0a363520ec 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentFileset.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentFileset.java @@ -245,7 +245,7 @@ public static boolean deleteFilesOnly(final Collection files) { final Optional notAFile = files.stream().filter(f -> f.exists() && !f.isFile()).findAny(); if (notAFile.isPresent()) { - LOGGER.warn(format("Found unexpected directory %s. Will not delete.", notAFile.get().getAbsolutePath())); + LOGGER.warn("Found unexpected directory {}. Will not delete.", notAFile.get().getAbsolutePath()); return false; } @@ -254,13 +254,13 @@ public static boolean deleteFilesOnly(final Collection files) { public static boolean deleteQuietly(final File file) { try { - LOGGER.trace("Deleting " + file.getAbsolutePath()); + LOGGER.trace("Deleting {}", file.getAbsolutePath()); if (!file.exists()) { return true; } return file.delete(); } catch (final Exception e) { - LOGGER.error(format("Encountered error while deleting %s", file.getAbsolutePath())); + LOGGER.error("Encountered error while deleting {}", file.getAbsolutePath(), e); } return false; diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteTopicPartitionDirectory.java b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteTopicPartitionDirectory.java index 95de800fcff42..90962fe1405db 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteTopicPartitionDirectory.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteTopicPartitionDirectory.java @@ -119,7 +119,7 @@ public static RemoteTopicPartitionDirectory openTopicPartitionDirectory(final To final boolean existed = directory.exists(); if (!existed) { - LOGGER.info("Creating directory: " + directory.getAbsolutePath()); + LOGGER.info("Creating directory: {}", directory.getAbsolutePath()); directory.mkdirs(); } diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/checkpoint/CleanShutdownFileHandlerTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/checkpoint/CleanShutdownFileHandlerTest.java index e2ba45db68159..f1685f6f96c52 100644 --- a/storage/src/test/java/org/apache/kafka/storage/internals/checkpoint/CleanShutdownFileHandlerTest.java +++ b/storage/src/test/java/org/apache/kafka/storage/internals/checkpoint/CleanShutdownFileHandlerTest.java @@ -45,7 +45,7 @@ public void testCleanShutdownFileBasic() { assertDoesNotThrow(() -> cleanShutdownFileHandler.write(10L)); assertTrue(cleanShutdownFileHandler.exists()); assertEquals(OptionalLong.of(10L), cleanShutdownFileHandler.read()); - assertDoesNotThrow(() -> cleanShutdownFileHandler.delete()); + assertDoesNotThrow(cleanShutdownFileHandler::delete); assertFalse(cleanShutdownFileHandler.exists()); } @@ -56,7 +56,7 @@ public void testCleanShutdownFileNonExist() { CleanShutdownFileHandler cleanShutdownFileHandler = new CleanShutdownFileHandler(logDir.getPath()); assertDoesNotThrow(() -> cleanShutdownFileHandler.write(10L, 0)); assertTrue(cleanShutdownFileHandler.exists()); - assertDoesNotThrow(() -> cleanShutdownFileHandler.delete()); + assertDoesNotThrow(cleanShutdownFileHandler::delete); assertFalse(cleanShutdownFileHandler.exists()); assertEquals(OptionalLong.empty(), cleanShutdownFileHandler.read()); } @@ -75,7 +75,7 @@ public void testCleanShutdownFileCanParseWithUnknownFields() throws IOException assertTrue(cleanShutdownFileHandler.exists()); assertEquals(OptionalLong.of(10L), cleanShutdownFileHandler.read()); - assertDoesNotThrow(() -> cleanShutdownFileHandler.delete()); + assertDoesNotThrow(cleanShutdownFileHandler::delete); assertFalse(cleanShutdownFileHandler.exists()); } -} \ No newline at end of file +} diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/LogSegmentTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/LogSegmentTest.java index 76607803400c9..2131aaf7ed98b 100644 --- a/storage/src/test/java/org/apache/kafka/storage/internals/log/LogSegmentTest.java +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/LogSegmentTest.java @@ -824,8 +824,8 @@ record = MemoryRecords.readableRecords(buffer2); segment.append(2L, record); assertEquals(2, segment.offsetIndex().entries()); - assertEquals(1, segment.offsetIndex().entry(0).offset); - assertEquals(2, segment.offsetIndex().entry(1).offset); + assertEquals(1, segment.offsetIndex().entry(0).offset()); + assertEquals(2, segment.offsetIndex().entry(1).offset()); assertEquals(2, segment.timeIndex().entries()); assertEquals(new TimestampOffset(1, 1), segment.timeIndex().entry(0)); @@ -857,8 +857,8 @@ public void testNonMonotonicTimestampForMultipleBatchesInMemoryRecords() throws segment.append(2L, record); assertEquals(2, segment.offsetIndex().entries()); - assertEquals(1, segment.offsetIndex().entry(0).offset); - assertEquals(2, segment.offsetIndex().entry(1).offset); + assertEquals(1, segment.offsetIndex().entry(0).offset()); + assertEquals(2, segment.offsetIndex().entry(1).offset()); assertEquals(2, segment.timeIndex().entries()); assertEquals(new TimestampOffset(1, 0), segment.timeIndex().entry(0)); diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/LogValidatorTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/LogValidatorTest.java index ac062dde346d5..6ddf75e6e60a7 100644 --- a/storage/src/test/java/org/apache/kafka/storage/internals/log/LogValidatorTest.java +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/LogValidatorTest.java @@ -121,7 +121,7 @@ public void testValidationOfBatchesWithNonSequentialInnerOffsets() { } else { ValidationResult result = validateMessages(invalidRecords, version.value, CompressionType.GZIP, compression); List recordsResult = new ArrayList<>(); - result.validatedRecords.records().forEach(s -> recordsResult.add(s.offset())); + result.validatedRecords().records().forEach(s -> recordsResult.add(s.offset())); assertEquals(LongStream.range(0, numRecords).boxed().toList(), recordsResult); } }); @@ -226,7 +226,7 @@ public void testCreateTimeUpConversionV1ToV2() { RequestLocal.withThreadConfinedCaching().bufferSupplier() ); - MemoryRecords validatedRecords = validatedResults.validatedRecords; + MemoryRecords validatedRecords = validatedResults.validatedRecords(); for (RecordBatch batch : validatedRecords.batches()) { assertTrue(batch.isValid()); @@ -238,11 +238,11 @@ public void testCreateTimeUpConversionV1ToV2() { assertEquals(RecordBatch.NO_SEQUENCE, batch.baseSequence()); } - assertEquals(timestamp, validatedResults.maxTimestampMs); - assertTrue(validatedResults.messageSizeMaybeChanged, "Message size should have been changed"); + assertEquals(timestamp, validatedResults.maxTimestampMs()); + assertTrue(validatedResults.messageSizeMaybeChanged(), "Message size should have been changed"); verifyRecordValidationStats( - validatedResults.recordValidationStats, + validatedResults.recordValidationStats(), 3, records, true @@ -272,7 +272,7 @@ public void checkCreateTimeUpConversionFromV0(byte toMagic) { metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier() ); - MemoryRecords validatedRecords = validatedResults.validatedRecords; + MemoryRecords validatedRecords = validatedResults.validatedRecords(); for (RecordBatch batch : validatedRecords.batches()) { assertTrue(batch.isValid()); @@ -283,11 +283,11 @@ public void checkCreateTimeUpConversionFromV0(byte toMagic) { assertEquals(RecordBatch.NO_PRODUCER_ID, batch.producerId()); assertEquals(RecordBatch.NO_SEQUENCE, batch.baseSequence()); } - assertEquals(RecordBatch.NO_TIMESTAMP, validatedResults.maxTimestampMs, + assertEquals(RecordBatch.NO_TIMESTAMP, validatedResults.maxTimestampMs(), "Max timestamp should be " + RecordBatch.NO_TIMESTAMP); - assertTrue(validatedResults.messageSizeMaybeChanged, "Message size should have been changed"); + assertTrue(validatedResults.messageSizeMaybeChanged(), "Message size should have been changed"); - verifyRecordValidationStats(validatedResults.recordValidationStats, 3, records, true); + verifyRecordValidationStats(validatedResults.recordValidationStats(), 3, records, true); } @ParameterizedTest @@ -354,7 +354,7 @@ public void checkRecompression(byte magic) { RequestLocal.withThreadConfinedCaching().bufferSupplier() ); - MemoryRecords validatedRecords = validatingResults.validatedRecords; + MemoryRecords validatedRecords = validatingResults.validatedRecords(); int i = 0; for (RecordBatch batch : validatedRecords.batches()) { @@ -374,16 +374,16 @@ public void checkRecompression(byte magic) { } } - assertEquals(now + 1, validatingResults.maxTimestampMs, + assertEquals(now + 1, validatingResults.maxTimestampMs(), "Max timestamp should be " + (now + 1)); // Both V2 and V1 have single batch in the validated records when compression is enabled, and hence their shallow // OffsetOfMaxTimestamp is the last offset of the single batch assertEquals(1, iteratorSize(validatedRecords.batches().iterator())); - assertTrue(validatingResults.messageSizeMaybeChanged, + assertTrue(validatingResults.messageSizeMaybeChanged(), "Message size should have been changed"); - verifyRecordValidationStats(validatingResults.recordValidationStats, 3, records, true); + verifyRecordValidationStats(validatingResults.recordValidationStats(), 3, records, true); } private MemoryRecords recordsWithInvalidInnerMagic(byte batchMagicValue, byte recordMagicValue, Compression codec) { @@ -545,12 +545,12 @@ public void checkCompressed(byte magic) { RequestLocal.withThreadConfinedCaching().bufferSupplier() ); - MemoryRecords validatedRecords = validatedResults.validatedRecords; + MemoryRecords validatedRecords = validatedResults.validatedRecords(); int i = 0; for (RecordBatch batch : validatedRecords.batches()) { assertTrue(batch.isValid()); - assertEquals(batch.timestampType(), TimestampType.CREATE_TIME); + assertEquals(TimestampType.CREATE_TIME, batch.timestampType()); maybeCheckBaseTimestamp(timestampSeq.get(0), batch); assertEquals(batch.maxTimestamp(), TestUtils.toList(batch).stream().map(Record::timestamp).max(Long::compare).get()); assertEquals(producerEpoch, batch.producerEpoch()); @@ -565,11 +565,11 @@ public void checkCompressed(byte magic) { } } - assertEquals(now + 1, validatedResults.maxTimestampMs, "Max timestamp should be " + (now + 1)); + assertEquals(now + 1, validatedResults.maxTimestampMs(), "Max timestamp should be " + (now + 1)); - assertFalse(validatedResults.messageSizeMaybeChanged, "Message size should not have been changed"); + assertFalse(validatedResults.messageSizeMaybeChanged(), "Message size should not have been changed"); - verifyRecordValidationStats(validatedResults.recordValidationStats, 0, records, true); + verifyRecordValidationStats(validatedResults.recordValidationStats(), 0, records, true); } private MemoryRecords createRecords(List records, @@ -883,7 +883,7 @@ public void testAbsoluteOffsetAssignmentNonCompressed() { PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier() - ).validatedRecords, offset + ).validatedRecords(), offset ); } @@ -914,7 +914,7 @@ public void testAbsoluteOffsetAssignmentCompressed() { PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier() - ).validatedRecords, + ).validatedRecords(), offset ); } @@ -944,7 +944,7 @@ public void testRelativeOffsetAssignmentNonCompressedV1() { PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier() - ).validatedRecords; + ).validatedRecords(); checkOffsets(messageWithOffset, offset); } @@ -974,7 +974,7 @@ public void testRelativeOffsetAssignmentNonCompressedV2() { PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier() - ).validatedRecords; + ).validatedRecords(); checkOffsets(messageWithOffset, offset); } @@ -1005,7 +1005,7 @@ public void testRelativeOffsetAssignmentCompressedV1() { PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier() - ).validatedRecords; + ).validatedRecords(); checkOffsets(compressedMessagesWithOffset, offset); } @@ -1036,7 +1036,7 @@ public void testRelativeOffsetAssignmentCompressedV2() { PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier() - ).validatedRecords; + ).validatedRecords(); checkOffsets(compressedMessagesWithOffset, offset); } @@ -1067,9 +1067,9 @@ public void testOffsetAssignmentAfterUpConversionV0ToV1NonCompressed() { RequestLocal.withThreadConfinedCaching().bufferSupplier() ); - checkOffsets(validatedResults.validatedRecords, offset); + checkOffsets(validatedResults.validatedRecords(), offset); verifyRecordValidationStats( - validatedResults.recordValidationStats, + validatedResults.recordValidationStats(), 3, // numConvertedRecords records, false // compressed @@ -1102,9 +1102,9 @@ public void testOffsetAssignmentAfterUpConversionV0ToV2NonCompressed() { RequestLocal.withThreadConfinedCaching().bufferSupplier() ); - checkOffsets(validatedResults.validatedRecords, offset); + checkOffsets(validatedResults.validatedRecords(), offset); verifyRecordValidationStats( - validatedResults.recordValidationStats, + validatedResults.recordValidationStats(), 3, // numConvertedRecords records, false // compressed @@ -1138,9 +1138,9 @@ public void testOffsetAssignmentAfterUpConversionV0ToV1Compressed() { RequestLocal.withThreadConfinedCaching().bufferSupplier() ); - checkOffsets(validatedResults.validatedRecords, offset); + checkOffsets(validatedResults.validatedRecords(), offset); verifyRecordValidationStats( - validatedResults.recordValidationStats, + validatedResults.recordValidationStats(), 3, // numConvertedRecords records, true // compressed @@ -1174,9 +1174,9 @@ public void testOffsetAssignmentAfterUpConversionV0ToV2Compressed() { RequestLocal.withThreadConfinedCaching().bufferSupplier() ); - checkOffsets(validatedResults.validatedRecords, offset); + checkOffsets(validatedResults.validatedRecords(), offset); verifyRecordValidationStats( - validatedResults.recordValidationStats, + validatedResults.recordValidationStats(), 3, // numConvertedRecords records, true // compressed @@ -1231,7 +1231,7 @@ public void testControlRecordsNotCompressed() { metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier() ); - MemoryRecords validatedRecords = result.validatedRecords; + MemoryRecords validatedRecords = result.validatedRecords(); assertEquals(1, TestUtils.toList(validatedRecords.batches()).size()); assertFalse(TestUtils.toList(validatedRecords.batches()).get(0).isCompressed()); } @@ -1259,7 +1259,7 @@ public void testOffsetAssignmentAfterDownConversionV1ToV0NonCompressed() { PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier() - ).validatedRecords, offset); + ).validatedRecords(), offset); } @Test @@ -1284,7 +1284,7 @@ public void testOffsetAssignmentAfterDownConversionV1ToV0Compressed() { AppendOrigin.CLIENT ).validateMessagesAndAssignOffsets( PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier() - ).validatedRecords, offset); + ).validatedRecords(), offset); } @Test @@ -1307,7 +1307,7 @@ public void testOffsetAssignmentAfterUpConversionV1ToV2NonCompressed() { AppendOrigin.CLIENT ).validateMessagesAndAssignOffsets( PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier() - ).validatedRecords, offset); + ).validatedRecords(), offset); } @Test @@ -1331,7 +1331,7 @@ public void testOffsetAssignmentAfterUpConversionV1ToV2Compressed() { AppendOrigin.CLIENT ).validateMessagesAndAssignOffsets( PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier() - ).validatedRecords, offset); + ).validatedRecords(), offset); } @Test @@ -1355,7 +1355,7 @@ public void testOffsetAssignmentAfterDownConversionV2ToV1NonCompressed() { AppendOrigin.CLIENT ).validateMessagesAndAssignOffsets( PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier() - ).validatedRecords, offset); + ).validatedRecords(), offset); } @Test @@ -1380,7 +1380,7 @@ public void testOffsetAssignmentAfterDownConversionV2ToV1Compressed() { AppendOrigin.CLIENT ).validateMessagesAndAssignOffsets( PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier() - ).validatedRecords, offset); + ).validatedRecords(), offset); } @Test @@ -1457,7 +1457,7 @@ public void testOffsetAssignmentAfterDownConversionV2ToV0NonCompressed() { AppendOrigin.CLIENT ).validateMessagesAndAssignOffsets( PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier() - ).validatedRecords, offset); + ).validatedRecords(), offset); } @@ -1483,7 +1483,7 @@ public void testOffsetAssignmentAfterDownConversionV2ToV0Compressed() { AppendOrigin.CLIENT ).validateMessagesAndAssignOffsets( PrimitiveRef.ofLong(offset), metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier() - ).validatedRecords, offset); + ).validatedRecords(), offset); } @Test @@ -1513,7 +1513,7 @@ public void testNonIncreasingOffsetRecordBatchHasMetricsLogged() { PrimitiveRef.ofLong(0L), metricsRecorder, RequestLocal.withThreadConfinedCaching().bufferSupplier() )); - assertEquals(metricsRecorder.recordInvalidOffsetCount, 1); + assertEquals(1, metricsRecorder.recordInvalidOffsetCount); } @Test @@ -1702,8 +1702,8 @@ public void testDifferentLevelDoesNotCauseRecompression() { ); // Ensure validated records have not been changed so they are the same as the producer records - assertEquals(recordsGzipMax, result.validatedRecords); - assertNotEquals(recordsGzipMin, result.validatedRecords); + assertEquals(recordsGzipMax, result.validatedRecords()); + assertNotEquals(recordsGzipMin, result.validatedRecords()); } @Test @@ -1740,7 +1740,7 @@ public void testDifferentCodecCausesRecompression() { ); // Ensure validated records have been recompressed and match lz4 min level - assertEquals(recordsLz4Min, result.validatedRecords); + assertEquals(recordsLz4Min, result.validatedRecords()); } @ParameterizedTest @@ -1798,7 +1798,7 @@ public void checkNonCompressed(byte magic) { RequestLocal.withThreadConfinedCaching().bufferSupplier() ); - MemoryRecords validatedRecords = validatingResults.validatedRecords; + MemoryRecords validatedRecords = validatingResults.validatedRecords(); int i = 0; for (RecordBatch batch : validatedRecords.batches()) { @@ -1820,7 +1820,7 @@ public void checkNonCompressed(byte magic) { } assertEquals(i, offsetCounter.value); - assertEquals(now + 1, validatingResults.maxTimestampMs, + assertEquals(now + 1, validatingResults.maxTimestampMs(), "Max timestamp should be " + (now + 1)); if (magic >= RecordBatch.MAGIC_VALUE_V2) { @@ -1829,9 +1829,9 @@ public void checkNonCompressed(byte magic) { assertEquals(3, iteratorSize(records.batches().iterator())); } - assertFalse(validatingResults.messageSizeMaybeChanged, + assertFalse(validatingResults.messageSizeMaybeChanged(), "Message size should not have been changed"); - verifyRecordValidationStats(validatingResults.recordValidationStats, 0, records, false); + verifyRecordValidationStats(validatingResults.recordValidationStats(), 0, records, false); } private void assertInvalidBatchCountOverrides(int lastOffsetDelta, int count) { @@ -1890,7 +1890,7 @@ public void checkLogAppendTimeWithoutRecompression(byte magic) { RequestLocal.withThreadConfinedCaching().bufferSupplier() ); - MemoryRecords validatedRecords = validatedResults.validatedRecords; + MemoryRecords validatedRecords = validatedResults.validatedRecords(); assertEquals(records.sizeInBytes(), validatedRecords.sizeInBytes(), "message set size should not change"); long now = mockTime.milliseconds(); @@ -1898,12 +1898,12 @@ public void checkLogAppendTimeWithoutRecompression(byte magic) { validateLogAppendTime(now, 1234L, batch); assertTrue(validatedRecords.batches().iterator().next().isValid(), "MessageSet should still valid"); - assertEquals(now, validatedResults.maxTimestampMs, + assertEquals(now, validatedResults.maxTimestampMs(), "Max timestamp should be " + now); - assertFalse(validatedResults.messageSizeMaybeChanged, + assertFalse(validatedResults.messageSizeMaybeChanged(), "Message size should not have been changed"); - verifyRecordValidationStats(validatedResults.recordValidationStats, 0, records, true); + verifyRecordValidationStats(validatedResults.recordValidationStats(), 0, records, true); } @ParameterizedTest @@ -1932,18 +1932,18 @@ public void checkLogAppendTimeWithRecompression(byte targetMagic) { RequestLocal.withThreadConfinedCaching().bufferSupplier() ); - MemoryRecords validatedRecords = validatedResults.validatedRecords; + MemoryRecords validatedRecords = validatedResults.validatedRecords(); assertEquals(iteratorSize(records.records().iterator()), iteratorSize(validatedRecords.records().iterator()), "message set size should not change"); long now = mockTime.milliseconds(); validatedRecords.batches().forEach(batch -> validateLogAppendTime(now, -1, batch)); assertTrue(validatedRecords.batches().iterator().next().isValid(), "MessageSet should still valid"); - assertEquals(now, validatedResults.maxTimestampMs, String.format("Max timestamp should be %d", now)); - assertTrue(validatedResults.messageSizeMaybeChanged, + assertEquals(now, validatedResults.maxTimestampMs(), String.format("Max timestamp should be %d", now)); + assertTrue(validatedResults.messageSizeMaybeChanged(), "Message size may have been changed"); - RecordValidationStats stats = validatedResults.recordValidationStats; + RecordValidationStats stats = validatedResults.recordValidationStats(); verifyRecordValidationStats(stats, 3, records, true); } @@ -1973,7 +1973,7 @@ public void checkLogAppendTimeNonCompressed(byte magic) { ); assertEquals(offsetCounter.value, iteratorSize(records.records().iterator())); - MemoryRecords validatedRecords = validatedResults.validatedRecords; + MemoryRecords validatedRecords = validatedResults.validatedRecords(); assertEquals(iteratorSize(records.records().iterator()), iteratorSize(validatedRecords.records().iterator()), "message set size should not change"); long now = mockTime.milliseconds(); @@ -1983,14 +1983,14 @@ public void checkLogAppendTimeNonCompressed(byte magic) { } if (magic == RecordBatch.MAGIC_VALUE_V0) { - assertEquals(RecordBatch.NO_TIMESTAMP, validatedResults.maxTimestampMs); + assertEquals(RecordBatch.NO_TIMESTAMP, validatedResults.maxTimestampMs()); } else { - assertEquals(now, validatedResults.maxTimestampMs); + assertEquals(now, validatedResults.maxTimestampMs()); } - assertFalse(validatedResults.messageSizeMaybeChanged, "Message size should not have been changed"); + assertFalse(validatedResults.messageSizeMaybeChanged(), "Message size should not have been changed"); - verifyRecordValidationStats(validatedResults.recordValidationStats, 0, records, false); + verifyRecordValidationStats(validatedResults.recordValidationStats(), 0, records, false); } /** @@ -1998,7 +1998,7 @@ public void checkLogAppendTimeNonCompressed(byte magic) { */ void validateLogAppendTime(long expectedLogAppendTime, long expectedBaseTimestamp, RecordBatch batch) { assertTrue(batch.isValid()); - assertEquals(batch.timestampType(), TimestampType.LOG_APPEND_TIME); + assertEquals(TimestampType.LOG_APPEND_TIME, batch.timestampType()); assertEquals(expectedLogAppendTime, batch.maxTimestamp(), "Unexpected max timestamp of batch $batch"); maybeCheckBaseTimestamp(expectedBaseTimestamp, batch); batch.forEach(record -> { diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/OffsetIndexTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/OffsetIndexTest.java index ad7fa5908529c..578cb4ac243c5 100644 --- a/storage/src/test/java/org/apache/kafka/storage/internals/log/OffsetIndexTest.java +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/OffsetIndexTest.java @@ -90,7 +90,7 @@ public void randomLookupTest() { rightAnswer = new OffsetPosition(index.baseOffset(), 0); } else { Map.Entry lastEntry = valMap.floorEntry((long) offset); - rightAnswer = new OffsetPosition(lastEntry.getKey(), lastEntry.getValue().position); + rightAnswer = new OffsetPosition(lastEntry.getKey(), lastEntry.getValue().position()); } assertEquals(rightAnswer, index.lookup(offset), "The index should give the same answer as the sorted map"); @@ -151,7 +151,7 @@ public void testFetchUpperBoundOffset() { assertEquals(Optional.empty(), index.fetchUpperBoundOffset(first, 5)); Stream.of(first, second, third, fourth) - .forEach(offsetPosition -> index.append(offsetPosition.offset, offsetPosition.position)); + .forEach(offsetPosition -> index.append(offsetPosition.offset(), offsetPosition.position())); assertEquals(Optional.of(second), index.fetchUpperBoundOffset(first, 5)); assertEquals(Optional.of(second), index.fetchUpperBoundOffset(first, 10)); @@ -167,13 +167,13 @@ public void testFetchUpperBoundOffset() { public void testReopen() throws IOException { OffsetPosition first = new OffsetPosition(51, 0); OffsetPosition sec = new OffsetPosition(52, 1); - index.append(first.offset, first.position); - index.append(sec.offset, sec.position); + index.append(first.offset(), first.position()); + index.append(sec.offset(), sec.position()); index.close(); OffsetIndex idxRo = new OffsetIndex(index.file(), index.baseOffset()); - assertEquals(first, idxRo.lookup(first.offset)); - assertEquals(sec, idxRo.lookup(sec.offset)); - assertEquals(sec.offset, idxRo.lastOffset()); + assertEquals(first, idxRo.lookup(first.offset())); + assertEquals(sec, idxRo.lookup(sec.offset())); + assertEquals(sec.offset(), idxRo.lastOffset()); assertEquals(2, idxRo.entries()); assertWriteFails("Append should fail on read-only index", idxRo, 53); } diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/OffsetMapTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/OffsetMapTest.java index e19ae2d15ab04..fd014ae38728f 100644 --- a/storage/src/test/java/org/apache/kafka/storage/internals/log/OffsetMapTest.java +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/OffsetMapTest.java @@ -36,7 +36,7 @@ public void testBasicValidation(int items) throws Exception { SkimpyOffsetMap map = new SkimpyOffsetMap(items * 48); IntStream.range(0, items).forEach(i -> assertDoesNotThrow(() -> map.put(key(i), i))); for (int i = 0; i < items; i++) { - assertEquals(map.get(key(i)), i); + assertEquals(i, map.get(key(i))); } } @@ -45,7 +45,7 @@ public void testClear() throws Exception { SkimpyOffsetMap map = new SkimpyOffsetMap(MEMORY_SIZE); IntStream.range(0, 10).forEach(i -> assertDoesNotThrow(() -> map.put(key(i), i))); for (int i = 0; i < 10; i++) { - assertEquals(map.get(key(i)), i); + assertEquals(i, map.get(key(i))); } map.clear(); for (int i = 0; i < 10; i++) { @@ -61,8 +61,8 @@ public void testGetWhenFull() throws Exception { map.put(key(i), i); i++; } - assertEquals(map.get(key(i)), -1); - assertEquals(map.get(key(i - 1)), i - 1); + assertEquals(-1, map.get(key(i))); + assertEquals(i - 1, map.get(key(i - 1))); } @Test @@ -74,9 +74,9 @@ public void testUpdateLatestOffset() throws Exception { i++; } int lastOffsets = 40; - assertEquals(map.get(key(i - 1)), i - 1); + assertEquals(i - 1, map.get(key(i - 1))); map.updateLatestOffset(lastOffsets); - assertEquals(map.get(key(lastOffsets)), lastOffsets); + assertEquals(lastOffsets, map.get(key(lastOffsets))); } @Test @@ -87,17 +87,17 @@ public void testLatestOffset() throws Exception { map.put(key(i), i); i++; } - assertEquals(map.latestOffset(), i - 1); + assertEquals(i - 1, map.latestOffset()); } @Test public void testUtilization() throws Exception { SkimpyOffsetMap map = new SkimpyOffsetMap(MEMORY_SIZE); int i = 37; - assertEquals(map.utilization(), 0.0); + assertEquals(0.0, map.utilization()); while (map.size() < map.slots()) { map.put(key(i), i); - assertEquals(map.utilization(), (double) map.size() / map.slots()); + assertEquals((double) map.size() / map.slots(), map.utilization()); i++; } } diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/ProducerStateManagerTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/ProducerStateManagerTest.java index f7420b45ffb83..faf969eb47028 100644 --- a/storage/src/test/java/org/apache/kafka/storage/internals/log/ProducerStateManagerTest.java +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/ProducerStateManagerTest.java @@ -382,10 +382,10 @@ public void updateProducerTransactionState() { CompletedTxn completedTxn = appendInfo.appendEndTxnMarker(endTxnMarker, epoch, 40L, time.milliseconds()) .orElseThrow(() -> new RuntimeException("The transaction should be completed")); - assertEquals(producerId, completedTxn.producerId); - assertEquals(16L, completedTxn.firstOffset); - assertEquals(40L, completedTxn.lastOffset); - assertFalse(completedTxn.isAborted); + assertEquals(producerId, completedTxn.producerId()); + assertEquals(16L, completedTxn.firstOffset()); + assertEquals(40L, completedTxn.lastOffset()); + assertFalse(completedTxn.isAborted()); ProducerStateEntry lastEntry = appendInfo.toEntry(); // verify that appending the transaction marker doesn't affect the metadata of the cached record batches. diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/RemoteIndexCacheTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/RemoteIndexCacheTest.java index 6111cb783a468..42002b69da6df 100644 --- a/storage/src/test/java/org/apache/kafka/storage/internals/log/RemoteIndexCacheTest.java +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/RemoteIndexCacheTest.java @@ -172,15 +172,15 @@ public void testFetchIndexFromRemoteStorage() throws RemoteStorageException { OffsetIndex offsetIndex = cache.getIndexEntry(rlsMetadata).offsetIndex(); OffsetPosition offsetPosition1 = offsetIndex.entry(1); // this call should have invoked fetchOffsetIndex, fetchTimestampIndex once - int resultPosition = cache.lookupOffset(rlsMetadata, offsetPosition1.offset); - assertEquals(offsetPosition1.position, resultPosition); + int resultPosition = cache.lookupOffset(rlsMetadata, offsetPosition1.offset()); + assertEquals(offsetPosition1.position(), resultPosition); verifyFetchIndexInvocation(1, List.of(IndexType.OFFSET, IndexType.TIMESTAMP)); // this should not cause fetching index from RemoteStorageManager as it is already fetched earlier reset(rsm); OffsetPosition offsetPosition2 = offsetIndex.entry(2); - int resultPosition2 = cache.lookupOffset(rlsMetadata, offsetPosition2.offset); - assertEquals(offsetPosition2.position, resultPosition2); + int resultPosition2 = cache.lookupOffset(rlsMetadata, offsetPosition2.offset()); + assertEquals(offsetPosition2.position(), resultPosition2); assertNotNull(cache.getIndexEntry(rlsMetadata)); verifyNoInteractions(rsm); } @@ -219,7 +219,7 @@ public void testPositionForNonExistentEntry() { // offsetIndex.lookup() returns OffsetPosition(baseOffset, 0) for offsets smaller than the last entry in the offset index. OffsetPosition nonExistentOffsetPosition = new OffsetPosition(baseOffset, 0); long lowerOffsetThanBaseOffset = offsetIndex.baseOffset() - 1; - assertEquals(nonExistentOffsetPosition.position, cache.lookupOffset(rlsMetadata, lowerOffsetThanBaseOffset)); + assertEquals(nonExistentOffsetPosition.position(), cache.lookupOffset(rlsMetadata, lowerOffsetThanBaseOffset)); } @Test @@ -1088,12 +1088,12 @@ public void testFetchIndexAccessibleWhenMarkedForCleanup() throws IOException, R name -> name.contains(segmentUuid.toString()) && name.endsWith(LogFileUtils.DELETED_FILE_SUFFIX))); // Ensure that the `indexEntry` object still able to access the renamed index files after being marked for deletion OffsetPosition offsetPosition = indexEntry.offsetIndex().entry(2); - assertEquals(offsetPosition.position, indexEntry.lookupOffset(offsetPosition.offset).position); + assertEquals(offsetPosition.position(), indexEntry.lookupOffset(offsetPosition.offset()).position()); assertNull(cache.internalCache().asMap().get(segmentUuid)); verifyFetchIndexInvocation(1); // Once the entry gets removed from cache, the subsequent call to the cache should re-fetch the entry from remote. - assertEquals(offsetPosition.position, cache.lookupOffset(rlsMetadata, offsetPosition.offset)); + assertEquals(offsetPosition.position(), cache.lookupOffset(rlsMetadata, offsetPosition.offset())); verifyFetchIndexInvocation(2); RemoteIndexCache.Entry indexEntry2 = cache.getIndexEntry(rlsMetadata); assertNotNull(indexEntry2); @@ -1111,18 +1111,18 @@ public void testFetchIndexAccessibleWhenMarkedForCleanup() throws IOException, R assertEquals(3, countFiles(cacheDir, name -> true)); assertEquals(3, countFiles(cacheDir, name -> name.contains(segmentUuid.toString()) && name.endsWith(LogFileUtils.DELETED_FILE_SUFFIX))); - assertEquals(offsetPosition.position, indexEntry.lookupOffset(offsetPosition.offset).position); - assertEquals(offsetPosition.position, indexEntry2.lookupOffset(offsetPosition.offset).position); + assertEquals(offsetPosition.position(), indexEntry.lookupOffset(offsetPosition.offset()).position()); + assertEquals(offsetPosition.position(), indexEntry2.lookupOffset(offsetPosition.offset()).position()); indexEntry.cleanup(); assertEquals(0, countFiles(cacheDir, name -> true)); - assertThrows(IllegalStateException.class, () -> indexEntry.lookupOffset(offsetPosition.offset)); - assertEquals(offsetPosition.position, indexEntry2.lookupOffset(offsetPosition.offset).position); + assertThrows(IllegalStateException.class, () -> indexEntry.lookupOffset(offsetPosition.offset())); + assertEquals(offsetPosition.position(), indexEntry2.lookupOffset(offsetPosition.offset()).position()); indexEntry2.cleanup(); assertEquals(0, countFiles(cacheDir, name -> true)); - assertThrows(IllegalStateException.class, () -> indexEntry.lookupOffset(offsetPosition.offset)); - assertThrows(IllegalStateException.class, () -> indexEntry2.lookupOffset(offsetPosition.offset)); + assertThrows(IllegalStateException.class, () -> indexEntry.lookupOffset(offsetPosition.offset())); + assertThrows(IllegalStateException.class, () -> indexEntry2.lookupOffset(offsetPosition.offset())); } private int countFiles(File cacheDir, Predicate condition) { diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/TimeIndexTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/TimeIndexTest.java index b6d74a4ada21c..44bfbc955549d 100644 --- a/storage/src/test/java/org/apache/kafka/storage/internals/log/TimeIndexTest.java +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/TimeIndexTest.java @@ -116,8 +116,8 @@ class MockTimeIndex extends TimeIndex { @Override public TimestampOffset lastEntry() { TimestampOffset superLastEntry = super.lastEntry(); - long offset = shouldCorruptOffset ? this.baseOffset() - 1 : superLastEntry.offset; - long timestamp = shouldCorruptTimestamp ? firstEntry.timestamp - 1 : superLastEntry.timestamp; + long offset = shouldCorruptOffset ? this.baseOffset() - 1 : superLastEntry.offset(); + long timestamp = shouldCorruptTimestamp ? firstEntry.timestamp() - 1 : superLastEntry.timestamp(); return new TimestampOffset(timestamp, offset); } @Override @@ -193,7 +193,7 @@ public void testParseEntry() { } private void appendEntries(Integer numEntries) { - IntStream.rangeClosed(1, numEntries).forEach(i -> idx.maybeAppend(i * 10, i * 10 + baseOffset)); + IntStream.rangeClosed(1, numEntries).forEach(i -> idx.maybeAppend(i * 10L, i * 10L + baseOffset)); } private File nonExistentTempFile() throws IOException { diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/TransactionIndexTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/TransactionIndexTest.java index b752074dbea30..c25c880cdf054 100644 --- a/storage/src/test/java/org/apache/kafka/storage/internals/log/TransactionIndexTest.java +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/TransactionIndexTest.java @@ -100,28 +100,28 @@ public void testCollectAbortedTransactions() { abortedTransactions.forEach(txn -> assertDoesNotThrow(() -> index.append(txn))); TxnIndexSearchResult result = index.collectAbortedTxns(0L, 100L); - assertEquals(abortedTransactions, result.abortedTransactions); - assertFalse(result.isComplete); + assertEquals(abortedTransactions, result.abortedTransactions()); + assertFalse(result.isComplete()); result = index.collectAbortedTxns(0L, 32); - assertEquals(abortedTransactions.subList(0, 3), result.abortedTransactions); - assertTrue(result.isComplete); + assertEquals(abortedTransactions.subList(0, 3), result.abortedTransactions()); + assertTrue(result.isComplete()); result = index.collectAbortedTxns(0L, 35); - assertEquals(abortedTransactions, result.abortedTransactions); - assertTrue(result.isComplete); + assertEquals(abortedTransactions, result.abortedTransactions()); + assertTrue(result.isComplete()); result = index.collectAbortedTxns(10, 35); - assertEquals(abortedTransactions, result.abortedTransactions); - assertTrue(result.isComplete); + assertEquals(abortedTransactions, result.abortedTransactions()); + assertTrue(result.isComplete()); result = index.collectAbortedTxns(11, 35); - assertEquals(abortedTransactions.subList(1, 4), result.abortedTransactions); - assertTrue(result.isComplete); + assertEquals(abortedTransactions.subList(1, 4), result.abortedTransactions()); + assertTrue(result.isComplete()); result = index.collectAbortedTxns(20, 41); - assertEquals(abortedTransactions.subList(2, 4), result.abortedTransactions); - assertFalse(result.isComplete); + assertEquals(abortedTransactions.subList(2, 4), result.abortedTransactions()); + assertFalse(result.isComplete()); } @Test @@ -135,13 +135,13 @@ public void testTruncate() throws IOException { abortedTransactions.forEach(txn -> assertDoesNotThrow(() -> index.append(txn))); index.truncateTo(51); - assertEquals(abortedTransactions, index.collectAbortedTxns(0L, 100L).abortedTransactions); + assertEquals(abortedTransactions, index.collectAbortedTxns(0L, 100L).abortedTransactions()); index.truncateTo(50); - assertEquals(abortedTransactions.subList(0, 3), index.collectAbortedTxns(0L, 100L).abortedTransactions); + assertEquals(abortedTransactions.subList(0, 3), index.collectAbortedTxns(0L, 100L).abortedTransactions()); index.reset(); - assertEquals(List.of(), index.collectAbortedTxns(0L, 100L).abortedTransactions); + assertEquals(List.of(), index.collectAbortedTxns(0L, 100L).abortedTransactions()); } @Test @@ -167,7 +167,7 @@ public void testRenameIndex() throws IOException { index.renameTo(renamed); index.append(new AbortedTxn(1L, 5, 15, 16)); - List abortedTxns = index.collectAbortedTxns(0L, 100L).abortedTransactions; + List abortedTxns = index.collectAbortedTxns(0L, 100L).abortedTransactions(); assertEquals(2, abortedTxns.size()); assertEquals(0, abortedTxns.get(0).firstOffset()); assertEquals(5, abortedTxns.get(1).firstOffset()); diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestBuilder.java b/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestBuilder.java index 67200077318a5..b8a75102b9cf0 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestBuilder.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestBuilder.java @@ -30,19 +30,16 @@ import org.apache.kafka.tiered.storage.actions.DeleteRecordsAction; import org.apache.kafka.tiered.storage.actions.DeleteTopicAction; import org.apache.kafka.tiered.storage.actions.EraseBrokerStorageAction; -import org.apache.kafka.tiered.storage.actions.ExpectBrokerInISRAction; import org.apache.kafka.tiered.storage.actions.ExpectEmptyRemoteStorageAction; import org.apache.kafka.tiered.storage.actions.ExpectLeaderAction; import org.apache.kafka.tiered.storage.actions.ExpectLeaderEpochCheckpointAction; import org.apache.kafka.tiered.storage.actions.ExpectListOffsetsAction; -import org.apache.kafka.tiered.storage.actions.ExpectTopicIdToMatchInRemoteStorageAction; import org.apache.kafka.tiered.storage.actions.ExpectUserTopicMappedToMetadataPartitionsAction; import org.apache.kafka.tiered.storage.actions.ProduceAction; import org.apache.kafka.tiered.storage.actions.ReassignReplicaAction; import org.apache.kafka.tiered.storage.actions.ShrinkReplicaAction; import org.apache.kafka.tiered.storage.actions.StartBrokerAction; import org.apache.kafka.tiered.storage.actions.StopBrokerAction; -import org.apache.kafka.tiered.storage.actions.UpdateBrokerConfigAction; import org.apache.kafka.tiered.storage.actions.UpdateTopicConfigAction; import org.apache.kafka.tiered.storage.specs.ConsumableSpec; import org.apache.kafka.tiered.storage.specs.DeletableSpec; @@ -83,16 +80,16 @@ public TieredStorageTestBuilder() { } public TieredStorageTestBuilder createTopic(String topic, - Integer partitionCount, - Integer replicationFactor, - Integer maxBatchCountPerSegment, + int partitionCount, + int replicationFactor, + int maxBatchCountPerSegment, Map> replicaAssignment, - Boolean enableRemoteLogStorage) { + boolean enableRemoteLogStorage) { assertTrue(maxBatchCountPerSegment >= 1, "Segments size for topic " + topic + " needs to be >= 1"); assertTrue(partitionCount >= 1, "Partition count for topic " + topic + " needs to be >= 1"); assertTrue(replicationFactor >= 1, "Replication factor for topic " + topic + " needs to be >= 1"); Map properties = new HashMap<>(); - properties.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, enableRemoteLogStorage.toString()); + properties.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, String.valueOf(enableRemoteLogStorage)); TopicSpec topicSpec = new TopicSpec(topic, partitionCount, replicationFactor, maxBatchCountPerSegment, replicaAssignment, properties); actions.add(new CreateTopicAction(topicSpec)); @@ -100,7 +97,7 @@ public TieredStorageTestBuilder createTopic(String topic, } public TieredStorageTestBuilder createPartitions(String topic, - Integer partitionCount, + int partitionCount, Map> replicaAssignment) { assertTrue(partitionCount >= 1, "Partition count for topic " + topic + " needs to be >= 1"); ExpandPartitionCountSpec spec = new ExpandPartitionCountSpec(topic, partitionCount, replicaAssignment); @@ -117,22 +114,13 @@ public TieredStorageTestBuilder updateTopicConfig(String topic, return this; } - public TieredStorageTestBuilder updateBrokerConfig(Integer brokerId, - Map configsToBeAdded, - List configsToBeDeleted) { - assertTrue(!configsToBeAdded.isEmpty() || !configsToBeDeleted.isEmpty(), - "Broker " + brokerId + " configs shouldn't be empty"); - actions.add(new UpdateBrokerConfigAction(brokerId, configsToBeAdded, configsToBeDeleted)); - return this; - } - public TieredStorageTestBuilder deleteTopic(List topics) { topics.forEach(topic -> actions.add(buildDeleteTopicAction(topic, true))); return this; } public TieredStorageTestBuilder produce(String topic, - Integer partition, + int partition, KeyValueSpec... keyValues) { assertTrue(partition >= 0, "Partition must be >= 0"); ProducableSpec spec = getOrCreateProducable(topic, partition); @@ -144,7 +132,7 @@ public TieredStorageTestBuilder produce(String topic, } public TieredStorageTestBuilder produceWithTimestamp(String topic, - Integer partition, + int partition, KeyValueSpec... keyValues) { assertTrue(partition >= 0, "Partition must be >= 0"); ProducableSpec spec = getOrCreateProducable(topic, partition); @@ -156,26 +144,18 @@ public TieredStorageTestBuilder produceWithTimestamp(String topic, return this; } - public TieredStorageTestBuilder withBatchSize(String topic, - Integer partition, - Integer batchSize) { - assertTrue(batchSize >= 1, "The size of a batch of produced records must >= 1"); - getOrCreateProducable(topic, partition).setBatchSize(batchSize); - return this; - } - public TieredStorageTestBuilder expectEarliestLocalOffsetInLogDirectory(String topic, - Integer partition, - Long earliestLocalOffset) { + int partition, + long earliestLocalOffset) { assertTrue(earliestLocalOffset >= 0, "Record offset must be >= 0"); getOrCreateProducable(topic, partition).setEarliestLocalLogOffset(earliestLocalOffset); return this; } - public TieredStorageTestBuilder expectSegmentToBeOffloaded(Integer fromBroker, + public TieredStorageTestBuilder expectSegmentToBeOffloaded(int fromBroker, String topic, - Integer partition, - Integer baseOffset, + int partition, + int baseOffset, KeyValueSpec... keyValues) { TopicPartition topicPartition = new TopicPartition(topic, partition); List> records = new ArrayList<>(); @@ -187,16 +167,11 @@ public TieredStorageTestBuilder expectSegmentToBeOffloaded(Integer fromBroker, return this; } - public TieredStorageTestBuilder expectTopicIdToMatchInRemoteStorage(String topic) { - actions.add(new ExpectTopicIdToMatchInRemoteStorageAction(topic)); - return this; - } - public TieredStorageTestBuilder consume(String topic, - Integer partition, - Long fetchOffset, - Integer expectedTotalRecord, - Integer expectedRecordsFromSecondTier) { + int partition, + long fetchOffset, + int expectedTotalRecord, + int expectedRecordsFromSecondTier) { TopicPartition topicPartition = new TopicPartition(topic, partition); assertTrue(partition >= 0, "Partition must be >= 0"); assertTrue(fetchOffset >= 0, "Fetch offset must be >=0"); @@ -211,30 +186,23 @@ public TieredStorageTestBuilder consume(String topic, } public TieredStorageTestBuilder expectLeader(String topic, - Integer partition, - Integer brokerId, - Boolean electLeader) { + int partition, + int brokerId, + boolean electLeader) { actions.add(new ExpectLeaderAction(new TopicPartition(topic, partition), brokerId, electLeader)); return this; } - public TieredStorageTestBuilder expectInIsr(String topic, - Integer partition, - Integer brokerId) { - actions.add(new ExpectBrokerInISRAction(new TopicPartition(topic, partition), brokerId)); - return this; - } - - public TieredStorageTestBuilder expectFetchFromTieredStorage(Integer fromBroker, + public TieredStorageTestBuilder expectFetchFromTieredStorage(int fromBroker, String topic, - Integer partition, - Integer segmentFetchRequestCount) { + int partition, + int segmentFetchRequestCount) { return expectFetchFromTieredStorage(fromBroker, topic, partition, new RemoteFetchCount(segmentFetchRequestCount)); } - public TieredStorageTestBuilder expectFetchFromTieredStorage(Integer fromBroker, + public TieredStorageTestBuilder expectFetchFromTieredStorage(int fromBroker, String topic, - Integer partition, + int partition, RemoteFetchCount remoteFetchRequestCount) { TopicPartition topicPartition = new TopicPartition(topic, partition); assertTrue(partition >= 0, "Partition must be >= 0"); @@ -244,11 +212,11 @@ public TieredStorageTestBuilder expectFetchFromTieredStorage(Integer fromBroker, return this; } - public TieredStorageTestBuilder expectDeletionInRemoteStorage(Integer fromBroker, + public TieredStorageTestBuilder expectDeletionInRemoteStorage(int fromBroker, String topic, - Integer partition, + int partition, LocalTieredStorageEvent.EventType eventType, - Integer eventCount) { + int eventCount) { TopicPartition topicPartition = new TopicPartition(topic, partition); deletables.computeIfAbsent(topicPartition, k -> new ArrayList<>()) .add(new DeletableSpec(fromBroker, eventType, eventCount)); @@ -260,18 +228,18 @@ public TieredStorageTestBuilder waitForRemoteLogSegmentDeletion(String topic) { return this; } - public TieredStorageTestBuilder expectLeaderEpochCheckpoint(Integer brokerId, + public TieredStorageTestBuilder expectLeaderEpochCheckpoint(int brokerId, String topic, - Integer partition, - Integer beginEpoch, - Long startOffset) { + int partition, + int beginEpoch, + long startOffset) { TopicPartition topicPartition = new TopicPartition(topic, partition); actions.add(new ExpectLeaderEpochCheckpointAction(brokerId, topicPartition, beginEpoch, startOffset)); return this; } public TieredStorageTestBuilder expectListOffsets(String topic, - Integer partition, + int partition, OffsetSpec offsetSpec, EpochEntry epochEntry) { TopicPartition topicPartition = new TopicPartition(topic, partition); @@ -279,27 +247,22 @@ public TieredStorageTestBuilder expectListOffsets(String topic, return this; } - public TieredStorageTestBuilder bounce(Integer brokerId) { + public TieredStorageTestBuilder bounce(int brokerId) { actions.add(new BounceBrokerAction(brokerId)); return this; } - public TieredStorageTestBuilder stop(Integer brokerId) { + public TieredStorageTestBuilder stop(int brokerId) { actions.add(new StopBrokerAction(brokerId)); return this; } - public TieredStorageTestBuilder start(Integer brokerId) { + public TieredStorageTestBuilder start(int brokerId) { actions.add(new StartBrokerAction(brokerId)); return this; } - public TieredStorageTestBuilder eraseBrokerStorage(Integer brokerId) { - actions.add(new EraseBrokerStorageAction(brokerId)); - return this; - } - - public TieredStorageTestBuilder eraseBrokerStorage(Integer brokerId, + public TieredStorageTestBuilder eraseBrokerStorage(int brokerId, FilenameFilter filenameFilter, boolean isStopped) { actions.add(new EraseBrokerStorageAction(brokerId, filenameFilter, isStopped)); @@ -307,14 +270,14 @@ public TieredStorageTestBuilder eraseBrokerStorage(Integer brokerId, } public TieredStorageTestBuilder expectEmptyRemoteStorage(String topic, - Integer partition) { + int partition) { TopicPartition topicPartition = new TopicPartition(topic, partition); actions.add(new ExpectEmptyRemoteStorageAction(topicPartition)); return this; } public TieredStorageTestBuilder shrinkReplica(String topic, - Integer partition, + int partition, List replicaIds) { TopicPartition topicPartition = new TopicPartition(topic, partition); actions.add(new ShrinkReplicaAction(topicPartition, replicaIds)); @@ -322,7 +285,7 @@ public TieredStorageTestBuilder shrinkReplica(String topic, } public TieredStorageTestBuilder reassignReplica(String topic, - Integer partition, + int partition, List replicaIds) { TopicPartition topicPartition = new TopicPartition(topic, partition); actions.add(new ReassignReplicaAction(topicPartition, replicaIds)); @@ -330,7 +293,7 @@ public TieredStorageTestBuilder reassignReplica(String topic, } public TieredStorageTestBuilder alterLogDir(String topic, - Integer partition, + int partition, int replicaIds) { TopicPartition topicPartition = new TopicPartition(topic, partition); actions.add(new AlterLogDirAction(topicPartition, replicaIds)); @@ -344,8 +307,8 @@ public TieredStorageTestBuilder expectUserTopicMappedToMetadataPartitions(String } public TieredStorageTestBuilder deleteRecords(String topic, - Integer partition, - Long beforeOffset) { + int partition, + long beforeOffset) { TopicPartition topicPartition = new TopicPartition(topic, partition); actions.add(new DeleteRecordsAction(topicPartition, beforeOffset, buildDeleteSegmentSpecList(topic))); return this; @@ -363,8 +326,8 @@ private void createProduceAction() { offloadables.computeIfAbsent(topicPartition, k -> new ArrayList<>()) .stream() .map(spec -> - new OffloadedSegmentSpec(spec.getSourceBrokerId(), topicPartition, spec.getBaseOffset(), - spec.getRecords())) + new OffloadedSegmentSpec(spec.sourceBrokerId(), topicPartition, spec.baseOffset(), + spec.records())) .toList(); ProduceAction action = new ProduceAction(topicPartition, offloadedSegmentSpecs, recordsToProduce, producableSpec.getBatchSize(), producableSpec.getEarliestLocalLogOffset()); @@ -379,10 +342,10 @@ private void createConsumeAction() { if (!consumables.isEmpty()) { consumables.forEach((topicPartition, consumableSpec) -> { FetchableSpec fetchableSpec = fetchables.computeIfAbsent(topicPartition, k -> new FetchableSpec(0, new RemoteFetchCount(0))); - RemoteFetchSpec remoteFetchSpec = new RemoteFetchSpec(fetchableSpec.getSourceBrokerId(), topicPartition, - fetchableSpec.getFetchCount()); - ConsumeAction action = new ConsumeAction(topicPartition, consumableSpec.getFetchOffset(), - consumableSpec.getExpectedTotalCount(), consumableSpec.getExpectedFromSecondTierCount(), + RemoteFetchSpec remoteFetchSpec = new RemoteFetchSpec(fetchableSpec.sourceBrokerId(), topicPartition, + fetchableSpec.fetchCount()); + ConsumeAction action = new ConsumeAction(topicPartition, consumableSpec.fetchOffset(), + consumableSpec.expectedTotalCount(), consumableSpec.expectedFromSecondTierCount(), remoteFetchSpec); actions.add(action); }); @@ -392,7 +355,7 @@ private void createConsumeAction() { } private ProducableSpec getOrCreateProducable(String topic, - Integer partition) { + int partition) { TopicPartition topicPartition = new TopicPartition(topic, partition); return producables.computeIfAbsent(topicPartition, k -> new ProducableSpec(new ArrayList<>(), defaultProducedBatchSize, @@ -400,7 +363,7 @@ private ProducableSpec getOrCreateProducable(String topic, } private DeleteTopicAction buildDeleteTopicAction(String topic, - Boolean shouldDelete) { + boolean shouldDelete) { return new DeleteTopicAction(topic, buildDeleteSegmentSpecList(topic), shouldDelete); } @@ -412,11 +375,11 @@ private List buildDeleteSegmentSpecList(String topic) { TopicPartition partition = e.getKey(); List deletableSpecs = e.getValue(); return deletableSpecs.stream() - .map(spec -> new RemoteDeleteSegmentSpec(spec.getSourceBrokerId(), partition, - spec.getEventType(), spec.getEventCount())); + .map(spec -> new RemoteDeleteSegmentSpec(spec.sourceBrokerId(), partition, + spec.eventType(), spec.eventCount())); }) .toList(); - deleteSegmentSpecList.forEach(spec -> deletables.remove(spec.getTopicPartition())); + deleteSegmentSpecList.forEach(spec -> deletables.remove(spec.topicPartition())); return deleteSegmentSpecList; } } diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestContext.java b/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestContext.java index 78f50f99ed797..2b344810dc653 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestContext.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestContext.java @@ -93,7 +93,7 @@ public TieredStorageTestContext(TieredStorageTestHarness harness) { } private void initClients() { - // rediscover the new bootstrap-server port incase of broker restarts + // rediscover the new bootstrap-server port in case of broker restarts ListenerName listenerName = harness.listenerName(); Properties commonOverrideProps = new Properties(); commonOverrideProps.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, harness.bootstrapServers(listenerName)); @@ -117,35 +117,35 @@ private void initContext() { public void createTopic(TopicSpec spec) throws ExecutionException, InterruptedException { NewTopic newTopic; - if (spec.getAssignment() == null || spec.getAssignment().isEmpty()) { - newTopic = new NewTopic(spec.getTopicName(), spec.getPartitionCount(), (short) spec.getReplicationFactor()); + if (spec.assignment() == null || spec.assignment().isEmpty()) { + newTopic = new NewTopic(spec.topicName(), spec.partitionCount(), (short) spec.replicationFactor()); } else { - Map> replicasAssignments = spec.getAssignment(); - newTopic = new NewTopic(spec.getTopicName(), replicasAssignments); + Map> replicasAssignments = spec.assignment(); + newTopic = new NewTopic(spec.topicName(), replicasAssignments); } - newTopic.configs(spec.getProperties()); + newTopic.configs(spec.properties()); admin.createTopics(List.of(newTopic)).all().get(); - TestUtils.waitForAllPartitionsMetadata(harness.brokers(), spec.getTopicName(), spec.getPartitionCount()); + TestUtils.waitForAllPartitionsMetadata(harness.brokers(), spec.topicName(), spec.partitionCount()); synchronized (this) { - topicSpecs.put(spec.getTopicName(), spec); + topicSpecs.put(spec.topicName(), spec); } } public void createPartitions(ExpandPartitionCountSpec spec) throws ExecutionException, InterruptedException { NewPartitions newPartitions; - if (spec.getAssignment() == null || spec.getAssignment().isEmpty()) { - newPartitions = NewPartitions.increaseTo(spec.getPartitionCount()); + if (spec.assignment() == null || spec.assignment().isEmpty()) { + newPartitions = NewPartitions.increaseTo(spec.partitionCount()); } else { - Map> assignment = spec.getAssignment(); + Map> assignment = spec.assignment(); List> newAssignments = assignment.entrySet().stream() .sorted(Map.Entry.comparingByKey()) .map(Map.Entry::getValue) .toList(); - newPartitions = NewPartitions.increaseTo(spec.getPartitionCount(), newAssignments); + newPartitions = NewPartitions.increaseTo(spec.partitionCount(), newAssignments); } - Map partitionsMap = Map.of(spec.getTopicName(), newPartitions); + Map partitionsMap = Map.of(spec.topicName(), newPartitions); admin.createPartitions(partitionsMap).all().get(); - TestUtils.waitForAllPartitionsMetadata(harness.brokers(), spec.getTopicName(), spec.getPartitionCount()); + TestUtils.waitForAllPartitionsMetadata(harness.brokers(), spec.topicName(), spec.partitionCount()); } public void updateTopicConfig(String topic, diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/actions/ConsumeAction.java b/storage/src/test/java/org/apache/kafka/tiered/storage/actions/ConsumeAction.java index 3edcc06e65281..f7a83bfe1dec4 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/actions/ConsumeAction.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/actions/ConsumeAction.java @@ -77,7 +77,7 @@ public void doExecute(TieredStorageTestContext context) throws InterruptedExcept // The latest event at the time of invocation for the interaction of type "FETCH_SEGMENT" between the // given broker and the second-tier storage is retrieved. It can be empty if an interaction of this // type has yet to happen. - LocalTieredStorageHistory history = context.tieredStorageHistory(remoteFetchSpec.getSourceBrokerId()); + LocalTieredStorageHistory history = context.tieredStorageHistory(remoteFetchSpec.sourceBrokerId()); Optional latestEventSoFar = history.latestEvent(FETCH_SEGMENT, topicPartition); Optional latestOffsetIdxEventSoFar = history.latestEvent(FETCH_OFFSET_INDEX, topicPartition); Optional latestTimeIdxEventSoFar = history.latestEvent(FETCH_TIME_INDEX, topicPartition); @@ -127,47 +127,27 @@ public void doExecute(TieredStorageTestContext context) throws InterruptedExcept // (B) Assessment of the interactions between the source broker and the second-tier storage. for (LocalTieredStorageEvent.EventType eventType : List.of(FETCH_SEGMENT, FETCH_OFFSET_INDEX, FETCH_TIME_INDEX, FETCH_TRANSACTION_INDEX)) { - Optional latestEvent; - switch (eventType) { - case FETCH_SEGMENT: - latestEvent = latestEventSoFar; - break; - case FETCH_OFFSET_INDEX: - latestEvent = latestOffsetIdxEventSoFar; - break; - case FETCH_TIME_INDEX: - latestEvent = latestTimeIdxEventSoFar; - break; - case FETCH_TRANSACTION_INDEX: - latestEvent = latestTxnIdxEventSoFar; - break; - default: - latestEvent = Optional.empty(); - } + Optional latestEvent = switch (eventType) { + case FETCH_SEGMENT -> latestEventSoFar; + case FETCH_OFFSET_INDEX -> latestOffsetIdxEventSoFar; + case FETCH_TIME_INDEX -> latestTimeIdxEventSoFar; + case FETCH_TRANSACTION_INDEX -> latestTxnIdxEventSoFar; + default -> Optional.empty(); + }; List events = history.getEvents(eventType, topicPartition); List eventsInScope = latestEvent .map(e -> events.stream().filter(event -> event.isAfter(e)).toList()) .orElse(events); - RemoteFetchCount remoteFetchCount = remoteFetchSpec.getRemoteFetchCount(); - RemoteFetchCount.FetchCountAndOp expectedCountAndOp; - switch (eventType) { - case FETCH_SEGMENT: - expectedCountAndOp = remoteFetchCount.getSegmentFetchCountAndOp(); - break; - case FETCH_OFFSET_INDEX: - expectedCountAndOp = remoteFetchCount.getOffsetIdxFetchCountAndOp(); - break; - case FETCH_TIME_INDEX: - expectedCountAndOp = remoteFetchCount.getTimeIdxFetchCountAndOp(); - break; - case FETCH_TRANSACTION_INDEX: - expectedCountAndOp = remoteFetchCount.getTxnIdxFetchCountAndOp(); - break; - default: - expectedCountAndOp = new RemoteFetchCount.FetchCountAndOp(-1, RemoteFetchCount.OperationType.EQUALS_TO); - } + RemoteFetchCount remoteFetchCount = remoteFetchSpec.remoteFetchCount(); + RemoteFetchCount.FetchCountAndOp expectedCountAndOp = switch (eventType) { + case FETCH_SEGMENT -> remoteFetchCount.getSegmentFetchCountAndOp(); + case FETCH_OFFSET_INDEX -> remoteFetchCount.getOffsetIdxFetchCountAndOp(); + case FETCH_TIME_INDEX -> remoteFetchCount.getTimeIdxFetchCountAndOp(); + case FETCH_TRANSACTION_INDEX -> remoteFetchCount.getTxnIdxFetchCountAndOp(); + default -> new RemoteFetchCount.FetchCountAndOp(-1, RemoteFetchCount.OperationType.EQUALS_TO); + }; RemoteFetchCount.OperationType exceptedOperationType = expectedCountAndOp.getOperationType(); int exceptedCount = expectedCountAndOp.getCount(); @@ -195,8 +175,8 @@ private String errorMessage( "Expected %s requests count from broker %d to tiered storage for topic-partition %s to be %s %d, " + "but actual count was %d.", eventType, - remoteFetchSpec.getSourceBrokerId(), - remoteFetchSpec.getTopicPartition(), + remoteFetchSpec.sourceBrokerId(), + remoteFetchSpec.topicPartition(), operationTypeToString(exceptedOperationType), exceptedCount, actualCount diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/actions/CreateTopicAction.java b/storage/src/test/java/org/apache/kafka/tiered/storage/actions/CreateTopicAction.java index c9d9b1b895926..e523d094b6f18 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/actions/CreateTopicAction.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/actions/CreateTopicAction.java @@ -38,11 +38,11 @@ public CreateTopicAction(TopicSpec spec) { public void doExecute(TieredStorageTestContext context) throws ExecutionException, InterruptedException { boolean enableRemoteStorage = true; Map topicConfigs = createTopicConfigForRemoteStorage( - enableRemoteStorage, spec.getMaxBatchCountPerSegment()); - topicConfigs.putAll(spec.getProperties()); + enableRemoteStorage, spec.maxBatchCountPerSegment()); + topicConfigs.putAll(spec.properties()); - spec.getProperties().clear(); - spec.getProperties().putAll(topicConfigs); + spec.properties().clear(); + spec.properties().putAll(topicConfigs); context.createTopic(spec); } diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/actions/DeleteRecordsAction.java b/storage/src/test/java/org/apache/kafka/tiered/storage/actions/DeleteRecordsAction.java index d955f282568df..f70191db5a4fb 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/actions/DeleteRecordsAction.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/actions/DeleteRecordsAction.java @@ -54,14 +54,14 @@ public void doExecute(TieredStorageTestContext context) throws InterruptedException, ExecutionException, TimeoutException { List tieredStorages = context.remoteStorageManagers(); List tieredStorageConditions = deleteSegmentSpecs.stream() - .filter(spec -> spec.getEventType() == DELETE_SEGMENT) + .filter(spec -> spec.eventType() == DELETE_SEGMENT) .map(spec -> expectEvent( tieredStorages, - spec.getEventType(), - spec.getSourceBrokerId(), - spec.getTopicPartition(), + spec.eventType(), + spec.sourceBrokerId(), + spec.topicPartition(), false, - spec.getEventCount())) + spec.eventCount())) .toList(); Map recordsToDeleteMap = diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/actions/DeleteTopicAction.java b/storage/src/test/java/org/apache/kafka/tiered/storage/actions/DeleteTopicAction.java index 0f1ae1fdee95b..364161879d089 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/actions/DeleteTopicAction.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/actions/DeleteTopicAction.java @@ -52,14 +52,14 @@ public void doExecute(TieredStorageTestContext context) throws ExecutionException, InterruptedException, TimeoutException { List tieredStorages = context.remoteStorageManagers(); List tieredStorageConditions = deleteSegmentSpecs.stream() - .filter(spec -> spec.getEventType() == DELETE_SEGMENT || spec.getEventType() == DELETE_PARTITION) + .filter(spec -> spec.eventType() == DELETE_SEGMENT || spec.eventType() == DELETE_PARTITION) .map(spec -> expectEvent( tieredStorages, - spec.getEventType(), - spec.getSourceBrokerId(), - spec.getTopicPartition(), + spec.eventType(), + spec.sourceBrokerId(), + spec.topicPartition(), false, - spec.getEventCount())) + spec.eventCount())) .toList(); if (shouldDelete) { context.deleteTopic(topic); diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/actions/ExpectLeaderEpochCheckpointAction.java b/storage/src/test/java/org/apache/kafka/tiered/storage/actions/ExpectLeaderEpochCheckpointAction.java index c1b1ae8cb2fcc..af9da3742d1b3 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/actions/ExpectLeaderEpochCheckpointAction.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/actions/ExpectLeaderEpochCheckpointAction.java @@ -57,8 +57,8 @@ public void doExecute(TieredStorageTestContext context) throws InterruptedExcept earliestEntry = leaderEpochCache.earliestEntry().orElse(null); } earliestEntryOpt.set(earliestEntry); - return earliestEntry != null && beginEpoch == earliestEntry.epoch - && startOffset == earliestEntry.startOffset; + return earliestEntry != null && beginEpoch == earliestEntry.epoch() + && startOffset == earliestEntry.startOffset(); }, 2000L, "leader-epoch-checkpoint begin-epoch: " + beginEpoch + " and start-offset: " + startOffset + " doesn't match with actual: " + earliestEntryOpt.get()); } diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/actions/ExpectListOffsetsAction.java b/storage/src/test/java/org/apache/kafka/tiered/storage/actions/ExpectListOffsetsAction.java index 3ce5e748b7226..03f3248965a7f 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/actions/ExpectListOffsetsAction.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/actions/ExpectListOffsetsAction.java @@ -52,10 +52,10 @@ public void doExecute(TieredStorageTestContext context) throws InterruptedExcept .all() .get() .get(partition); - assertEquals(expected.startOffset, listOffsetsResult.offset()); - if (expected.epoch != -1) { + assertEquals(expected.startOffset(), listOffsetsResult.offset()); + if (expected.epoch() != -1) { assertTrue(listOffsetsResult.leaderEpoch().isPresent()); - assertEquals(expected.epoch, listOffsetsResult.leaderEpoch().get()); + assertEquals(expected.epoch(), listOffsetsResult.leaderEpoch().get()); } else { assertFalse(listOffsetsResult.leaderEpoch().isPresent()); } diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/actions/ProduceAction.java b/storage/src/test/java/org/apache/kafka/tiered/storage/actions/ProduceAction.java index 5634570eb8c8f..153ff98a4d976 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/actions/ProduceAction.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/actions/ProduceAction.java @@ -80,9 +80,9 @@ public void doExecute(TieredStorageTestContext context) .map(spec -> expectEvent( tieredStorages, COPY_SEGMENT, - spec.getSourceBrokerId(), - spec.getTopicPartition(), - spec.getBaseOffset(), + spec.sourceBrokerId(), + spec.topicPartition(), + spec.baseOffset(), false)) .toList(); @@ -109,7 +109,7 @@ public void doExecute(TieredStorageTestContext context) TopicSpec topicSpec = context.topicSpec(topicPartition.topic()); long earliestLocalOffset = expectedEarliestLocalOffset != -1L ? expectedEarliestLocalOffset : startOffset + recordsToProduce.size() - - (recordsToProduce.size() % topicSpec.getMaxBatchCountPerSegment()) - 1; + - (recordsToProduce.size() % topicSpec.maxBatchCountPerSegment()) - 1; for (BrokerLocalStorage localStorage : localStorages) { // Select brokers which are assigned a replica of the topic-partition @@ -138,7 +138,7 @@ public void doExecute(TieredStorageTestContext context) tieredStorageRecords.subList((int) (startOffset - beginOffset), tieredStorageRecords.size()); List> producerRecords = offloadedSegmentSpecs.stream() - .flatMap(spec -> spec.getRecords().stream()) + .flatMap(spec -> spec.records().stream()) .toList(); compareRecords(discoveredRecords, producerRecords, topicPartition); } diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/AlterLogDirTest.java b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/AlterLogDirTest.java index d31ea80953d75..b9230b2081280 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/AlterLogDirTest.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/AlterLogDirTest.java @@ -54,7 +54,7 @@ protected void writeTestSpecifications(TieredStorageTestBuilder builder) { .produce(topicB, p0, new KeyValueSpec("k0", "v0"), new KeyValueSpec("k1", "v1"), new KeyValueSpec("k2", "v2")) // alter dir within the replica, we only expect one replicaId - .alterLogDir(topicB, p0, List.of(broker0).get(0)) + .alterLogDir(topicB, p0, broker0) // make sure the altered replica can still be elected as the leader .expectLeader(topicB, p0, broker0, true) // produce some more events and verify the earliest local offset diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/BaseDeleteSegmentsTest.java b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/BaseDeleteSegmentsTest.java index 9dab310468fad..3c6886b8871e1 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/BaseDeleteSegmentsTest.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/BaseDeleteSegmentsTest.java @@ -35,12 +35,12 @@ public int brokerCount() { @Override protected void writeTestSpecifications(TieredStorageTestBuilder builder) { - final Integer broker0 = 0; + final int broker0 = 0; final String topicA = "topicA"; - final Integer p0 = 0; - final Integer partitionCount = 1; - final Integer replicationFactor = 1; - final Integer maxBatchCountPerSegment = 1; + final int p0 = 0; + final int partitionCount = 1; + final int replicationFactor = 1; + final int maxBatchCountPerSegment = 1; final Map> replicaAssignment = null; final boolean enableRemoteLogStorage = true; final int beginEpoch = 0; diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/BaseReassignReplicaTest.java b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/BaseReassignReplicaTest.java index 018dbae28459d..ff31f0dd2485f 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/BaseReassignReplicaTest.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/BaseReassignReplicaTest.java @@ -53,10 +53,10 @@ public int numRemoteLogMetadataPartitions() { protected void writeTestSpecifications(TieredStorageTestBuilder builder) { final String topicA = "topicA"; final String topicB = "topicB"; - final Integer p0 = 0; - final Integer partitionCount = 1; - final Integer replicationFactor = 1; - final Integer maxBatchCountPerSegment = 1; + final int p0 = 0; + final int partitionCount = 1; + final int replicationFactor = 1; + final int maxBatchCountPerSegment = 1; final Map> replicaAssignment = null; final boolean enableRemoteLogStorage = true; final List metadataPartitions = new ArrayList<>(); diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/DeleteSegmentsDueToLogStartOffsetBreachTest.java b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/DeleteSegmentsDueToLogStartOffsetBreachTest.java index 25885a89a6840..8a189700bfba8 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/DeleteSegmentsDueToLogStartOffsetBreachTest.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/DeleteSegmentsDueToLogStartOffsetBreachTest.java @@ -36,13 +36,13 @@ public int brokerCount() { @Override protected void writeTestSpecifications(TieredStorageTestBuilder builder) { - final Integer broker0 = 0; - final Integer broker1 = 1; + final int broker0 = 0; + final int broker1 = 1; final String topicA = "topicA"; - final Integer p0 = 0; - final Integer partitionCount = 1; - final Integer replicationFactor = 2; - final Integer maxBatchCountPerSegment = 2; + final int p0 = 0; + final int partitionCount = 1; + final int replicationFactor = 2; + final int maxBatchCountPerSegment = 2; final Map> replicaAssignment = mkMap(mkEntry(p0, List.of(broker0, broker1))); final boolean enableRemoteLogStorage = true; final int beginEpoch = 0; diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/DeleteTopicTest.java b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/DeleteTopicTest.java index 9940547b89850..e962d03f24a45 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/DeleteTopicTest.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/DeleteTopicTest.java @@ -36,14 +36,14 @@ public int brokerCount() { @Override protected void writeTestSpecifications(TieredStorageTestBuilder builder) { - final Integer broker0 = 0; - final Integer broker1 = 1; + final int broker0 = 0; + final int broker1 = 1; final String topicA = "topicA"; - final Integer p0 = 0; - final Integer p1 = 1; - final Integer partitionCount = 2; - final Integer replicationFactor = 2; - final Integer maxBatchCountPerSegment = 1; + final int p0 = 0; + final int p1 = 1; + final int partitionCount = 2; + final int replicationFactor = 2; + final int maxBatchCountPerSegment = 1; final boolean enableRemoteLogStorage = true; final Map> assignment = mkMap( mkEntry(p0, List.of(broker0, broker1)), diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/DisableRemoteLogOnTopicTest.java b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/DisableRemoteLogOnTopicTest.java index 3d464e793801b..18d5b4e5b0282 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/DisableRemoteLogOnTopicTest.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/DisableRemoteLogOnTopicTest.java @@ -48,13 +48,13 @@ public void executeTieredStorageTest(String groupProtocol) { @Override protected void writeTestSpecifications(TieredStorageTestBuilder builder) { - final Integer broker0 = 0; - final Integer broker1 = 1; + final int broker0 = 0; + final int broker1 = 1; final String topicA = "topicA"; - final Integer p0 = 0; - final Integer partitionCount = 1; - final Integer replicationFactor = 2; - final Integer maxBatchCountPerSegment = 1; + final int p0 = 0; + final int partitionCount = 1; + final int replicationFactor = 2; + final int maxBatchCountPerSegment = 1; final boolean enableRemoteLogStorage = true; final Map> assignment = mkMap( mkEntry(p0, List.of(broker0, broker1)) diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/EnableRemoteLogOnTopicTest.java b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/EnableRemoteLogOnTopicTest.java index 1a48994ea2583..5717b91739e2c 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/EnableRemoteLogOnTopicTest.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/EnableRemoteLogOnTopicTest.java @@ -36,14 +36,14 @@ public int brokerCount() { @Override protected void writeTestSpecifications(TieredStorageTestBuilder builder) { - final Integer broker0 = 0; - final Integer broker1 = 1; + final int broker0 = 0; + final int broker1 = 1; final String topicA = "topicA"; - final Integer p0 = 0; - final Integer p1 = 1; - final Integer partitionCount = 2; - final Integer replicationFactor = 2; - final Integer maxBatchCountPerSegment = 1; + final int p0 = 0; + final int p1 = 1; + final int partitionCount = 2; + final int replicationFactor = 2; + final int maxBatchCountPerSegment = 1; final boolean enableRemoteLogStorage = false; final Map> assignment = mkMap( mkEntry(p0, List.of(broker0, broker1)), diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/FetchFromLeaderWithCorruptedCheckpointTest.java b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/FetchFromLeaderWithCorruptedCheckpointTest.java index e82aeff7f8faf..7f385fdb2a939 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/FetchFromLeaderWithCorruptedCheckpointTest.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/FetchFromLeaderWithCorruptedCheckpointTest.java @@ -39,13 +39,13 @@ public int brokerCount() { @Override protected void writeTestSpecifications(TieredStorageTestBuilder builder) { - final Integer broker0 = 0; - final Integer broker1 = 1; + final int broker0 = 0; + final int broker1 = 1; final String topicA = "topicA"; - final Integer p0 = 0; - final Integer partitionCount = 1; - final Integer replicationFactor = 2; - final Integer maxBatchCountPerSegment = 1; + final int p0 = 0; + final int partitionCount = 1; + final int replicationFactor = 2; + final int maxBatchCountPerSegment = 1; final boolean enableRemoteLogStorage = true; final Map> assignment = mkMap(mkEntry(p0, List.of(broker0, broker1))); final List checkpointFiles = List.of( diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/OffloadAndConsumeFromLeaderTest.java b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/OffloadAndConsumeFromLeaderTest.java index af21f0742e7f3..fad98604f72d8 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/OffloadAndConsumeFromLeaderTest.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/OffloadAndConsumeFromLeaderTest.java @@ -40,14 +40,14 @@ public int brokerCount() { @Override protected void writeTestSpecifications(TieredStorageTestBuilder builder) { - final Integer broker = 0; + final int broker = 0; final String topicA = "topicA"; final String topicB = "topicB"; - final Integer p0 = 0; - final Integer partitionCount = 1; - final Integer replicationFactor = 1; - final Integer oneBatchPerSegment = 1; - final Integer twoBatchPerSegment = 2; + final int p0 = 0; + final int partitionCount = 1; + final int replicationFactor = 1; + final int oneBatchPerSegment = 1; + final int twoBatchPerSegment = 2; final Map> replicaAssignment = null; final boolean enableRemoteLogStorage = true; diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/OffloadAndTxnConsumeFromLeaderTest.java b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/OffloadAndTxnConsumeFromLeaderTest.java index 9c298346cc324..ba210722c9c57 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/OffloadAndTxnConsumeFromLeaderTest.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/OffloadAndTxnConsumeFromLeaderTest.java @@ -61,12 +61,12 @@ protected void overrideConsumerConfig(Properties consumerConfig) { @Override protected void writeTestSpecifications(TieredStorageTestBuilder builder) { - final Integer broker = 0; + final int broker = 0; final String topicA = "topicA"; - final Integer p0 = 0; - final Integer partitionCount = 1; - final Integer replicationFactor = 1; - final Integer oneBatchPerSegment = 1; + final int p0 = 0; + final int partitionCount = 1; + final int replicationFactor = 1; + final int oneBatchPerSegment = 1; final Map> replicaAssignment = null; final boolean enableRemoteLogStorage = true; diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/PartitionsExpandTest.java b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/PartitionsExpandTest.java index c7ad07c0bb4fd..fea0bc6b1b59a 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/PartitionsExpandTest.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/PartitionsExpandTest.java @@ -35,15 +35,15 @@ public int brokerCount() { @Override protected void writeTestSpecifications(TieredStorageTestBuilder builder) { - final Integer broker0 = 0; - final Integer broker1 = 1; + final int broker0 = 0; + final int broker1 = 1; final String topicA = "topicA"; - final Integer p0 = 0; - final Integer p1 = 1; - final Integer p2 = 2; - final Integer partitionCount = 1; - final Integer replicationFactor = 2; - final Integer maxBatchCountPerSegment = 1; + final int p0 = 0; + final int p1 = 1; + final int p2 = 2; + final int partitionCount = 1; + final int replicationFactor = 2; + final int maxBatchCountPerSegment = 1; final boolean enableRemoteLogStorage = true; final List p0Assignment = List.of(broker0, broker1); final List p1Assignment = List.of(broker0, broker1); diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/ReassignReplicaShrinkTest.java b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/ReassignReplicaShrinkTest.java index 7b07cb59b67fb..0dee8b0bda6dd 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/ReassignReplicaShrinkTest.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/ReassignReplicaShrinkTest.java @@ -48,14 +48,14 @@ public int numRemoteLogMetadataPartitions() { @Override protected void writeTestSpecifications(TieredStorageTestBuilder builder) { - final Integer broker0 = 0; - final Integer broker1 = 1; + final int broker0 = 0; + final int broker1 = 1; final String topicA = "topicA"; - final Integer p0 = 0; - final Integer p1 = 1; - final Integer partitionCount = 2; - final Integer replicationFactor = 2; - final Integer maxBatchCountPerSegment = 1; + final int p0 = 0; + final int p1 = 1; + final int partitionCount = 2; + final int replicationFactor = 2; + final int maxBatchCountPerSegment = 1; final boolean enableRemoteLogStorage = true; final Map> replicaAssignment = mkMap( mkEntry(p0, List.of(broker0, broker1)), diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/RollAndOffloadActiveSegmentTest.java b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/RollAndOffloadActiveSegmentTest.java index 8bd81f50598e1..5009793b94b5a 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/RollAndOffloadActiveSegmentTest.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/RollAndOffloadActiveSegmentTest.java @@ -38,12 +38,12 @@ public int brokerCount() { @Override protected void writeTestSpecifications(TieredStorageTestBuilder builder) { - final Integer broker0 = 0; + final int broker0 = 0; final String topicA = "topicA"; - final Integer p0 = 0; - final Integer partitionCount = 1; - final Integer replicationFactor = 1; - final Integer maxBatchCountPerSegment = 1; + final int p0 = 0; + final int partitionCount = 1; + final int replicationFactor = 1; + final int maxBatchCountPerSegment = 1; final Map> replicaAssignment = null; final boolean enableRemoteLogStorage = true; diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/specs/ConsumableSpec.java b/storage/src/test/java/org/apache/kafka/tiered/storage/specs/ConsumableSpec.java index cbdd01d8bb237..bd6fbf74f283b 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/specs/ConsumableSpec.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/specs/ConsumableSpec.java @@ -16,33 +16,7 @@ */ package org.apache.kafka.tiered.storage.specs; -import java.util.Objects; - -public final class ConsumableSpec { - - private final Long fetchOffset; - private final Integer expectedTotalCount; - private final Integer expectedFromSecondTierCount; - - public ConsumableSpec(Long fetchOffset, - Integer expectedTotalCount, - Integer expectedFromSecondTierCount) { - this.fetchOffset = fetchOffset; - this.expectedTotalCount = expectedTotalCount; - this.expectedFromSecondTierCount = expectedFromSecondTierCount; - } - - public Long getFetchOffset() { - return fetchOffset; - } - - public Integer getExpectedTotalCount() { - return expectedTotalCount; - } - - public Integer getExpectedFromSecondTierCount() { - return expectedFromSecondTierCount; - } +public record ConsumableSpec(long fetchOffset, int expectedTotalCount, int expectedFromSecondTierCount) { @Override public String toString() { @@ -53,18 +27,4 @@ public String toString() { '}'; } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - ConsumableSpec that = (ConsumableSpec) o; - return Objects.equals(fetchOffset, that.fetchOffset) - && Objects.equals(expectedTotalCount, that.expectedTotalCount) - && Objects.equals(expectedFromSecondTierCount, that.expectedFromSecondTierCount); - } - - @Override - public int hashCode() { - return Objects.hash(fetchOffset, expectedTotalCount, expectedFromSecondTierCount); - } } diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/specs/DeletableSpec.java b/storage/src/test/java/org/apache/kafka/tiered/storage/specs/DeletableSpec.java index 514f84655680c..0b24afdf51d61 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/specs/DeletableSpec.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/specs/DeletableSpec.java @@ -18,33 +18,7 @@ import org.apache.kafka.server.log.remote.storage.LocalTieredStorageEvent; -import java.util.Objects; - -public final class DeletableSpec { - - private final Integer sourceBrokerId; - private final LocalTieredStorageEvent.EventType eventType; - private final Integer eventCount; - - public DeletableSpec(Integer sourceBrokerId, - LocalTieredStorageEvent.EventType eventType, - Integer eventCount) { - this.sourceBrokerId = sourceBrokerId; - this.eventType = eventType; - this.eventCount = eventCount; - } - - public Integer getSourceBrokerId() { - return sourceBrokerId; - } - - public LocalTieredStorageEvent.EventType getEventType() { - return eventType; - } - - public Integer getEventCount() { - return eventCount; - } +public record DeletableSpec(int sourceBrokerId, LocalTieredStorageEvent.EventType eventType, int eventCount) { @Override public String toString() { @@ -55,18 +29,4 @@ public String toString() { '}'; } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - DeletableSpec that = (DeletableSpec) o; - return Objects.equals(sourceBrokerId, that.sourceBrokerId) - && eventType == that.eventType - && Objects.equals(eventCount, that.eventCount); - } - - @Override - public int hashCode() { - return Objects.hash(sourceBrokerId, eventType, eventCount); - } } diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/specs/ExpandPartitionCountSpec.java b/storage/src/test/java/org/apache/kafka/tiered/storage/specs/ExpandPartitionCountSpec.java index 487b36e23fc9c..0336dfe1cb954 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/specs/ExpandPartitionCountSpec.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/specs/ExpandPartitionCountSpec.java @@ -18,33 +18,8 @@ import java.util.List; import java.util.Map; -import java.util.Objects; -public final class ExpandPartitionCountSpec { - - private final String topicName; - private final int partitionCount; - private final Map> assignment; - - public ExpandPartitionCountSpec(String topicName, - int partitionCount, - Map> assignment) { - this.topicName = topicName; - this.partitionCount = partitionCount; - this.assignment = assignment; - } - - public String getTopicName() { - return topicName; - } - - public int getPartitionCount() { - return partitionCount; - } - - public Map> getAssignment() { - return assignment; - } +public record ExpandPartitionCountSpec(String topicName, int partitionCount, Map> assignment) { @Override public String toString() { @@ -52,18 +27,4 @@ public String toString() { topicName, partitionCount, assignment); } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - ExpandPartitionCountSpec that = (ExpandPartitionCountSpec) o; - return partitionCount == that.partitionCount - && Objects.equals(topicName, that.topicName) - && Objects.equals(assignment, that.assignment); - } - - @Override - public int hashCode() { - return Objects.hash(topicName, partitionCount, assignment); - } } diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/specs/FetchableSpec.java b/storage/src/test/java/org/apache/kafka/tiered/storage/specs/FetchableSpec.java index 9e2f23af7a7f3..a8e43634e4041 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/specs/FetchableSpec.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/specs/FetchableSpec.java @@ -16,26 +16,7 @@ */ package org.apache.kafka.tiered.storage.specs; -import java.util.Objects; - -public final class FetchableSpec { - - private final Integer sourceBrokerId; - private final RemoteFetchCount fetchCount; - - public FetchableSpec(Integer sourceBrokerId, - RemoteFetchCount fetchCount) { - this.sourceBrokerId = sourceBrokerId; - this.fetchCount = fetchCount; - } - - public Integer getSourceBrokerId() { - return sourceBrokerId; - } - - public RemoteFetchCount getFetchCount() { - return fetchCount; - } +public record FetchableSpec(int sourceBrokerId, RemoteFetchCount fetchCount) { @Override public String toString() { @@ -45,16 +26,4 @@ public String toString() { '}'; } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - FetchableSpec that = (FetchableSpec) o; - return Objects.equals(sourceBrokerId, that.sourceBrokerId) && Objects.equals(fetchCount, that.fetchCount); - } - - @Override - public int hashCode() { - return Objects.hash(sourceBrokerId, fetchCount); - } } diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/specs/OffloadableSpec.java b/storage/src/test/java/org/apache/kafka/tiered/storage/specs/OffloadableSpec.java index e465e8c0bb520..7a18a7f78051b 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/specs/OffloadableSpec.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/specs/OffloadableSpec.java @@ -19,33 +19,9 @@ import org.apache.kafka.clients.producer.ProducerRecord; import java.util.List; -import java.util.Objects; -public final class OffloadableSpec { - - private final Integer sourceBrokerId; - private final Integer baseOffset; - private final List> records; - - public OffloadableSpec(Integer sourceBrokerId, - Integer baseOffset, - List> records) { - this.sourceBrokerId = sourceBrokerId; - this.baseOffset = baseOffset; - this.records = records; - } - - public Integer getSourceBrokerId() { - return sourceBrokerId; - } - - public Integer getBaseOffset() { - return baseOffset; - } - - public List> getRecords() { - return records; - } +public record OffloadableSpec(int sourceBrokerId, int baseOffset, + List> records) { @Override public String toString() { @@ -56,18 +32,4 @@ public String toString() { '}'; } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - OffloadableSpec that = (OffloadableSpec) o; - return Objects.equals(sourceBrokerId, that.sourceBrokerId) - && Objects.equals(baseOffset, that.baseOffset) - && Objects.equals(records, that.records); - } - - @Override - public int hashCode() { - return Objects.hash(sourceBrokerId, baseOffset, records); - } } diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/specs/OffloadedSegmentSpec.java b/storage/src/test/java/org/apache/kafka/tiered/storage/specs/OffloadedSegmentSpec.java index cbb3150153367..70dedba99dfb2 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/specs/OffloadedSegmentSpec.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/specs/OffloadedSegmentSpec.java @@ -20,48 +20,17 @@ import org.apache.kafka.common.TopicPartition; import java.util.List; -import java.util.Objects; -public final class OffloadedSegmentSpec { - - private final int sourceBrokerId; - private final TopicPartition topicPartition; - private final int baseOffset; - private final List> records; - - /** - * Specifies a remote log segment expected to be found in a second-tier storage. - * - * @param sourceBrokerId The broker which offloaded (uploaded) the segment to the second-tier storage. - * @param topicPartition The topic-partition which the remote log segment belongs to. - * @param baseOffset The base offset of the remote log segment. - * @param records The records *expected* in the remote log segment. - */ - public OffloadedSegmentSpec(int sourceBrokerId, - TopicPartition topicPartition, - int baseOffset, - List> records) { - this.sourceBrokerId = sourceBrokerId; - this.topicPartition = topicPartition; - this.baseOffset = baseOffset; - this.records = records; - } - - public int getSourceBrokerId() { - return sourceBrokerId; - } - - public TopicPartition getTopicPartition() { - return topicPartition; - } - - public int getBaseOffset() { - return baseOffset; - } - - public List> getRecords() { - return records; - } +/** + * Specifies a remote log segment expected to be found in a second-tier storage. + * + * @param sourceBrokerId The broker which offloaded (uploaded) the segment to the second-tier storage. + * @param topicPartition The topic-partition which the remote log segment belongs to. + * @param baseOffset The base offset of the remote log segment. + * @param records The records *expected* in the remote log segment. + */ +public record OffloadedSegmentSpec(int sourceBrokerId, TopicPartition topicPartition, int baseOffset, + List> records) { @Override public String toString() { @@ -69,19 +38,4 @@ public String toString() { topicPartition, sourceBrokerId, baseOffset, records.size()); } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - OffloadedSegmentSpec that = (OffloadedSegmentSpec) o; - return sourceBrokerId == that.sourceBrokerId - && baseOffset == that.baseOffset - && Objects.equals(topicPartition, that.topicPartition) - && Objects.equals(records, that.records); - } - - @Override - public int hashCode() { - return Objects.hash(sourceBrokerId, topicPartition, baseOffset, records); - } } diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/specs/ProducableSpec.java b/storage/src/test/java/org/apache/kafka/tiered/storage/specs/ProducableSpec.java index 40a58a83802cd..81a1913dfa708 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/specs/ProducableSpec.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/specs/ProducableSpec.java @@ -24,12 +24,12 @@ public final class ProducableSpec { private final List> records; - private Integer batchSize; - private Long earliestLocalLogOffset; + private int batchSize; + private long earliestLocalLogOffset; public ProducableSpec(List> records, - Integer batchSize, - Long earliestLocalLogOffset) { + int batchSize, + long earliestLocalLogOffset) { this.records = records; this.batchSize = batchSize; this.earliestLocalLogOffset = earliestLocalLogOffset; @@ -39,15 +39,15 @@ public List> getRecords() { return records; } - public Integer getBatchSize() { + public int getBatchSize() { return batchSize; } - public void setBatchSize(Integer batchSize) { + public void setBatchSize(int batchSize) { this.batchSize = batchSize; } - public Long getEarliestLocalLogOffset() { + public long getEarliestLocalLogOffset() { return earliestLocalLogOffset; } diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/specs/RemoteDeleteSegmentSpec.java b/storage/src/test/java/org/apache/kafka/tiered/storage/specs/RemoteDeleteSegmentSpec.java index 1344b70e31a40..6f949d1e6a1d2 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/specs/RemoteDeleteSegmentSpec.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/specs/RemoteDeleteSegmentSpec.java @@ -19,51 +19,19 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.server.log.remote.storage.LocalTieredStorageEvent; -import java.util.Objects; - -public final class RemoteDeleteSegmentSpec { - - private final int sourceBrokerId; - private final TopicPartition topicPartition; - private final LocalTieredStorageEvent.EventType eventType; - private final int eventCount; - - /** - * Specifies a delete segment/partition event from a second-tier storage. This is used to ensure the - * interactions between Kafka and the second-tier storage match expectations. - * - * @param sourceBrokerId The broker which deletes (a) remote log segments(s) (or) deletes the entire partition from - * the second-tier storage. - * @param topicPartition The topic-partition in which the deletion should happen. - * @param eventType Allowed event types are {@link LocalTieredStorageEvent.EventType#DELETE_PARTITION} and - * {@link LocalTieredStorageEvent.EventType#DELETE_SEGMENT} - * @param eventCount How many events are expected to interact with the second-tier storage. - */ - public RemoteDeleteSegmentSpec(int sourceBrokerId, - TopicPartition topicPartition, - LocalTieredStorageEvent.EventType eventType, - int eventCount) { - this.sourceBrokerId = sourceBrokerId; - this.topicPartition = topicPartition; - this.eventType = eventType; - this.eventCount = eventCount; - } - - public int getSourceBrokerId() { - return sourceBrokerId; - } - - public TopicPartition getTopicPartition() { - return topicPartition; - } - - public LocalTieredStorageEvent.EventType getEventType() { - return eventType; - } - - public int getEventCount() { - return eventCount; - } +/** + * Specifies a delete segment/partition event from a second-tier storage. This is used to ensure the + * interactions between Kafka and the second-tier storage match expectations. + * + * @param sourceBrokerId The broker which deletes (a) remote log segments(s) (or) deletes the entire partition from + * the second-tier storage. + * @param topicPartition The topic-partition in which the deletion should happen. + * @param eventType Allowed event types are {@link LocalTieredStorageEvent.EventType#DELETE_PARTITION} and + * {@link LocalTieredStorageEvent.EventType#DELETE_SEGMENT} + * @param eventCount How many events are expected to interact with the second-tier storage. + */ +public record RemoteDeleteSegmentSpec(int sourceBrokerId, TopicPartition topicPartition, + LocalTieredStorageEvent.EventType eventType, int eventCount) { @Override public String toString() { @@ -71,19 +39,4 @@ public String toString() { sourceBrokerId, topicPartition, eventType, eventCount); } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - RemoteDeleteSegmentSpec that = (RemoteDeleteSegmentSpec) o; - return sourceBrokerId == that.sourceBrokerId - && eventCount == that.eventCount - && Objects.equals(topicPartition, that.topicPartition) - && eventType == that.eventType; - } - - @Override - public int hashCode() { - return Objects.hash(sourceBrokerId, topicPartition, eventType, eventCount); - } } diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/specs/RemoteFetchCount.java b/storage/src/test/java/org/apache/kafka/tiered/storage/specs/RemoteFetchCount.java index 8dfc9a762b8bd..67a1d8e5b4914 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/specs/RemoteFetchCount.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/specs/RemoteFetchCount.java @@ -19,7 +19,7 @@ import java.util.Objects; public class RemoteFetchCount { - private FetchCountAndOp segmentFetchCountAndOp; + private final FetchCountAndOp segmentFetchCountAndOp; private FetchCountAndOp offsetIdxFetchCountAndOp = new FetchCountAndOp(-1); private FetchCountAndOp timeIdxFetchCountAndOp = new FetchCountAndOp(-1); private FetchCountAndOp txnIdxFetchCountAndOp = new FetchCountAndOp(-1); @@ -28,16 +28,6 @@ public RemoteFetchCount(int segmentFetchCountAndOp) { this.segmentFetchCountAndOp = new FetchCountAndOp(segmentFetchCountAndOp); } - public RemoteFetchCount(int segmentFetchCountAndOp, - int offsetIdxFetchCountAndOp, - int timeIdxFetchCountAndOp, - int txnIdxFetchCountAndOp) { - this.segmentFetchCountAndOp = new FetchCountAndOp(segmentFetchCountAndOp); - this.offsetIdxFetchCountAndOp = new FetchCountAndOp(offsetIdxFetchCountAndOp); - this.timeIdxFetchCountAndOp = new FetchCountAndOp(timeIdxFetchCountAndOp); - this.txnIdxFetchCountAndOp = new FetchCountAndOp(txnIdxFetchCountAndOp); - } - public RemoteFetchCount(FetchCountAndOp segmentFetchCountAndOp, FetchCountAndOp offsetIdxFetchCountAndOp, FetchCountAndOp timeIdxFetchCountAndOp, @@ -52,34 +42,18 @@ public FetchCountAndOp getSegmentFetchCountAndOp() { return segmentFetchCountAndOp; } - public void setSegmentFetchCountAndOp(FetchCountAndOp segmentFetchCountAndOp) { - this.segmentFetchCountAndOp = segmentFetchCountAndOp; - } - public FetchCountAndOp getOffsetIdxFetchCountAndOp() { return offsetIdxFetchCountAndOp; } - public void setOffsetIdxFetchCountAndOp(FetchCountAndOp offsetIdxFetchCountAndOp) { - this.offsetIdxFetchCountAndOp = offsetIdxFetchCountAndOp; - } - public FetchCountAndOp getTimeIdxFetchCountAndOp() { return timeIdxFetchCountAndOp; } - public void setTimeIdxFetchCountAndOp(FetchCountAndOp timeIdxFetchCountAndOp) { - this.timeIdxFetchCountAndOp = timeIdxFetchCountAndOp; - } - public FetchCountAndOp getTxnIdxFetchCountAndOp() { return txnIdxFetchCountAndOp; } - public void setTxnIdxFetchCountAndOp(FetchCountAndOp txnIdxFetchCountAndOp) { - this.txnIdxFetchCountAndOp = txnIdxFetchCountAndOp; - } - @Override public String toString() { return "RemoteFetchCount{" + diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/specs/RemoteFetchSpec.java b/storage/src/test/java/org/apache/kafka/tiered/storage/specs/RemoteFetchSpec.java index 823d4321bc73a..77e665ab6d9cf 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/specs/RemoteFetchSpec.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/specs/RemoteFetchSpec.java @@ -18,41 +18,15 @@ import org.apache.kafka.common.TopicPartition; -import java.util.Objects; - -public final class RemoteFetchSpec { - - private final int sourceBrokerId; - private final TopicPartition topicPartition; - private final RemoteFetchCount remoteFetchCount; - - /** - * Specifies a fetch (download) event from a second-tier storage. This is used to ensure the - * interactions between Kafka and the second-tier storage match expectations. - * - * @param sourceBrokerId The broker which fetched (a) remote log segment(s) from the second-tier storage. - * @param topicPartition The topic-partition which segment(s) were fetched. - * @param remoteFetchCount The number of remote log segment(s) and indexes fetched. - */ - public RemoteFetchSpec(int sourceBrokerId, - TopicPartition topicPartition, - RemoteFetchCount remoteFetchCount) { - this.sourceBrokerId = sourceBrokerId; - this.topicPartition = topicPartition; - this.remoteFetchCount = remoteFetchCount; - } - - public int getSourceBrokerId() { - return sourceBrokerId; - } - - public TopicPartition getTopicPartition() { - return topicPartition; - } - - public RemoteFetchCount getRemoteFetchCount() { - return remoteFetchCount; - } +/** + * Specifies a fetch (download) event from a second-tier storage. This is used to ensure the + * interactions between Kafka and the second-tier storage match expectations. + * + * @param sourceBrokerId The broker which fetched (a) remote log segment(s) from the second-tier storage. + * @param topicPartition The topic-partition which segment(s) were fetched. + * @param remoteFetchCount The number of remote log segment(s) and indexes fetched. + */ +public record RemoteFetchSpec(int sourceBrokerId, TopicPartition topicPartition, RemoteFetchCount remoteFetchCount) { @Override public String toString() { @@ -60,18 +34,4 @@ public String toString() { sourceBrokerId, topicPartition, remoteFetchCount); } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - RemoteFetchSpec that = (RemoteFetchSpec) o; - return sourceBrokerId == that.sourceBrokerId - && Objects.equals(remoteFetchCount, that.remoteFetchCount) - && Objects.equals(topicPartition, that.topicPartition); - } - - @Override - public int hashCode() { - return Objects.hash(sourceBrokerId, topicPartition, remoteFetchCount); - } } diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/specs/TopicSpec.java b/storage/src/test/java/org/apache/kafka/tiered/storage/specs/TopicSpec.java index 0da5bde24f407..430ea969ed449 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/specs/TopicSpec.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/specs/TopicSpec.java @@ -18,65 +18,20 @@ import java.util.List; import java.util.Map; -import java.util.Objects; -public final class TopicSpec { - - private final String topicName; - private final int partitionCount; - private final int replicationFactor; - private final int maxBatchCountPerSegment; - private final Map> assignment; - private final Map properties; - - /** - * Specifies a topic-partition with attributes customized for the purpose of tiered-storage tests. - * - * @param topicName The name of the topic. - * @param partitionCount The number of partitions for the topic. - * @param replicationFactor The replication factor of the topic. - * @param maxBatchCountPerSegment The maximal number of batch in segments of the topic. - * This allows to obtain a fixed, pre-determined size for the segment, which ease - * reasoning on the expected states of local and tiered storages. - * @param properties Configuration of the topic customized for the purpose of tiered-storage tests. - */ - public TopicSpec(String topicName, - int partitionCount, - int replicationFactor, - int maxBatchCountPerSegment, - Map> assignment, - Map properties) { - this.topicName = topicName; - this.partitionCount = partitionCount; - this.replicationFactor = replicationFactor; - this.maxBatchCountPerSegment = maxBatchCountPerSegment; - this.assignment = assignment; - this.properties = properties; - } - - public String getTopicName() { - return topicName; - } - - public int getPartitionCount() { - return partitionCount; - } - - public int getReplicationFactor() { - return replicationFactor; - } - - public int getMaxBatchCountPerSegment() { - return maxBatchCountPerSegment; - } - - public Map> getAssignment() { - return assignment; - } - - public Map getProperties() { - return properties; - } +/** + * Specifies a topic-partition with attributes customized for the purpose of tiered-storage tests. + * + * @param topicName The name of the topic. + * @param partitionCount The number of partitions for the topic. + * @param replicationFactor The replication factor of the topic. + * @param maxBatchCountPerSegment The maximal number of batch in segments of the topic. + * This allows to obtain a fixed, pre-determined size for the segment, which ease + * reasoning on the expected states of local and tiered storages. + * @param properties Configuration of the topic customized for the purpose of tiered-storage tests. + */ +public record TopicSpec(String topicName, int partitionCount, int replicationFactor, int maxBatchCountPerSegment, + Map> assignment, Map properties) { @Override public String toString() { @@ -85,22 +40,4 @@ public String toString() { topicName, partitionCount, replicationFactor, maxBatchCountPerSegment, assignment, properties); } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - TopicSpec topicSpec = (TopicSpec) o; - return partitionCount == topicSpec.partitionCount - && replicationFactor == topicSpec.replicationFactor - && maxBatchCountPerSegment == topicSpec.maxBatchCountPerSegment - && Objects.equals(topicName, topicSpec.topicName) - && Objects.equals(assignment, topicSpec.assignment) - && Objects.equals(properties, topicSpec.properties); - } - - @Override - public int hashCode() { - return Objects.hash(topicName, partitionCount, replicationFactor, maxBatchCountPerSegment, assignment, - properties); - } } diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/utils/BrokerLocalStorage.java b/storage/src/test/java/org/apache/kafka/tiered/storage/utils/BrokerLocalStorage.java index 70219e180e7c4..9ba0a73de349e 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/utils/BrokerLocalStorage.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/utils/BrokerLocalStorage.java @@ -46,10 +46,10 @@ public final class BrokerLocalStorage { private final Time time = Time.SYSTEM; public BrokerLocalStorage(Integer brokerId, - Set storageDirnames, + Set storageDirNames, Integer storageWaitTimeoutSec) { this.brokerId = brokerId; - this.brokerStorageDirectories = storageDirnames.stream().map(File::new).collect(Collectors.toSet()); + this.brokerStorageDirectories = storageDirNames.stream().map(File::new).collect(Collectors.toSet()); this.storageWaitTimeoutSec = storageWaitTimeoutSec; } @@ -220,13 +220,6 @@ private List getTopicPartitionFileNames(TopicPartition topicPartition) { .toList(); } - private static final class OffsetHolder { - private final long firstLogFileBaseOffset; - private final List partitionFiles; - - public OffsetHolder(long firstLogFileBaseOffset, List partitionFiles) { - this.firstLogFileBaseOffset = firstLogFileBaseOffset; - this.partitionFiles = partitionFiles; - } + private record OffsetHolder(long firstLogFileBaseOffset, List partitionFiles) { } } diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/utils/LocalTieredStorageOutput.java b/storage/src/test/java/org/apache/kafka/tiered/storage/utils/LocalTieredStorageOutput.java index b2f2d386558fc..505f3c4dd7cad 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/utils/LocalTieredStorageOutput.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/utils/LocalTieredStorageOutput.java @@ -92,13 +92,6 @@ private String des(Deserializer de, ByteBuffer bytes) { return de.deserialize(currentTopic, Utils.toNullableArray(bytes)).toString(); } - private static class Tuple2 { - private final T1 t1; - private final T2 t2; - - Tuple2(T1 t1, T2 t2) { - this.t1 = t1; - this.t2 = t2; - } + private record Tuple2(T1 t1, T2 t2) { } } diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/utils/RecordsKeyValueMatcher.java b/storage/src/test/java/org/apache/kafka/tiered/storage/utils/RecordsKeyValueMatcher.java index 3796426c1d1ed..7f0cea2aecff5 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/utils/RecordsKeyValueMatcher.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/utils/RecordsKeyValueMatcher.java @@ -34,6 +34,12 @@ import java.util.Collection; import java.util.Iterator; +/** + * @param The type of records used to formulate the expectations. + * @param The type of records compared against the expectations. + * @param The type of the record keys. + * @param The type of the record values. + */ public final class RecordsKeyValueMatcher extends TypeSafeDiagnosingMatcher> { private final Collection expectedRecords; @@ -43,10 +49,10 @@ public final class RecordsKeyValueMatcher extends TypeSafeDiagnosi /** * Heterogeneous matcher between alternative types of records: - * [[ProducerRecord]], [[ConsumerRecord]] or [[Record]]. + * {@link ProducerRecord}, {@link ConsumerRecord} or {@link Record}. * * It is conceptually incorrect to try to match records of different natures. - * Only a committed [[Record]] is univoque, whereas a [[ProducerRecord]] or [[ConsumerRecord]] is + * Only a committed {@link Record} is univoque, whereas a {@link ProducerRecord} or {@link ConsumerRecord} is * a physical representation of a record-to-be or viewed record. * * This matcher breaches that semantic so that testers can avoid performing manual comparisons on @@ -55,12 +61,8 @@ public final class RecordsKeyValueMatcher extends TypeSafeDiagnosi * * @param expectedRecords The records expected. * @param topicPartition The topic-partition which the records belong to. - * @param keySerde The [[Serde]] for the keys of the records. - * @param valueSerde The [[Serde]] for the values of the records. - * @tparam R1 The type of records used to formulate the expectations. - * @tparam R2 The type of records compared against the expectations. - * @tparam K The type of the record keys. - * @tparam V The type of the record values. + * @param keySerde The {@link Serde} for the keys of the records. + * @param valueSerde The {@link Serde} for the values of the records. */ public RecordsKeyValueMatcher(Collection expectedRecords, TopicPartition topicPartition, @@ -164,12 +166,8 @@ private SimpleRecord convert(Object recordCandidate) { * * @param expectedRecords The records expected. * @param topicPartition The topic-partition which the records belong to. - * @param keySerde The [[Serde]] for the keys of the records. - * @param valueSerde The [[Serde]] for the values of the records. - * @tparam R1 The type of records used to formulate the expectations. - * @tparam R2 The type of records compared against the expectations. - * @tparam K The type of the record keys. - * @tparam V The type of the record values. + * @param keySerde The {@link Serde} for the keys of the records. + * @param valueSerde The {@link Serde} for the values of the records. */ public static RecordsKeyValueMatcher correspondTo(Collection expectedRecords, TopicPartition topicPartition,