From 9761bfe425b10c48739f7e616db00a4dde9aa88d Mon Sep 17 00:00:00 2001 From: TaiJu Wu Date: Thu, 20 Mar 2025 11:57:01 +0000 Subject: [PATCH 01/24] implment production code --- .../main/java/kafka/log/remote/RemoteLogManager.java | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/kafka/log/remote/RemoteLogManager.java b/core/src/main/java/kafka/log/remote/RemoteLogManager.java index cb220baa2cdcd..4ab4774dfe17a 100644 --- a/core/src/main/java/kafka/log/remote/RemoteLogManager.java +++ b/core/src/main/java/kafka/log/remote/RemoteLogManager.java @@ -26,6 +26,7 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.errors.OffsetOutOfRangeException; import org.apache.kafka.common.errors.RetriableException; +import org.apache.kafka.common.internals.Plugin; import org.apache.kafka.common.internals.SecurityManagerCompatibility; import org.apache.kafka.common.message.FetchResponseData; import org.apache.kafka.common.metrics.Metrics; @@ -169,8 +170,10 @@ public class RemoteLogManager implements Closeable, AsyncOffsetReader { private final Metrics metrics; private final RemoteStorageManager remoteLogStorageManager; + private Plugin remoteLogStorageManagerPlugin; private final RemoteLogMetadataManager remoteLogMetadataManager; + private Plugin remoteLogMetadataManagerPlugin; private final ReentrantLock copyQuotaManagerLock = new ReentrantLock(true); private final Condition copyQuotaManagerLockCondition = copyQuotaManagerLock.newCondition(); @@ -415,6 +418,9 @@ public void startup() { // in connecting to the brokers or remote storages. configureRSM(); configureRLMM(); + // the withPluginMetrics() method will be called when the plugin is instantiated (after configure() if the plugin also implements Configurable) + remoteLogStorageManagerPlugin = Plugin.wrapInstance(remoteLogStorageManager, metrics, rlmConfig.remoteStorageManagerClassName()); + remoteLogMetadataManagerPlugin = Plugin.wrapInstance(remoteLogMetadataManager, metrics, rlmConfig.remoteLogMetadataManagerClassName()); remoteLogManagerConfigured = true; } @@ -2045,8 +2051,8 @@ public void close() { leaderCopyRLMTasks.values().forEach(RLMTaskWithFuture::cancel); leaderExpirationRLMTasks.values().forEach(RLMTaskWithFuture::cancel); followerRLMTasks.values().forEach(RLMTaskWithFuture::cancel); - Utils.closeQuietly(remoteLogStorageManager, "RemoteLogStorageManager"); - Utils.closeQuietly(remoteLogMetadataManager, "RemoteLogMetadataManager"); + Utils.closeQuietly(remoteLogStorageManagerPlugin, "remoteLogStorageManagerPlugin"); + Utils.closeQuietly(remoteLogMetadataManagerPlugin, "remoteLogMetadataManagerPlugin"); Utils.closeQuietly(indexCache, "RemoteIndexCache"); rlmCopyThreadPool.close(); From 108129df77529aa53cfa42cdaae5a40f47459ff9 Mon Sep 17 00:00:00 2001 From: TaiJu Wu Date: Sat, 22 Mar 2025 09:50:56 +0000 Subject: [PATCH 02/24] implement prod code --- .../kafka/log/remote/RemoteLogManager.java | 93 ++++++++++--------- 1 file changed, 47 insertions(+), 46 deletions(-) diff --git a/core/src/main/java/kafka/log/remote/RemoteLogManager.java b/core/src/main/java/kafka/log/remote/RemoteLogManager.java index 4ab4774dfe17a..7ce35d236c926 100644 --- a/core/src/main/java/kafka/log/remote/RemoteLogManager.java +++ b/core/src/main/java/kafka/log/remote/RemoteLogManager.java @@ -169,11 +169,9 @@ public class RemoteLogManager implements Closeable, AsyncOffsetReader { private final BrokerTopicStats brokerTopicStats; private final Metrics metrics; - private final RemoteStorageManager remoteLogStorageManager; - private Plugin remoteLogStorageManagerPlugin; + private final Plugin remoteLogStorageManagerPlugin; - private final RemoteLogMetadataManager remoteLogMetadataManager; - private Plugin remoteLogMetadataManagerPlugin; + private final Plugin remoteLogMetadataManagerPlugin; private final ReentrantLock copyQuotaManagerLock = new ReentrantLock(true); private final Condition copyQuotaManagerLockCondition = copyQuotaManagerLock.newCondition(); @@ -241,8 +239,8 @@ public RemoteLogManager(RemoteLogManagerConfig rlmConfig, this.brokerTopicStats = brokerTopicStats; this.metrics = metrics; - remoteLogStorageManager = createRemoteStorageManager(); - remoteLogMetadataManager = createRemoteLogMetadataManager(); + remoteLogStorageManagerPlugin = createRemoteStorageManagerPlugin(metrics); + remoteLogMetadataManagerPlugin = createRemoteLogMetadataManagerPlugin(metrics); rlmCopyQuotaManager = createRLMCopyQuotaManager(); rlmFetchQuotaManager = createRLMFetchQuotaManager(); @@ -251,7 +249,7 @@ public RemoteLogManager(RemoteLogManagerConfig rlmConfig, copyThrottleTimeSensor = new RLMQuotaMetrics(metrics, "remote-copy-throttle-time", RemoteLogManager.class.getSimpleName(), "The %s time in millis remote copies was throttled by a broker", INACTIVE_SENSOR_EXPIRATION_TIME_SECONDS).sensor(); - indexCache = new RemoteIndexCache(rlmConfig.remoteLogIndexFileCacheTotalSizeBytes(), remoteLogStorageManager, logDir); + indexCache = new RemoteIndexCache(rlmConfig.remoteLogIndexFileCacheTotalSizeBytes(), remoteLogStorageManagerPlugin.get(), logDir); delayInMs = rlmConfig.remoteLogManagerTaskIntervalMs(); rlmCopyThreadPool = new RLMScheduledThreadPool(rlmConfig.remoteLogManagerCopierThreadPoolSize(), "RLMCopyThreadPool", "kafka-rlm-copy-thread-pool-%d"); @@ -361,34 +359,42 @@ private T createDelegate(ClassLoader classLoader, String className) { } } - RemoteStorageManager createRemoteStorageManager() { + Plugin createRemoteStorageManagerPlugin(Metrics metrics) { return SecurityManagerCompatibility.get().doPrivileged(() -> { final String classPath = rlmConfig.remoteStorageManagerClassPath(); if (classPath != null && !classPath.trim().isEmpty()) { ChildFirstClassLoader classLoader = new ChildFirstClassLoader(classPath, this.getClass().getClassLoader()); RemoteStorageManager delegate = createDelegate(classLoader, rlmConfig.remoteStorageManagerClassName()); - return (RemoteStorageManager) new ClassLoaderAwareRemoteStorageManager(delegate, classLoader); + RemoteStorageManager rsm = new ClassLoaderAwareRemoteStorageManager(delegate, classLoader); + configureRSM(rsm); + return Plugin.wrapInstance(rsm, metrics, rlmConfig.remoteStorageManagerClassName()); } else { - return createDelegate(this.getClass().getClassLoader(), rlmConfig.remoteStorageManagerClassName()); + RemoteStorageManager rsm = createDelegate(this.getClass().getClassLoader(), rlmConfig.remoteStorageManagerClassName()); + configureRSM(rsm); + return Plugin.wrapInstance(rsm, metrics, rlmConfig.remoteStorageManagerClassName()); } }); } - private void configureRSM() { + private void configureRSM(RemoteStorageManager rsm) { final Map rsmProps = new HashMap<>(rlmConfig.remoteStorageManagerProps()); rsmProps.put(ServerConfigs.BROKER_ID_CONFIG, brokerId); - remoteLogStorageManager.configure(rsmProps); + rsm.configure(rsmProps); } - RemoteLogMetadataManager createRemoteLogMetadataManager() { + Plugin createRemoteLogMetadataManagerPlugin(Metrics metrics) { return SecurityManagerCompatibility.get().doPrivileged(() -> { final String classPath = rlmConfig.remoteLogMetadataManagerClassPath(); if (classPath != null && !classPath.trim().isEmpty()) { ClassLoader classLoader = new ChildFirstClassLoader(classPath, this.getClass().getClassLoader()); RemoteLogMetadataManager delegate = createDelegate(classLoader, rlmConfig.remoteLogMetadataManagerClassName()); - return (RemoteLogMetadataManager) new ClassLoaderAwareRemoteLogMetadataManager(delegate, classLoader); + RemoteLogMetadataManager rlmm = new ClassLoaderAwareRemoteLogMetadataManager(delegate, classLoader); + configureRLMM(rlmm); + return Plugin.wrapInstance(rlmm, metrics, rlmConfig.remoteLogMetadataManagerClassName()); } else { - return createDelegate(this.getClass().getClassLoader(), rlmConfig.remoteLogMetadataManagerClassName()); + RemoteLogMetadataManager rlmm = createDelegate(this.getClass().getClassLoader(), rlmConfig.remoteLogMetadataManagerClassName()); + configureRLMM(rlmm); + return Plugin.wrapInstance(rlmm, metrics, rlmConfig.remoteLogMetadataManagerClassName()); } }); } @@ -397,7 +403,7 @@ public void onEndPointCreated(Endpoint endpoint) { this.endpoint = Optional.of(endpoint); } - private void configureRLMM() { + private void configureRLMM(RemoteLogMetadataManager rlmm) { final Map rlmmProps = new HashMap<>(); endpoint.ifPresent(e -> { rlmmProps.put(REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX + "bootstrap.servers", e.host() + ":" + e.port()); @@ -410,17 +416,12 @@ private void configureRLMM() { rlmmProps.put(LOG_DIR_CONFIG, logDir); rlmmProps.put("cluster.id", clusterId); - remoteLogMetadataManager.configure(rlmmProps); + rlmm.configure(rlmmProps); } public void startup() { // Initialize and configure RSM and RLMM. This will start RSM, RLMM resources which may need to start resources // in connecting to the brokers or remote storages. - configureRSM(); - configureRLMM(); - // the withPluginMetrics() method will be called when the plugin is instantiated (after configure() if the plugin also implements Configurable) - remoteLogStorageManagerPlugin = Plugin.wrapInstance(remoteLogStorageManager, metrics, rlmConfig.remoteStorageManagerClassName()); - remoteLogMetadataManagerPlugin = Plugin.wrapInstance(remoteLogMetadataManager, metrics, rlmConfig.remoteLogMetadataManagerClassName()); remoteLogManagerConfigured = true; } @@ -429,7 +430,7 @@ private boolean isRemoteLogManagerConfigured() { } public RemoteStorageManager storageManager() { - return remoteLogStorageManager; + return remoteLogStorageManagerPlugin.get(); } private Stream filterPartitions(Set partitions) { @@ -478,7 +479,7 @@ public void onLeadershipChange(Set partitionsBecomeLeader, leaderPartitions.forEach((tp, __) -> cacheTopicPartitionIds(tp)); followerPartitions.forEach((tp, __) -> cacheTopicPartitionIds(tp)); - remoteLogMetadataManager.onPartitionLeadershipChanges(leaderPartitions.keySet(), followerPartitions.keySet()); + remoteLogMetadataManagerPlugin.get().onPartitionLeadershipChanges(leaderPartitions.keySet(), followerPartitions.keySet()); followerPartitions.forEach((tp, __) -> doHandleFollowerPartition(tp)); // If this node was the previous leader for the partition, then the RLMTask might be running in the @@ -562,13 +563,13 @@ public void stopPartitions(Set stopPartitions, if (!pendingActionsPartitions.isEmpty()) { pendingActionsPartitions.forEach(tpId -> topicIdByPartitionMap.remove(tpId.topicPartition())); - remoteLogMetadataManager.onStopPartitions(pendingActionsPartitions); + remoteLogMetadataManagerPlugin.get().onStopPartitions(pendingActionsPartitions); } } private void deleteRemoteLogPartition(TopicIdPartition partition) throws RemoteStorageException, ExecutionException, InterruptedException { List metadataList = new ArrayList<>(); - remoteLogMetadataManager.listRemoteLogSegments(partition).forEachRemaining(metadataList::add); + remoteLogMetadataManagerPlugin.get().listRemoteLogSegments(partition).forEachRemaining(metadataList::add); List deleteSegmentStartedEvents = metadataList.stream() .map(metadata -> @@ -581,7 +582,7 @@ private void deleteRemoteLogPartition(TopicIdPartition partition) throws RemoteS Collection deletedSegmentIds = new ArrayList<>(); for (RemoteLogSegmentMetadata metadata: metadataList) { deletedSegmentIds.add(metadata.remoteLogSegmentId().id()); - remoteLogStorageManager.deleteLogSegmentData(metadata); + remoteLogStorageManagerPlugin.get().deleteLogSegmentData(metadata); } indexCache.removeAll(deletedSegmentIds); @@ -596,7 +597,7 @@ private void deleteRemoteLogPartition(TopicIdPartition partition) throws RemoteS private CompletableFuture publishEvents(List events) throws RemoteStorageException { List> result = new ArrayList<>(); for (RemoteLogSegmentMetadataUpdate event : events) { - result.add(remoteLogMetadataManager.updateRemoteLogSegmentMetadata(event)); + result.add(remoteLogMetadataManagerPlugin.get().updateRemoteLogSegmentMetadata(event)); } return CompletableFuture.allOf(result.toArray(new CompletableFuture[0])); } @@ -608,7 +609,7 @@ public Optional fetchRemoteLogSegmentMetadata(TopicPar if (topicId == null) { throw new KafkaException("No topic id registered for topic partition: " + topicPartition); } - return remoteLogMetadataManager.remoteLogSegmentMetadata(new TopicIdPartition(topicId, topicPartition), epochForOffset, offset); + return remoteLogMetadataManagerPlugin.get().remoteLogSegmentMetadata(new TopicIdPartition(topicId, topicPartition), epochForOffset, offset); } /** @@ -628,7 +629,7 @@ public Optional fetchNextSegmentWithTxnIndex(TopicPart throw new KafkaException("No topic id registered for topic partition: " + topicPartition); } TopicIdPartition tpId = new TopicIdPartition(topicId, topicPartition); - return remoteLogMetadataManager.nextSegmentWithTxnIndex(tpId, epochForOffset, offset); + return remoteLogMetadataManagerPlugin.get().nextSegmentWithTxnIndex(tpId, epochForOffset, offset); } Optional lookupTimestamp(RemoteLogSegmentMetadata rlsMetadata, long timestamp, long startingOffset) @@ -638,7 +639,7 @@ Optional lookupTimestamp(RemoteLogSegmentMetadat InputStream remoteSegInputStream = null; try { // Search forward for the position of the last offset that is greater than or equal to the startingOffset - remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(rlsMetadata, startPos); + remoteSegInputStream = remoteLogStorageManagerPlugin.get().fetchLogSegment(rlsMetadata, startPos); RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream); while (true) { @@ -728,7 +729,7 @@ public Optional findOffsetByTimestamp(TopicParti int epoch = maybeEpoch.getAsInt(); // KAFKA-15802: Add a new API for RLMM to choose how to implement the predicate. // currently, all segments are returned and then iterated, and filtered - Iterator iterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch); + Iterator iterator = remoteLogMetadataManagerPlugin.get().listRemoteLogSegments(topicIdPartition, epoch); while (iterator.hasNext()) { RemoteLogSegmentMetadata rlsMetadata = iterator.next(); if (rlsMetadata.maxTimestampMs() >= timestamp @@ -812,7 +813,7 @@ public void run() { logger.debug("Skipping the current run for partition {} as it is cancelled", topicIdPartition); return; } - if (!remoteLogMetadataManager.isReady(topicIdPartition)) { + if (!remoteLogMetadataManagerPlugin.get().isReady(topicIdPartition)) { logger.debug("Skipping the current run for partition {} as the remote-log metadata is not ready", topicIdPartition); return; } @@ -1022,7 +1023,7 @@ private void copyLogSegment(UnifiedLog log, LogSegment segment, RemoteLogSegment segment.largestTimestamp(), brokerId, time.milliseconds(), segment.log().sizeInBytes(), segmentLeaderEpochs, isTxnIdxEmpty); - remoteLogMetadataManager.addRemoteLogSegmentMetadata(copySegmentStartedRlsm).get(); + remoteLogMetadataManagerPlugin.get().addRemoteLogSegmentMetadata(copySegmentStartedRlsm).get(); ByteBuffer leaderEpochsIndex = epochEntriesAsByteBuffer(getLeaderEpochEntries(log, -1, nextSegmentBaseOffset)); LogSegmentData segmentData = new LogSegmentData(logFile.toPath(), toPathIfExists(segment.offsetIndex().file()), @@ -1033,7 +1034,7 @@ private void copyLogSegment(UnifiedLog log, LogSegment segment, RemoteLogSegment Optional customMetadata; try { - customMetadata = remoteLogStorageManager.copyLogSegmentData(copySegmentStartedRlsm, segmentData); + customMetadata = remoteLogStorageManagerPlugin.get().copyLogSegmentData(copySegmentStartedRlsm, segmentData); } catch (RemoteStorageException e) { logger.info("Copy failed, cleaning segment {}", copySegmentStartedRlsm.remoteLogSegmentId()); try { @@ -1069,7 +1070,7 @@ private void copyLogSegment(UnifiedLog log, LogSegment segment, RemoteLogSegment } } - remoteLogMetadataManager.updateRemoteLogSegmentMetadata(copySegmentFinishedRlsm).get(); + remoteLogMetadataManagerPlugin.get().updateRemoteLogSegmentMetadata(copySegmentFinishedRlsm).get(); brokerTopicStats.topicStats(log.topicPartition().topic()) .remoteCopyBytesRate().mark(copySegmentStartedRlsm.segmentSizeInBytes()); brokerTopicStats.allTopicsStats().remoteCopyBytesRate().mark(copySegmentStartedRlsm.segmentSizeInBytes()); @@ -1252,7 +1253,7 @@ void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionE final UnifiedLog log = logOptional.get(); // Cleanup remote log segments and update the log start offset if applicable. - final Iterator segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition); + final Iterator segmentMetadataIter = remoteLogMetadataManagerPlugin.get().listRemoteLogSegments(topicIdPartition); if (!segmentMetadataIter.hasNext()) { updateMetadataCountAndLogSizeWith(0, 0); logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition); @@ -1294,7 +1295,7 @@ void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionE long sizeOfDeletableSegmentsBytes = 0L; while (canProcess && epochIterator.hasNext()) { Integer epoch = epochIterator.next(); - Iterator segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch); + Iterator segmentsIterator = remoteLogMetadataManagerPlugin.get().listRemoteLogSegments(topicIdPartition, epoch); while (canProcess && segmentsIterator.hasNext()) { if (isCancelled()) { logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed."); @@ -1387,7 +1388,7 @@ void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionE while (epochsToClean.hasNext()) { int epoch = epochsToClean.next(); - Iterator segmentsToBeCleaned = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch); + Iterator segmentsToBeCleaned = remoteLogMetadataManagerPlugin.get().listRemoteLogSegments(topicIdPartition, epoch); while (segmentsToBeCleaned.hasNext()) { if (!isCancelled()) { RemoteLogSegmentMetadata nextSegmentMetadata = segmentsToBeCleaned.next(); @@ -1432,7 +1433,7 @@ private Optional buildRetentionSizeData(long retentionSize, // log size may be computed for all the segments but not for segments with in the current // partition's leader epoch lineage. Better to revisit this API. // remoteLogSizeBytes += remoteLogMetadataManager.remoteLogSize(topicIdPartition, epochEntry.epoch); - Iterator segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch); + Iterator segmentsIterator = remoteLogMetadataManagerPlugin.get().listRemoteLogSegments(topicIdPartition, epoch); while (segmentsIterator.hasNext()) { RemoteLogSegmentMetadata segmentMetadata = segmentsIterator.next(); // Count only the size of segments in "COPY_SEGMENT_FINISHED" state because @@ -1489,7 +1490,7 @@ private boolean deleteRemoteLogSegment( String topic = segmentMetadata.topicIdPartition().topic(); // Publish delete segment started event. - remoteLogMetadataManager.updateRemoteLogSegmentMetadata( + remoteLogMetadataManagerPlugin.get().updateRemoteLogSegmentMetadata( new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(), segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get(); @@ -1498,7 +1499,7 @@ private boolean deleteRemoteLogSegment( // Delete the segment in remote storage. try { - remoteLogStorageManager.deleteLogSegmentData(segmentMetadata); + remoteLogStorageManagerPlugin.get().deleteLogSegmentData(segmentMetadata); } catch (RemoteStorageException e) { brokerTopicStats.topicStats(topic).failedRemoteDeleteRequestRate().mark(); brokerTopicStats.allTopicsStats().failedRemoteDeleteRequestRate().mark(); @@ -1506,7 +1507,7 @@ private boolean deleteRemoteLogSegment( } // Publish delete segment finished event. - remoteLogMetadataManager.updateRemoteLogSegmentMetadata( + remoteLogMetadataManagerPlugin.get().updateRemoteLogSegmentMetadata( new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(), segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get(); LOGGER.debug("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId()); @@ -1702,7 +1703,7 @@ public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws remoteLogSegmentMetadata = rlsMetadataOptional.get(); // Search forward for the position of the last offset that is greater than or equal to the target offset startPos = lookupPositionForOffset(remoteLogSegmentMetadata, offset); - remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(remoteLogSegmentMetadata, startPos); + remoteSegInputStream = remoteLogStorageManagerPlugin.get().fetchLogSegment(remoteLogSegmentMetadata, startPos); RemoteLogInputStream remoteLogInputStream = getRemoteLogInputStream(remoteSegInputStream); enrichedRecordBatch = findFirstBatch(remoteLogInputStream, offset); if (enrichedRecordBatch.batch == null) { @@ -1922,7 +1923,7 @@ OffsetAndEpoch findHighestRemoteOffset(TopicIdPartition topicIdPartition, Unifie while (offsetAndEpoch == null && maybeEpochEntry.isPresent()) { int epoch = maybeEpochEntry.get().epoch; Optional highestRemoteOffsetOpt = - remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, epoch); + remoteLogMetadataManagerPlugin.get().highestOffsetForEpoch(topicIdPartition, epoch); if (highestRemoteOffsetOpt.isPresent()) { Map.Entry entry = leaderEpochCache.endOffsetFor(epoch, log.logEndOffset()); int requestedEpoch = entry.getKey(); @@ -1953,7 +1954,7 @@ long findLogStartOffset(TopicIdPartition topicIdPartition, UnifiedLog log) throw .orElseGet(OptionalInt::empty); while (logStartOffset.isEmpty() && earliestEpochOpt.isPresent()) { Iterator iterator = - remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, earliestEpochOpt.getAsInt()); + remoteLogMetadataManagerPlugin.get().listRemoteLogSegments(topicIdPartition, earliestEpochOpt.getAsInt()); if (iterator.hasNext()) { logStartOffset = Optional.of(iterator.next().startOffset()); } From c21ec898a5312b61badb33c5a93ba930ed215fe0 Mon Sep 17 00:00:00 2001 From: TaiJu Wu Date: Sat, 22 Mar 2025 09:59:20 +0000 Subject: [PATCH 03/24] fix compiler --- .../java/kafka/log/remote/RemoteLogManagerTest.java | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java index f782061d0715d..80a1e3b47846e 100644 --- a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java +++ b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java @@ -26,6 +26,7 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.compress.Compression; import org.apache.kafka.common.errors.ReplicaNotAvailableException; +import org.apache.kafka.common.internals.Plugin; import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.record.FileRecords; @@ -245,11 +246,11 @@ void setUp() throws Exception { tp -> Optional.of(mockLog), (topicPartition, offset) -> currentLogStartOffset.set(offset), brokerTopicStats, metrics) { - public RemoteStorageManager createRemoteStorageManager() { - return remoteStorageManager; + public Plugin createRemoteStorageManagerPlugin(Metrics metrics) { + return Plugin.wrapInstance(remoteStorageManager, metrics, "mockRemoteStorageManager"); } - public RemoteLogMetadataManager createRemoteLogMetadataManager() { - return remoteLogMetadataManager; + public Plugin createRemoteLogMetadataManagerPlugin(Metrics metrics) { + return Plugin.wrapInstance(remoteLogMetadataManager, metrics, "mockRemoteStorageManager"); } public RLMQuotaManager createRLMCopyQuotaManager() { return rlmCopyQuotaManager; @@ -1752,8 +1753,8 @@ void testFetchOffsetByTimestampWithTieredStorageDoesNotFetchIndexWhenExistsLocal (topicPartition, offset) -> currentLogStartOffset.set(offset), brokerTopicStats, metrics) { @Override - public RemoteLogMetadataManager createRemoteLogMetadataManager() { - return remoteLogMetadataManager; + public Plugin createRemoteLogMetadataManagerPlugin(Metrics metrics) { + return Plugin.wrapInstance(remoteLogMetadataManager, metrics, "mockRemoteLogMetadataManager"); } @Override Optional lookupTimestamp(RemoteLogSegmentMetadata rlsMetadata, long timestamp, long startingOffset) { From 3f6ca5cd5ac9eaafcb53a94814246656efedbb57 Mon Sep 17 00:00:00 2001 From: TaiJu Wu Date: Sat, 22 Mar 2025 11:02:19 +0000 Subject: [PATCH 04/24] Revert "fix compiler" This reverts commit 86487b2bc20f7cd5e0c1ef3fb7db4dc746345545. --- .../java/kafka/log/remote/RemoteLogManagerTest.java | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java index 80a1e3b47846e..f782061d0715d 100644 --- a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java +++ b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java @@ -26,7 +26,6 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.compress.Compression; import org.apache.kafka.common.errors.ReplicaNotAvailableException; -import org.apache.kafka.common.internals.Plugin; import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.record.FileRecords; @@ -246,11 +245,11 @@ void setUp() throws Exception { tp -> Optional.of(mockLog), (topicPartition, offset) -> currentLogStartOffset.set(offset), brokerTopicStats, metrics) { - public Plugin createRemoteStorageManagerPlugin(Metrics metrics) { - return Plugin.wrapInstance(remoteStorageManager, metrics, "mockRemoteStorageManager"); + public RemoteStorageManager createRemoteStorageManager() { + return remoteStorageManager; } - public Plugin createRemoteLogMetadataManagerPlugin(Metrics metrics) { - return Plugin.wrapInstance(remoteLogMetadataManager, metrics, "mockRemoteStorageManager"); + public RemoteLogMetadataManager createRemoteLogMetadataManager() { + return remoteLogMetadataManager; } public RLMQuotaManager createRLMCopyQuotaManager() { return rlmCopyQuotaManager; @@ -1753,8 +1752,8 @@ void testFetchOffsetByTimestampWithTieredStorageDoesNotFetchIndexWhenExistsLocal (topicPartition, offset) -> currentLogStartOffset.set(offset), brokerTopicStats, metrics) { @Override - public Plugin createRemoteLogMetadataManagerPlugin(Metrics metrics) { - return Plugin.wrapInstance(remoteLogMetadataManager, metrics, "mockRemoteLogMetadataManager"); + public RemoteLogMetadataManager createRemoteLogMetadataManager() { + return remoteLogMetadataManager; } @Override Optional lookupTimestamp(RemoteLogSegmentMetadata rlsMetadata, long timestamp, long startingOffset) { From 5e0ff34ef96d4f12389c604f97173f842ef60155 Mon Sep 17 00:00:00 2001 From: TaiJu Wu Date: Sat, 22 Mar 2025 11:02:22 +0000 Subject: [PATCH 05/24] Revert "implement prod code" This reverts commit 08c1b2a56b0797de0f739e372545f185d9f76c78. --- .../kafka/log/remote/RemoteLogManager.java | 93 +++++++++---------- 1 file changed, 46 insertions(+), 47 deletions(-) diff --git a/core/src/main/java/kafka/log/remote/RemoteLogManager.java b/core/src/main/java/kafka/log/remote/RemoteLogManager.java index 7ce35d236c926..4ab4774dfe17a 100644 --- a/core/src/main/java/kafka/log/remote/RemoteLogManager.java +++ b/core/src/main/java/kafka/log/remote/RemoteLogManager.java @@ -169,9 +169,11 @@ public class RemoteLogManager implements Closeable, AsyncOffsetReader { private final BrokerTopicStats brokerTopicStats; private final Metrics metrics; - private final Plugin remoteLogStorageManagerPlugin; + private final RemoteStorageManager remoteLogStorageManager; + private Plugin remoteLogStorageManagerPlugin; - private final Plugin remoteLogMetadataManagerPlugin; + private final RemoteLogMetadataManager remoteLogMetadataManager; + private Plugin remoteLogMetadataManagerPlugin; private final ReentrantLock copyQuotaManagerLock = new ReentrantLock(true); private final Condition copyQuotaManagerLockCondition = copyQuotaManagerLock.newCondition(); @@ -239,8 +241,8 @@ public RemoteLogManager(RemoteLogManagerConfig rlmConfig, this.brokerTopicStats = brokerTopicStats; this.metrics = metrics; - remoteLogStorageManagerPlugin = createRemoteStorageManagerPlugin(metrics); - remoteLogMetadataManagerPlugin = createRemoteLogMetadataManagerPlugin(metrics); + remoteLogStorageManager = createRemoteStorageManager(); + remoteLogMetadataManager = createRemoteLogMetadataManager(); rlmCopyQuotaManager = createRLMCopyQuotaManager(); rlmFetchQuotaManager = createRLMFetchQuotaManager(); @@ -249,7 +251,7 @@ public RemoteLogManager(RemoteLogManagerConfig rlmConfig, copyThrottleTimeSensor = new RLMQuotaMetrics(metrics, "remote-copy-throttle-time", RemoteLogManager.class.getSimpleName(), "The %s time in millis remote copies was throttled by a broker", INACTIVE_SENSOR_EXPIRATION_TIME_SECONDS).sensor(); - indexCache = new RemoteIndexCache(rlmConfig.remoteLogIndexFileCacheTotalSizeBytes(), remoteLogStorageManagerPlugin.get(), logDir); + indexCache = new RemoteIndexCache(rlmConfig.remoteLogIndexFileCacheTotalSizeBytes(), remoteLogStorageManager, logDir); delayInMs = rlmConfig.remoteLogManagerTaskIntervalMs(); rlmCopyThreadPool = new RLMScheduledThreadPool(rlmConfig.remoteLogManagerCopierThreadPoolSize(), "RLMCopyThreadPool", "kafka-rlm-copy-thread-pool-%d"); @@ -359,42 +361,34 @@ private T createDelegate(ClassLoader classLoader, String className) { } } - Plugin createRemoteStorageManagerPlugin(Metrics metrics) { + RemoteStorageManager createRemoteStorageManager() { return SecurityManagerCompatibility.get().doPrivileged(() -> { final String classPath = rlmConfig.remoteStorageManagerClassPath(); if (classPath != null && !classPath.trim().isEmpty()) { ChildFirstClassLoader classLoader = new ChildFirstClassLoader(classPath, this.getClass().getClassLoader()); RemoteStorageManager delegate = createDelegate(classLoader, rlmConfig.remoteStorageManagerClassName()); - RemoteStorageManager rsm = new ClassLoaderAwareRemoteStorageManager(delegate, classLoader); - configureRSM(rsm); - return Plugin.wrapInstance(rsm, metrics, rlmConfig.remoteStorageManagerClassName()); + return (RemoteStorageManager) new ClassLoaderAwareRemoteStorageManager(delegate, classLoader); } else { - RemoteStorageManager rsm = createDelegate(this.getClass().getClassLoader(), rlmConfig.remoteStorageManagerClassName()); - configureRSM(rsm); - return Plugin.wrapInstance(rsm, metrics, rlmConfig.remoteStorageManagerClassName()); + return createDelegate(this.getClass().getClassLoader(), rlmConfig.remoteStorageManagerClassName()); } }); } - private void configureRSM(RemoteStorageManager rsm) { + private void configureRSM() { final Map rsmProps = new HashMap<>(rlmConfig.remoteStorageManagerProps()); rsmProps.put(ServerConfigs.BROKER_ID_CONFIG, brokerId); - rsm.configure(rsmProps); + remoteLogStorageManager.configure(rsmProps); } - Plugin createRemoteLogMetadataManagerPlugin(Metrics metrics) { + RemoteLogMetadataManager createRemoteLogMetadataManager() { return SecurityManagerCompatibility.get().doPrivileged(() -> { final String classPath = rlmConfig.remoteLogMetadataManagerClassPath(); if (classPath != null && !classPath.trim().isEmpty()) { ClassLoader classLoader = new ChildFirstClassLoader(classPath, this.getClass().getClassLoader()); RemoteLogMetadataManager delegate = createDelegate(classLoader, rlmConfig.remoteLogMetadataManagerClassName()); - RemoteLogMetadataManager rlmm = new ClassLoaderAwareRemoteLogMetadataManager(delegate, classLoader); - configureRLMM(rlmm); - return Plugin.wrapInstance(rlmm, metrics, rlmConfig.remoteLogMetadataManagerClassName()); + return (RemoteLogMetadataManager) new ClassLoaderAwareRemoteLogMetadataManager(delegate, classLoader); } else { - RemoteLogMetadataManager rlmm = createDelegate(this.getClass().getClassLoader(), rlmConfig.remoteLogMetadataManagerClassName()); - configureRLMM(rlmm); - return Plugin.wrapInstance(rlmm, metrics, rlmConfig.remoteLogMetadataManagerClassName()); + return createDelegate(this.getClass().getClassLoader(), rlmConfig.remoteLogMetadataManagerClassName()); } }); } @@ -403,7 +397,7 @@ public void onEndPointCreated(Endpoint endpoint) { this.endpoint = Optional.of(endpoint); } - private void configureRLMM(RemoteLogMetadataManager rlmm) { + private void configureRLMM() { final Map rlmmProps = new HashMap<>(); endpoint.ifPresent(e -> { rlmmProps.put(REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX + "bootstrap.servers", e.host() + ":" + e.port()); @@ -416,12 +410,17 @@ private void configureRLMM(RemoteLogMetadataManager rlmm) { rlmmProps.put(LOG_DIR_CONFIG, logDir); rlmmProps.put("cluster.id", clusterId); - rlmm.configure(rlmmProps); + remoteLogMetadataManager.configure(rlmmProps); } public void startup() { // Initialize and configure RSM and RLMM. This will start RSM, RLMM resources which may need to start resources // in connecting to the brokers or remote storages. + configureRSM(); + configureRLMM(); + // the withPluginMetrics() method will be called when the plugin is instantiated (after configure() if the plugin also implements Configurable) + remoteLogStorageManagerPlugin = Plugin.wrapInstance(remoteLogStorageManager, metrics, rlmConfig.remoteStorageManagerClassName()); + remoteLogMetadataManagerPlugin = Plugin.wrapInstance(remoteLogMetadataManager, metrics, rlmConfig.remoteLogMetadataManagerClassName()); remoteLogManagerConfigured = true; } @@ -430,7 +429,7 @@ private boolean isRemoteLogManagerConfigured() { } public RemoteStorageManager storageManager() { - return remoteLogStorageManagerPlugin.get(); + return remoteLogStorageManager; } private Stream filterPartitions(Set partitions) { @@ -479,7 +478,7 @@ public void onLeadershipChange(Set partitionsBecomeLeader, leaderPartitions.forEach((tp, __) -> cacheTopicPartitionIds(tp)); followerPartitions.forEach((tp, __) -> cacheTopicPartitionIds(tp)); - remoteLogMetadataManagerPlugin.get().onPartitionLeadershipChanges(leaderPartitions.keySet(), followerPartitions.keySet()); + remoteLogMetadataManager.onPartitionLeadershipChanges(leaderPartitions.keySet(), followerPartitions.keySet()); followerPartitions.forEach((tp, __) -> doHandleFollowerPartition(tp)); // If this node was the previous leader for the partition, then the RLMTask might be running in the @@ -563,13 +562,13 @@ public void stopPartitions(Set stopPartitions, if (!pendingActionsPartitions.isEmpty()) { pendingActionsPartitions.forEach(tpId -> topicIdByPartitionMap.remove(tpId.topicPartition())); - remoteLogMetadataManagerPlugin.get().onStopPartitions(pendingActionsPartitions); + remoteLogMetadataManager.onStopPartitions(pendingActionsPartitions); } } private void deleteRemoteLogPartition(TopicIdPartition partition) throws RemoteStorageException, ExecutionException, InterruptedException { List metadataList = new ArrayList<>(); - remoteLogMetadataManagerPlugin.get().listRemoteLogSegments(partition).forEachRemaining(metadataList::add); + remoteLogMetadataManager.listRemoteLogSegments(partition).forEachRemaining(metadataList::add); List deleteSegmentStartedEvents = metadataList.stream() .map(metadata -> @@ -582,7 +581,7 @@ private void deleteRemoteLogPartition(TopicIdPartition partition) throws RemoteS Collection deletedSegmentIds = new ArrayList<>(); for (RemoteLogSegmentMetadata metadata: metadataList) { deletedSegmentIds.add(metadata.remoteLogSegmentId().id()); - remoteLogStorageManagerPlugin.get().deleteLogSegmentData(metadata); + remoteLogStorageManager.deleteLogSegmentData(metadata); } indexCache.removeAll(deletedSegmentIds); @@ -597,7 +596,7 @@ private void deleteRemoteLogPartition(TopicIdPartition partition) throws RemoteS private CompletableFuture publishEvents(List events) throws RemoteStorageException { List> result = new ArrayList<>(); for (RemoteLogSegmentMetadataUpdate event : events) { - result.add(remoteLogMetadataManagerPlugin.get().updateRemoteLogSegmentMetadata(event)); + result.add(remoteLogMetadataManager.updateRemoteLogSegmentMetadata(event)); } return CompletableFuture.allOf(result.toArray(new CompletableFuture[0])); } @@ -609,7 +608,7 @@ public Optional fetchRemoteLogSegmentMetadata(TopicPar if (topicId == null) { throw new KafkaException("No topic id registered for topic partition: " + topicPartition); } - return remoteLogMetadataManagerPlugin.get().remoteLogSegmentMetadata(new TopicIdPartition(topicId, topicPartition), epochForOffset, offset); + return remoteLogMetadataManager.remoteLogSegmentMetadata(new TopicIdPartition(topicId, topicPartition), epochForOffset, offset); } /** @@ -629,7 +628,7 @@ public Optional fetchNextSegmentWithTxnIndex(TopicPart throw new KafkaException("No topic id registered for topic partition: " + topicPartition); } TopicIdPartition tpId = new TopicIdPartition(topicId, topicPartition); - return remoteLogMetadataManagerPlugin.get().nextSegmentWithTxnIndex(tpId, epochForOffset, offset); + return remoteLogMetadataManager.nextSegmentWithTxnIndex(tpId, epochForOffset, offset); } Optional lookupTimestamp(RemoteLogSegmentMetadata rlsMetadata, long timestamp, long startingOffset) @@ -639,7 +638,7 @@ Optional lookupTimestamp(RemoteLogSegmentMetadat InputStream remoteSegInputStream = null; try { // Search forward for the position of the last offset that is greater than or equal to the startingOffset - remoteSegInputStream = remoteLogStorageManagerPlugin.get().fetchLogSegment(rlsMetadata, startPos); + remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(rlsMetadata, startPos); RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream); while (true) { @@ -729,7 +728,7 @@ public Optional findOffsetByTimestamp(TopicParti int epoch = maybeEpoch.getAsInt(); // KAFKA-15802: Add a new API for RLMM to choose how to implement the predicate. // currently, all segments are returned and then iterated, and filtered - Iterator iterator = remoteLogMetadataManagerPlugin.get().listRemoteLogSegments(topicIdPartition, epoch); + Iterator iterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch); while (iterator.hasNext()) { RemoteLogSegmentMetadata rlsMetadata = iterator.next(); if (rlsMetadata.maxTimestampMs() >= timestamp @@ -813,7 +812,7 @@ public void run() { logger.debug("Skipping the current run for partition {} as it is cancelled", topicIdPartition); return; } - if (!remoteLogMetadataManagerPlugin.get().isReady(topicIdPartition)) { + if (!remoteLogMetadataManager.isReady(topicIdPartition)) { logger.debug("Skipping the current run for partition {} as the remote-log metadata is not ready", topicIdPartition); return; } @@ -1023,7 +1022,7 @@ private void copyLogSegment(UnifiedLog log, LogSegment segment, RemoteLogSegment segment.largestTimestamp(), brokerId, time.milliseconds(), segment.log().sizeInBytes(), segmentLeaderEpochs, isTxnIdxEmpty); - remoteLogMetadataManagerPlugin.get().addRemoteLogSegmentMetadata(copySegmentStartedRlsm).get(); + remoteLogMetadataManager.addRemoteLogSegmentMetadata(copySegmentStartedRlsm).get(); ByteBuffer leaderEpochsIndex = epochEntriesAsByteBuffer(getLeaderEpochEntries(log, -1, nextSegmentBaseOffset)); LogSegmentData segmentData = new LogSegmentData(logFile.toPath(), toPathIfExists(segment.offsetIndex().file()), @@ -1034,7 +1033,7 @@ private void copyLogSegment(UnifiedLog log, LogSegment segment, RemoteLogSegment Optional customMetadata; try { - customMetadata = remoteLogStorageManagerPlugin.get().copyLogSegmentData(copySegmentStartedRlsm, segmentData); + customMetadata = remoteLogStorageManager.copyLogSegmentData(copySegmentStartedRlsm, segmentData); } catch (RemoteStorageException e) { logger.info("Copy failed, cleaning segment {}", copySegmentStartedRlsm.remoteLogSegmentId()); try { @@ -1070,7 +1069,7 @@ private void copyLogSegment(UnifiedLog log, LogSegment segment, RemoteLogSegment } } - remoteLogMetadataManagerPlugin.get().updateRemoteLogSegmentMetadata(copySegmentFinishedRlsm).get(); + remoteLogMetadataManager.updateRemoteLogSegmentMetadata(copySegmentFinishedRlsm).get(); brokerTopicStats.topicStats(log.topicPartition().topic()) .remoteCopyBytesRate().mark(copySegmentStartedRlsm.segmentSizeInBytes()); brokerTopicStats.allTopicsStats().remoteCopyBytesRate().mark(copySegmentStartedRlsm.segmentSizeInBytes()); @@ -1253,7 +1252,7 @@ void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionE final UnifiedLog log = logOptional.get(); // Cleanup remote log segments and update the log start offset if applicable. - final Iterator segmentMetadataIter = remoteLogMetadataManagerPlugin.get().listRemoteLogSegments(topicIdPartition); + final Iterator segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition); if (!segmentMetadataIter.hasNext()) { updateMetadataCountAndLogSizeWith(0, 0); logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition); @@ -1295,7 +1294,7 @@ void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionE long sizeOfDeletableSegmentsBytes = 0L; while (canProcess && epochIterator.hasNext()) { Integer epoch = epochIterator.next(); - Iterator segmentsIterator = remoteLogMetadataManagerPlugin.get().listRemoteLogSegments(topicIdPartition, epoch); + Iterator segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch); while (canProcess && segmentsIterator.hasNext()) { if (isCancelled()) { logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed."); @@ -1388,7 +1387,7 @@ void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionE while (epochsToClean.hasNext()) { int epoch = epochsToClean.next(); - Iterator segmentsToBeCleaned = remoteLogMetadataManagerPlugin.get().listRemoteLogSegments(topicIdPartition, epoch); + Iterator segmentsToBeCleaned = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch); while (segmentsToBeCleaned.hasNext()) { if (!isCancelled()) { RemoteLogSegmentMetadata nextSegmentMetadata = segmentsToBeCleaned.next(); @@ -1433,7 +1432,7 @@ private Optional buildRetentionSizeData(long retentionSize, // log size may be computed for all the segments but not for segments with in the current // partition's leader epoch lineage. Better to revisit this API. // remoteLogSizeBytes += remoteLogMetadataManager.remoteLogSize(topicIdPartition, epochEntry.epoch); - Iterator segmentsIterator = remoteLogMetadataManagerPlugin.get().listRemoteLogSegments(topicIdPartition, epoch); + Iterator segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch); while (segmentsIterator.hasNext()) { RemoteLogSegmentMetadata segmentMetadata = segmentsIterator.next(); // Count only the size of segments in "COPY_SEGMENT_FINISHED" state because @@ -1490,7 +1489,7 @@ private boolean deleteRemoteLogSegment( String topic = segmentMetadata.topicIdPartition().topic(); // Publish delete segment started event. - remoteLogMetadataManagerPlugin.get().updateRemoteLogSegmentMetadata( + remoteLogMetadataManager.updateRemoteLogSegmentMetadata( new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(), segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get(); @@ -1499,7 +1498,7 @@ private boolean deleteRemoteLogSegment( // Delete the segment in remote storage. try { - remoteLogStorageManagerPlugin.get().deleteLogSegmentData(segmentMetadata); + remoteLogStorageManager.deleteLogSegmentData(segmentMetadata); } catch (RemoteStorageException e) { brokerTopicStats.topicStats(topic).failedRemoteDeleteRequestRate().mark(); brokerTopicStats.allTopicsStats().failedRemoteDeleteRequestRate().mark(); @@ -1507,7 +1506,7 @@ private boolean deleteRemoteLogSegment( } // Publish delete segment finished event. - remoteLogMetadataManagerPlugin.get().updateRemoteLogSegmentMetadata( + remoteLogMetadataManager.updateRemoteLogSegmentMetadata( new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(), segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get(); LOGGER.debug("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId()); @@ -1703,7 +1702,7 @@ public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws remoteLogSegmentMetadata = rlsMetadataOptional.get(); // Search forward for the position of the last offset that is greater than or equal to the target offset startPos = lookupPositionForOffset(remoteLogSegmentMetadata, offset); - remoteSegInputStream = remoteLogStorageManagerPlugin.get().fetchLogSegment(remoteLogSegmentMetadata, startPos); + remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(remoteLogSegmentMetadata, startPos); RemoteLogInputStream remoteLogInputStream = getRemoteLogInputStream(remoteSegInputStream); enrichedRecordBatch = findFirstBatch(remoteLogInputStream, offset); if (enrichedRecordBatch.batch == null) { @@ -1923,7 +1922,7 @@ OffsetAndEpoch findHighestRemoteOffset(TopicIdPartition topicIdPartition, Unifie while (offsetAndEpoch == null && maybeEpochEntry.isPresent()) { int epoch = maybeEpochEntry.get().epoch; Optional highestRemoteOffsetOpt = - remoteLogMetadataManagerPlugin.get().highestOffsetForEpoch(topicIdPartition, epoch); + remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, epoch); if (highestRemoteOffsetOpt.isPresent()) { Map.Entry entry = leaderEpochCache.endOffsetFor(epoch, log.logEndOffset()); int requestedEpoch = entry.getKey(); @@ -1954,7 +1953,7 @@ long findLogStartOffset(TopicIdPartition topicIdPartition, UnifiedLog log) throw .orElseGet(OptionalInt::empty); while (logStartOffset.isEmpty() && earliestEpochOpt.isPresent()) { Iterator iterator = - remoteLogMetadataManagerPlugin.get().listRemoteLogSegments(topicIdPartition, earliestEpochOpt.getAsInt()); + remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, earliestEpochOpt.getAsInt()); if (iterator.hasNext()) { logStartOffset = Optional.of(iterator.next().startOffset()); } From 0f5c308fd5d40be0b21b506108bb1933a3fda953 Mon Sep 17 00:00:00 2001 From: TaiJu Wu Date: Sat, 22 Mar 2025 14:32:35 +0000 Subject: [PATCH 06/24] fix test --- core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java index f782061d0715d..7661dcdfa805d 100644 --- a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java +++ b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java @@ -1795,6 +1795,7 @@ private LogSegment mockLogSegment(long baseOffset, @Test void testIdempotentClose() throws IOException { + remoteLogManager.startup(); remoteLogManager.close(); remoteLogManager.close(); InOrder inorder = inOrder(remoteStorageManager, remoteLogMetadataManager); From 5063d0d03525348e49c98a6b695dfd1bf0d3a835 Mon Sep 17 00:00:00 2001 From: TaiJu Wu Date: Sat, 22 Mar 2025 14:43:08 +0000 Subject: [PATCH 07/24] add Utils.closeQuietly --- core/src/main/java/kafka/log/remote/RemoteLogManager.java | 2 ++ core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java | 1 - 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/kafka/log/remote/RemoteLogManager.java b/core/src/main/java/kafka/log/remote/RemoteLogManager.java index 4ab4774dfe17a..8930e4cc0f3f7 100644 --- a/core/src/main/java/kafka/log/remote/RemoteLogManager.java +++ b/core/src/main/java/kafka/log/remote/RemoteLogManager.java @@ -2053,6 +2053,8 @@ public void close() { followerRLMTasks.values().forEach(RLMTaskWithFuture::cancel); Utils.closeQuietly(remoteLogStorageManagerPlugin, "remoteLogStorageManagerPlugin"); Utils.closeQuietly(remoteLogMetadataManagerPlugin, "remoteLogMetadataManagerPlugin"); + Utils.closeQuietly(remoteLogStorageManager, "remoteLogStorageManager"); + Utils.closeQuietly(remoteLogMetadataManager, "remoteLogMetadataManager"); Utils.closeQuietly(indexCache, "RemoteIndexCache"); rlmCopyThreadPool.close(); diff --git a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java index 7661dcdfa805d..f782061d0715d 100644 --- a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java +++ b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java @@ -1795,7 +1795,6 @@ private LogSegment mockLogSegment(long baseOffset, @Test void testIdempotentClose() throws IOException { - remoteLogManager.startup(); remoteLogManager.close(); remoteLogManager.close(); InOrder inorder = inOrder(remoteStorageManager, remoteLogMetadataManager); From da3e54ef326d15f79a8b8daf5764c488e183b960 Mon Sep 17 00:00:00 2001 From: TaiJu Wu Date: Tue, 25 Mar 2025 09:21:18 +0000 Subject: [PATCH 08/24] add test --- .../log/remote/RemoteLogManagerTest.java | 48 +++++++++++++++++++ 1 file changed, 48 insertions(+) diff --git a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java index f782061d0715d..f55ea7934b8d3 100644 --- a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java +++ b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java @@ -28,6 +28,8 @@ import org.apache.kafka.common.errors.ReplicaNotAvailableException; import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.Monitorable; +import org.apache.kafka.common.metrics.PluginMetrics; import org.apache.kafka.common.record.FileRecords; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.RecordBatch; @@ -154,6 +156,7 @@ import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -3741,6 +3744,33 @@ long findLogStartOffset(TopicIdPartition topicIdPartition, UnifiedLog log) { verifyNoMoreInteractions(remoteStorageManager); } + @Test + public void testMonitorableRemoteLogStorageManager() throws IOException { + Properties props = new Properties(); + props.putAll(brokerConfig); + appendRLMConfig(props); + props.put(RemoteLogManagerConfig.REMOTE_STORAGE_MANAGER_CLASS_NAME_PROP, MonitorableNoOpRemoteStorageManager.class.getName()); + props.put(RemoteLogManagerConfig.REMOTE_LOG_METADATA_MANAGER_CLASS_NAME_PROP, MonitorableNoOpRemoteLogMetadataManager.class.getName()); + KafkaConfig config = KafkaConfig.fromProps(props); + try (RemoteLogManager remoteLogManager = new RemoteLogManager( + config.remoteLogManagerConfig(), + brokerId, + logDir, + clusterId, + time, + tp -> Optional.of(mockLog), + (topicPartition, offset) -> { }, + brokerTopicStats, + metrics)) { + // We need to call startup for call config and wrap instance + remoteLogManager.startup(); + assertInstanceOf(MonitorableNoOpRemoteStorageManager.class, remoteLogManager.storageManager()); + assertInstanceOf(MonitorableNoOpRemoteLogMetadataManager.class, remoteLogManager.remoteLogMetadataManager()); + assertEquals(true, ((MonitorableNoOpRemoteStorageManager) remoteLogManager.storageManager()).pluginMetrics); + assertEquals(true, ((MonitorableNoOpRemoteLogMetadataManager) remoteLogManager.remoteLogMetadataManager()).pluginMetrics); + } + } + private void appendRecordsToFile(File file, int nRecords, int nRecordsPerBatch) throws IOException { byte magic = RecordBatch.CURRENT_MAGIC_VALUE; Compression compression = Compression.NONE; @@ -3785,4 +3815,22 @@ private void appendRLMConfig(Properties props) { props.put(DEFAULT_REMOTE_LOG_METADATA_MANAGER_CONFIG_PREFIX + remoteLogMetadataProducerTestProp, remoteLogMetadataProducerTestVal); } + public static class MonitorableNoOpRemoteStorageManager extends NoOpRemoteStorageManager implements Monitorable { + public boolean pluginMetrics = false; + + public MonitorableNoOpRemoteStorageManager() { } + + @Override + public void withPluginMetrics(PluginMetrics metrics) { + pluginMetrics = true; + } + } + + public static class MonitorableNoOpRemoteLogMetadataManager extends NoOpRemoteLogMetadataManager implements Monitorable { + public boolean pluginMetrics = false; + @Override + public void withPluginMetrics(PluginMetrics metrics) { + pluginMetrics = true; + } + } } From 5979f304cdcf9d69b5d9a67a13444742e6591a70 Mon Sep 17 00:00:00 2001 From: TaiJu Wu Date: Tue, 25 Mar 2025 21:42:54 +0000 Subject: [PATCH 09/24] refactor --- .../src/main/java/kafka/log/remote/RemoteLogManager.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/kafka/log/remote/RemoteLogManager.java b/core/src/main/java/kafka/log/remote/RemoteLogManager.java index 8930e4cc0f3f7..61376e430b8bc 100644 --- a/core/src/main/java/kafka/log/remote/RemoteLogManager.java +++ b/core/src/main/java/kafka/log/remote/RemoteLogManager.java @@ -393,6 +393,11 @@ RemoteLogMetadataManager createRemoteLogMetadataManager() { }); } + + RemoteLogMetadataManager remoteLogMetadataManager() { + return remoteLogMetadataManager; + } + public void onEndPointCreated(Endpoint endpoint) { this.endpoint = Optional.of(endpoint); } @@ -2053,8 +2058,8 @@ public void close() { followerRLMTasks.values().forEach(RLMTaskWithFuture::cancel); Utils.closeQuietly(remoteLogStorageManagerPlugin, "remoteLogStorageManagerPlugin"); Utils.closeQuietly(remoteLogMetadataManagerPlugin, "remoteLogMetadataManagerPlugin"); - Utils.closeQuietly(remoteLogStorageManager, "remoteLogStorageManager"); - Utils.closeQuietly(remoteLogMetadataManager, "remoteLogMetadataManager"); + Utils.closeQuietly(remoteLogStorageManager, "RemoteLogStorageManager"); + Utils.closeQuietly(remoteLogMetadataManager, "RemoteLogMetadataManager"); Utils.closeQuietly(indexCache, "RemoteIndexCache"); rlmCopyThreadPool.close(); From 3e7f5b20f675bd207fc2c95f92d0e53f6fe8d892 Mon Sep 17 00:00:00 2001 From: TaiJu Wu Date: Wed, 26 Mar 2025 04:24:38 +0000 Subject: [PATCH 10/24] merge two variable fix test --- .../apache/kafka/common/internals/Plugin.java | 14 +++ .../kafka/log/remote/RemoteLogManager.java | 84 +++++++-------- .../log/remote/RemoteLogManagerTest.java | 101 +++++++++--------- 3 files changed, 105 insertions(+), 94 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/internals/Plugin.java b/clients/src/main/java/org/apache/kafka/common/internals/Plugin.java index 620cd0c07ec0f..c32105ec169f8 100644 --- a/clients/src/main/java/org/apache/kafka/common/internals/Plugin.java +++ b/clients/src/main/java/org/apache/kafka/common/internals/Plugin.java @@ -68,6 +68,20 @@ public static Plugin wrapInstance(T instance, Metrics metrics, Supplier(instance, pluginMetrics); } + public static Plugin wrapInstanceDelayedInit(T instance) { + return new Plugin<>(instance, null); + } + + public Plugin initialize(Metrics metrics, String key) { + PluginMetricsImpl pluginMetrics = null; + if (instance instanceof Monitorable && metrics != null) { + pluginMetrics = new PluginMetricsImpl(metrics, tags(key, instance)); + ((Monitorable) instance).withPluginMetrics(pluginMetrics); + } + return this; + } + + @Override public T get() { return instance; diff --git a/core/src/main/java/kafka/log/remote/RemoteLogManager.java b/core/src/main/java/kafka/log/remote/RemoteLogManager.java index 61376e430b8bc..baa6acf8650a5 100644 --- a/core/src/main/java/kafka/log/remote/RemoteLogManager.java +++ b/core/src/main/java/kafka/log/remote/RemoteLogManager.java @@ -169,11 +169,9 @@ public class RemoteLogManager implements Closeable, AsyncOffsetReader { private final BrokerTopicStats brokerTopicStats; private final Metrics metrics; - private final RemoteStorageManager remoteLogStorageManager; - private Plugin remoteLogStorageManagerPlugin; + private final Plugin remoteLogStorageManagerPlugin; - private final RemoteLogMetadataManager remoteLogMetadataManager; - private Plugin remoteLogMetadataManagerPlugin; + private final Plugin remoteLogMetadataManagerPlugin; private final ReentrantLock copyQuotaManagerLock = new ReentrantLock(true); private final Condition copyQuotaManagerLockCondition = copyQuotaManagerLock.newCondition(); @@ -241,8 +239,8 @@ public RemoteLogManager(RemoteLogManagerConfig rlmConfig, this.brokerTopicStats = brokerTopicStats; this.metrics = metrics; - remoteLogStorageManager = createRemoteStorageManager(); - remoteLogMetadataManager = createRemoteLogMetadataManager(); + remoteLogStorageManagerPlugin = createRemoteStorageManagerPlugin(); + remoteLogMetadataManagerPlugin = createRemoteLogMetadataManagerPlugin(); rlmCopyQuotaManager = createRLMCopyQuotaManager(); rlmFetchQuotaManager = createRLMFetchQuotaManager(); @@ -251,7 +249,7 @@ public RemoteLogManager(RemoteLogManagerConfig rlmConfig, copyThrottleTimeSensor = new RLMQuotaMetrics(metrics, "remote-copy-throttle-time", RemoteLogManager.class.getSimpleName(), "The %s time in millis remote copies was throttled by a broker", INACTIVE_SENSOR_EXPIRATION_TIME_SECONDS).sensor(); - indexCache = new RemoteIndexCache(rlmConfig.remoteLogIndexFileCacheTotalSizeBytes(), remoteLogStorageManager, logDir); + indexCache = new RemoteIndexCache(rlmConfig.remoteLogIndexFileCacheTotalSizeBytes(), remoteLogStorageManagerPlugin.get(), logDir); delayInMs = rlmConfig.remoteLogManagerTaskIntervalMs(); rlmCopyThreadPool = new RLMScheduledThreadPool(rlmConfig.remoteLogManagerCopierThreadPoolSize(), "RLMCopyThreadPool", "kafka-rlm-copy-thread-pool-%d"); @@ -361,15 +359,15 @@ private T createDelegate(ClassLoader classLoader, String className) { } } - RemoteStorageManager createRemoteStorageManager() { + Plugin createRemoteStorageManagerPlugin() { return SecurityManagerCompatibility.get().doPrivileged(() -> { final String classPath = rlmConfig.remoteStorageManagerClassPath(); if (classPath != null && !classPath.trim().isEmpty()) { ChildFirstClassLoader classLoader = new ChildFirstClassLoader(classPath, this.getClass().getClassLoader()); RemoteStorageManager delegate = createDelegate(classLoader, rlmConfig.remoteStorageManagerClassName()); - return (RemoteStorageManager) new ClassLoaderAwareRemoteStorageManager(delegate, classLoader); + return Plugin.wrapInstanceDelayedInit(new ClassLoaderAwareRemoteStorageManager(delegate, classLoader)); } else { - return createDelegate(this.getClass().getClassLoader(), rlmConfig.remoteStorageManagerClassName()); + return Plugin.wrapInstanceDelayedInit(createDelegate(this.getClass().getClassLoader(), rlmConfig.remoteStorageManagerClassName())); } }); } @@ -377,25 +375,25 @@ RemoteStorageManager createRemoteStorageManager() { private void configureRSM() { final Map rsmProps = new HashMap<>(rlmConfig.remoteStorageManagerProps()); rsmProps.put(ServerConfigs.BROKER_ID_CONFIG, brokerId); - remoteLogStorageManager.configure(rsmProps); + remoteLogStorageManagerPlugin.get().configure(rsmProps); } - RemoteLogMetadataManager createRemoteLogMetadataManager() { + Plugin createRemoteLogMetadataManagerPlugin() { return SecurityManagerCompatibility.get().doPrivileged(() -> { final String classPath = rlmConfig.remoteLogMetadataManagerClassPath(); if (classPath != null && !classPath.trim().isEmpty()) { ClassLoader classLoader = new ChildFirstClassLoader(classPath, this.getClass().getClassLoader()); RemoteLogMetadataManager delegate = createDelegate(classLoader, rlmConfig.remoteLogMetadataManagerClassName()); - return (RemoteLogMetadataManager) new ClassLoaderAwareRemoteLogMetadataManager(delegate, classLoader); + return Plugin.wrapInstanceDelayedInit(new ClassLoaderAwareRemoteLogMetadataManager(delegate, classLoader)); } else { - return createDelegate(this.getClass().getClassLoader(), rlmConfig.remoteLogMetadataManagerClassName()); + return Plugin.wrapInstanceDelayedInit(createDelegate(this.getClass().getClassLoader(), rlmConfig.remoteLogMetadataManagerClassName())); } }); } RemoteLogMetadataManager remoteLogMetadataManager() { - return remoteLogMetadataManager; + return remoteLogMetadataManagerPlugin.get(); } public void onEndPointCreated(Endpoint endpoint) { @@ -415,7 +413,7 @@ private void configureRLMM() { rlmmProps.put(LOG_DIR_CONFIG, logDir); rlmmProps.put("cluster.id", clusterId); - remoteLogMetadataManager.configure(rlmmProps); + remoteLogMetadataManagerPlugin.get().configure(rlmmProps); } public void startup() { @@ -424,8 +422,8 @@ public void startup() { configureRSM(); configureRLMM(); // the withPluginMetrics() method will be called when the plugin is instantiated (after configure() if the plugin also implements Configurable) - remoteLogStorageManagerPlugin = Plugin.wrapInstance(remoteLogStorageManager, metrics, rlmConfig.remoteStorageManagerClassName()); - remoteLogMetadataManagerPlugin = Plugin.wrapInstance(remoteLogMetadataManager, metrics, rlmConfig.remoteLogMetadataManagerClassName()); + remoteLogStorageManagerPlugin.initialize(metrics, rlmConfig.remoteStorageManagerClassName()); + remoteLogMetadataManagerPlugin.initialize(metrics, rlmConfig.remoteLogMetadataManagerClassName()); remoteLogManagerConfigured = true; } @@ -434,7 +432,7 @@ private boolean isRemoteLogManagerConfigured() { } public RemoteStorageManager storageManager() { - return remoteLogStorageManager; + return remoteLogStorageManagerPlugin.get(); } private Stream filterPartitions(Set partitions) { @@ -483,7 +481,7 @@ public void onLeadershipChange(Set partitionsBecomeLeader, leaderPartitions.forEach((tp, __) -> cacheTopicPartitionIds(tp)); followerPartitions.forEach((tp, __) -> cacheTopicPartitionIds(tp)); - remoteLogMetadataManager.onPartitionLeadershipChanges(leaderPartitions.keySet(), followerPartitions.keySet()); + remoteLogMetadataManagerPlugin.get().onPartitionLeadershipChanges(leaderPartitions.keySet(), followerPartitions.keySet()); followerPartitions.forEach((tp, __) -> doHandleFollowerPartition(tp)); // If this node was the previous leader for the partition, then the RLMTask might be running in the @@ -567,13 +565,13 @@ public void stopPartitions(Set stopPartitions, if (!pendingActionsPartitions.isEmpty()) { pendingActionsPartitions.forEach(tpId -> topicIdByPartitionMap.remove(tpId.topicPartition())); - remoteLogMetadataManager.onStopPartitions(pendingActionsPartitions); + remoteLogMetadataManagerPlugin.get().onStopPartitions(pendingActionsPartitions); } } private void deleteRemoteLogPartition(TopicIdPartition partition) throws RemoteStorageException, ExecutionException, InterruptedException { List metadataList = new ArrayList<>(); - remoteLogMetadataManager.listRemoteLogSegments(partition).forEachRemaining(metadataList::add); + remoteLogMetadataManagerPlugin.get().listRemoteLogSegments(partition).forEachRemaining(metadataList::add); List deleteSegmentStartedEvents = metadataList.stream() .map(metadata -> @@ -586,7 +584,7 @@ private void deleteRemoteLogPartition(TopicIdPartition partition) throws RemoteS Collection deletedSegmentIds = new ArrayList<>(); for (RemoteLogSegmentMetadata metadata: metadataList) { deletedSegmentIds.add(metadata.remoteLogSegmentId().id()); - remoteLogStorageManager.deleteLogSegmentData(metadata); + remoteLogStorageManagerPlugin.get().deleteLogSegmentData(metadata); } indexCache.removeAll(deletedSegmentIds); @@ -601,7 +599,7 @@ private void deleteRemoteLogPartition(TopicIdPartition partition) throws RemoteS private CompletableFuture publishEvents(List events) throws RemoteStorageException { List> result = new ArrayList<>(); for (RemoteLogSegmentMetadataUpdate event : events) { - result.add(remoteLogMetadataManager.updateRemoteLogSegmentMetadata(event)); + result.add(remoteLogMetadataManagerPlugin.get().updateRemoteLogSegmentMetadata(event)); } return CompletableFuture.allOf(result.toArray(new CompletableFuture[0])); } @@ -613,7 +611,7 @@ public Optional fetchRemoteLogSegmentMetadata(TopicPar if (topicId == null) { throw new KafkaException("No topic id registered for topic partition: " + topicPartition); } - return remoteLogMetadataManager.remoteLogSegmentMetadata(new TopicIdPartition(topicId, topicPartition), epochForOffset, offset); + return remoteLogMetadataManagerPlugin.get().remoteLogSegmentMetadata(new TopicIdPartition(topicId, topicPartition), epochForOffset, offset); } /** @@ -633,7 +631,7 @@ public Optional fetchNextSegmentWithTxnIndex(TopicPart throw new KafkaException("No topic id registered for topic partition: " + topicPartition); } TopicIdPartition tpId = new TopicIdPartition(topicId, topicPartition); - return remoteLogMetadataManager.nextSegmentWithTxnIndex(tpId, epochForOffset, offset); + return remoteLogMetadataManagerPlugin.get().nextSegmentWithTxnIndex(tpId, epochForOffset, offset); } Optional lookupTimestamp(RemoteLogSegmentMetadata rlsMetadata, long timestamp, long startingOffset) @@ -643,7 +641,7 @@ Optional lookupTimestamp(RemoteLogSegmentMetadat InputStream remoteSegInputStream = null; try { // Search forward for the position of the last offset that is greater than or equal to the startingOffset - remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(rlsMetadata, startPos); + remoteSegInputStream = remoteLogStorageManagerPlugin.get().fetchLogSegment(rlsMetadata, startPos); RemoteLogInputStream remoteLogInputStream = new RemoteLogInputStream(remoteSegInputStream); while (true) { @@ -733,7 +731,7 @@ public Optional findOffsetByTimestamp(TopicParti int epoch = maybeEpoch.getAsInt(); // KAFKA-15802: Add a new API for RLMM to choose how to implement the predicate. // currently, all segments are returned and then iterated, and filtered - Iterator iterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch); + Iterator iterator = remoteLogMetadataManagerPlugin.get().listRemoteLogSegments(topicIdPartition, epoch); while (iterator.hasNext()) { RemoteLogSegmentMetadata rlsMetadata = iterator.next(); if (rlsMetadata.maxTimestampMs() >= timestamp @@ -817,7 +815,7 @@ public void run() { logger.debug("Skipping the current run for partition {} as it is cancelled", topicIdPartition); return; } - if (!remoteLogMetadataManager.isReady(topicIdPartition)) { + if (!remoteLogMetadataManagerPlugin.get().isReady(topicIdPartition)) { logger.debug("Skipping the current run for partition {} as the remote-log metadata is not ready", topicIdPartition); return; } @@ -1027,7 +1025,7 @@ private void copyLogSegment(UnifiedLog log, LogSegment segment, RemoteLogSegment segment.largestTimestamp(), brokerId, time.milliseconds(), segment.log().sizeInBytes(), segmentLeaderEpochs, isTxnIdxEmpty); - remoteLogMetadataManager.addRemoteLogSegmentMetadata(copySegmentStartedRlsm).get(); + remoteLogMetadataManagerPlugin.get().addRemoteLogSegmentMetadata(copySegmentStartedRlsm).get(); ByteBuffer leaderEpochsIndex = epochEntriesAsByteBuffer(getLeaderEpochEntries(log, -1, nextSegmentBaseOffset)); LogSegmentData segmentData = new LogSegmentData(logFile.toPath(), toPathIfExists(segment.offsetIndex().file()), @@ -1038,7 +1036,7 @@ private void copyLogSegment(UnifiedLog log, LogSegment segment, RemoteLogSegment Optional customMetadata; try { - customMetadata = remoteLogStorageManager.copyLogSegmentData(copySegmentStartedRlsm, segmentData); + customMetadata = remoteLogStorageManagerPlugin.get().copyLogSegmentData(copySegmentStartedRlsm, segmentData); } catch (RemoteStorageException e) { logger.info("Copy failed, cleaning segment {}", copySegmentStartedRlsm.remoteLogSegmentId()); try { @@ -1074,7 +1072,7 @@ private void copyLogSegment(UnifiedLog log, LogSegment segment, RemoteLogSegment } } - remoteLogMetadataManager.updateRemoteLogSegmentMetadata(copySegmentFinishedRlsm).get(); + remoteLogMetadataManagerPlugin.get().updateRemoteLogSegmentMetadata(copySegmentFinishedRlsm).get(); brokerTopicStats.topicStats(log.topicPartition().topic()) .remoteCopyBytesRate().mark(copySegmentStartedRlsm.segmentSizeInBytes()); brokerTopicStats.allTopicsStats().remoteCopyBytesRate().mark(copySegmentStartedRlsm.segmentSizeInBytes()); @@ -1257,7 +1255,7 @@ void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionE final UnifiedLog log = logOptional.get(); // Cleanup remote log segments and update the log start offset if applicable. - final Iterator segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition); + final Iterator segmentMetadataIter = remoteLogMetadataManagerPlugin.get().listRemoteLogSegments(topicIdPartition); if (!segmentMetadataIter.hasNext()) { updateMetadataCountAndLogSizeWith(0, 0); logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition); @@ -1299,7 +1297,7 @@ void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionE long sizeOfDeletableSegmentsBytes = 0L; while (canProcess && epochIterator.hasNext()) { Integer epoch = epochIterator.next(); - Iterator segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch); + Iterator segmentsIterator = remoteLogMetadataManagerPlugin.get().listRemoteLogSegments(topicIdPartition, epoch); while (canProcess && segmentsIterator.hasNext()) { if (isCancelled()) { logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed."); @@ -1392,7 +1390,7 @@ void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionE while (epochsToClean.hasNext()) { int epoch = epochsToClean.next(); - Iterator segmentsToBeCleaned = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch); + Iterator segmentsToBeCleaned = remoteLogMetadataManagerPlugin.get().listRemoteLogSegments(topicIdPartition, epoch); while (segmentsToBeCleaned.hasNext()) { if (!isCancelled()) { RemoteLogSegmentMetadata nextSegmentMetadata = segmentsToBeCleaned.next(); @@ -1437,7 +1435,7 @@ private Optional buildRetentionSizeData(long retentionSize, // log size may be computed for all the segments but not for segments with in the current // partition's leader epoch lineage. Better to revisit this API. // remoteLogSizeBytes += remoteLogMetadataManager.remoteLogSize(topicIdPartition, epochEntry.epoch); - Iterator segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch); + Iterator segmentsIterator = remoteLogMetadataManagerPlugin.get().listRemoteLogSegments(topicIdPartition, epoch); while (segmentsIterator.hasNext()) { RemoteLogSegmentMetadata segmentMetadata = segmentsIterator.next(); // Count only the size of segments in "COPY_SEGMENT_FINISHED" state because @@ -1494,7 +1492,7 @@ private boolean deleteRemoteLogSegment( String topic = segmentMetadata.topicIdPartition().topic(); // Publish delete segment started event. - remoteLogMetadataManager.updateRemoteLogSegmentMetadata( + remoteLogMetadataManagerPlugin.get().updateRemoteLogSegmentMetadata( new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(), segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get(); @@ -1503,7 +1501,7 @@ private boolean deleteRemoteLogSegment( // Delete the segment in remote storage. try { - remoteLogStorageManager.deleteLogSegmentData(segmentMetadata); + remoteLogStorageManagerPlugin.get().deleteLogSegmentData(segmentMetadata); } catch (RemoteStorageException e) { brokerTopicStats.topicStats(topic).failedRemoteDeleteRequestRate().mark(); brokerTopicStats.allTopicsStats().failedRemoteDeleteRequestRate().mark(); @@ -1511,7 +1509,7 @@ private boolean deleteRemoteLogSegment( } // Publish delete segment finished event. - remoteLogMetadataManager.updateRemoteLogSegmentMetadata( + remoteLogMetadataManagerPlugin.get().updateRemoteLogSegmentMetadata( new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(), segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get(); LOGGER.debug("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId()); @@ -1707,7 +1705,7 @@ public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) throws remoteLogSegmentMetadata = rlsMetadataOptional.get(); // Search forward for the position of the last offset that is greater than or equal to the target offset startPos = lookupPositionForOffset(remoteLogSegmentMetadata, offset); - remoteSegInputStream = remoteLogStorageManager.fetchLogSegment(remoteLogSegmentMetadata, startPos); + remoteSegInputStream = remoteLogStorageManagerPlugin.get().fetchLogSegment(remoteLogSegmentMetadata, startPos); RemoteLogInputStream remoteLogInputStream = getRemoteLogInputStream(remoteSegInputStream); enrichedRecordBatch = findFirstBatch(remoteLogInputStream, offset); if (enrichedRecordBatch.batch == null) { @@ -1927,7 +1925,7 @@ OffsetAndEpoch findHighestRemoteOffset(TopicIdPartition topicIdPartition, Unifie while (offsetAndEpoch == null && maybeEpochEntry.isPresent()) { int epoch = maybeEpochEntry.get().epoch; Optional highestRemoteOffsetOpt = - remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, epoch); + remoteLogMetadataManagerPlugin.get().highestOffsetForEpoch(topicIdPartition, epoch); if (highestRemoteOffsetOpt.isPresent()) { Map.Entry entry = leaderEpochCache.endOffsetFor(epoch, log.logEndOffset()); int requestedEpoch = entry.getKey(); @@ -1958,7 +1956,7 @@ long findLogStartOffset(TopicIdPartition topicIdPartition, UnifiedLog log) throw .orElseGet(OptionalInt::empty); while (logStartOffset.isEmpty() && earliestEpochOpt.isPresent()) { Iterator iterator = - remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, earliestEpochOpt.getAsInt()); + remoteLogMetadataManagerPlugin.get().listRemoteLogSegments(topicIdPartition, earliestEpochOpt.getAsInt()); if (iterator.hasNext()) { logStartOffset = Optional.of(iterator.next().startOffset()); } @@ -2058,8 +2056,6 @@ public void close() { followerRLMTasks.values().forEach(RLMTaskWithFuture::cancel); Utils.closeQuietly(remoteLogStorageManagerPlugin, "remoteLogStorageManagerPlugin"); Utils.closeQuietly(remoteLogMetadataManagerPlugin, "remoteLogMetadataManagerPlugin"); - Utils.closeQuietly(remoteLogStorageManager, "RemoteLogStorageManager"); - Utils.closeQuietly(remoteLogMetadataManager, "RemoteLogMetadataManager"); Utils.closeQuietly(indexCache, "RemoteIndexCache"); rlmCopyThreadPool.close(); diff --git a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java index f55ea7934b8d3..75718029b9a5b 100644 --- a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java +++ b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java @@ -26,6 +26,7 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.compress.Compression; import org.apache.kafka.common.errors.ReplicaNotAvailableException; +import org.apache.kafka.common.internals.Plugin; import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Monitorable; @@ -248,11 +249,11 @@ void setUp() throws Exception { tp -> Optional.of(mockLog), (topicPartition, offset) -> currentLogStartOffset.set(offset), brokerTopicStats, metrics) { - public RemoteStorageManager createRemoteStorageManager() { - return remoteStorageManager; + public Plugin createRemoteStorageManagerPlugin() { + return Plugin.wrapInstanceDelayedInit(remoteStorageManager); } - public RemoteLogMetadataManager createRemoteLogMetadataManager() { - return remoteLogMetadataManager; + public Plugin createRemoteLogMetadataManagerPlugin() { + return Plugin.wrapInstanceDelayedInit(remoteLogMetadataManager); } public RLMQuotaManager createRLMCopyQuotaManager() { return rlmCopyQuotaManager; @@ -417,11 +418,11 @@ void testRemoteLogMetadataManagerWithEndpointConfigOverridden() throws IOExcepti (topicPartition, offset) -> { }, brokerTopicStats, metrics) { - public RemoteStorageManager createRemoteStorageManager() { - return remoteStorageManager; + public Plugin createRemoteStorageManagerPlugin() { + return Plugin.wrapInstanceDelayedInit(remoteStorageManager); } - public RemoteLogMetadataManager createRemoteLogMetadataManager() { - return remoteLogMetadataManager; + public Plugin createRemoteLogMetadataManagerPlugin() { + return Plugin.wrapInstanceDelayedInit(remoteLogMetadataManager); } }) { @@ -1363,8 +1364,8 @@ void testGetClassLoaderAwareRemoteStorageManager() throws Exception { t -> Optional.empty(), (topicPartition, offset) -> { }, brokerTopicStats, metrics) { - public RemoteStorageManager createRemoteStorageManager() { - return rsmManager; + public Plugin createRemoteStorageManagerPlugin() { + return Plugin.wrapInstanceDelayedInit(rsmManager); } } ) { @@ -1755,8 +1756,8 @@ void testFetchOffsetByTimestampWithTieredStorageDoesNotFetchIndexWhenExistsLocal (topicPartition, offset) -> currentLogStartOffset.set(offset), brokerTopicStats, metrics) { @Override - public RemoteLogMetadataManager createRemoteLogMetadataManager() { - return remoteLogMetadataManager; + public Plugin createRemoteLogMetadataManagerPlugin() { + return Plugin.wrapInstanceDelayedInit(remoteLogMetadataManager); } @Override Optional lookupTimestamp(RemoteLogSegmentMetadata rlsMetadata, long timestamp, long startingOffset) { @@ -1811,12 +1812,12 @@ public void testRemoveMetricsOnClose() throws IOException { RemoteLogManager remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), brokerId, logDir, clusterId, time, tp -> Optional.of(mockLog), (topicPartition, offset) -> { }, brokerTopicStats, metrics) { - public RemoteStorageManager createRemoteStorageManager() { - return remoteStorageManager; + public Plugin createRemoteStorageManagerPlugin() { + return Plugin.wrapInstanceDelayedInit(remoteStorageManager); } - public RemoteLogMetadataManager createRemoteLogMetadataManager() { - return remoteLogMetadataManager; + public Plugin createRemoteLogMetadataManagerPlugin() { + return Plugin.wrapInstanceDelayedInit(remoteLogMetadataManager); } }; // Close RemoteLogManager so that metrics are removed @@ -2209,8 +2210,8 @@ public void testFindLogStartOffset() throws RemoteStorageException, IOException tp -> Optional.of(mockLog), (topicPartition, offset) -> { }, brokerTopicStats, metrics) { - public RemoteLogMetadataManager createRemoteLogMetadataManager() { - return remoteLogMetadataManager; + public Plugin createRemoteLogMetadataManagerPlugin() { + return Plugin.wrapInstanceDelayedInit(remoteLogMetadataManager); } }) { assertEquals(500L, remoteLogManager.findLogStartOffset(leaderTopicIdPartition, mockLog)); @@ -2234,8 +2235,8 @@ public void testFindLogStartOffsetFallbackToLocalLogStartOffsetWhenRemoteIsEmpty tp -> Optional.of(mockLog), (topicPartition, offset) -> { }, brokerTopicStats, metrics) { - public RemoteLogMetadataManager createRemoteLogMetadataManager() { - return remoteLogMetadataManager; + public Plugin createRemoteLogMetadataManagerPlugin() { + return Plugin.wrapInstanceDelayedInit(remoteLogMetadataManager); } }) { assertEquals(250L, remoteLogManager.findLogStartOffset(leaderTopicIdPartition, mockLog)); @@ -2268,8 +2269,8 @@ public void testLogStartOffsetUpdatedOnStartup() throws RemoteStorageException, tp -> Optional.of(mockLog), (topicPartition, offset) -> logStartOffset.set(offset), brokerTopicStats, metrics) { - public RemoteLogMetadataManager createRemoteLogMetadataManager() { - return remoteLogMetadataManager; + public Plugin createRemoteLogMetadataManagerPlugin() { + return Plugin.wrapInstanceDelayedInit(remoteLogMetadataManager); } }) { RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, 128); @@ -2322,11 +2323,11 @@ public synchronized Iterator listRemoteLogSegments(Top tp -> Optional.of(mockLog), (topicPartition, offset) -> currentLogStartOffset.set(offset), brokerTopicStats, metrics) { - public RemoteStorageManager createRemoteStorageManager() { - return remoteStorageManager; + public Plugin createRemoteStorageManagerPlugin() { + return Plugin.wrapInstanceDelayedInit(remoteStorageManager); } - public RemoteLogMetadataManager createRemoteLogMetadataManager() { - return remoteLogMetadataManager; + public Plugin createRemoteLogMetadataManagerPlugin() { + return Plugin.wrapInstanceDelayedInit(remoteLogMetadataManager); } public RLMQuotaManager createRLMCopyQuotaManager() { return rlmCopyQuotaManager; @@ -2976,11 +2977,11 @@ public void testDeleteRetentionMsOnExpiredSegment() throws RemoteStorageExceptio tp -> Optional.of(mockLog), (topicPartition, offset) -> logStartOffset.set(offset), brokerTopicStats, metrics) { - public RemoteStorageManager createRemoteStorageManager() { - return remoteStorageManager; + public Plugin createRemoteStorageManagerPlugin() { + return Plugin.wrapInstanceDelayedInit(remoteStorageManager); } - public RemoteLogMetadataManager createRemoteLogMetadataManager() { - return remoteLogMetadataManager; + public Plugin createRemoteLogMetadataManagerPlugin() { + return Plugin.wrapInstanceDelayedInit(remoteLogMetadataManager); } }) { RemoteLogManager.RLMExpirationTask task = remoteLogManager.new RLMExpirationTask(leaderTopicIdPartition); @@ -3142,11 +3143,11 @@ public void testReadForMissingFirstBatchInRemote() throws RemoteStorageException (topicPartition, offset) -> { }, brokerTopicStats, metrics) { - public RemoteStorageManager createRemoteStorageManager() { - return rsmManager; + public Plugin createRemoteStorageManagerPlugin() { + return Plugin.wrapInstanceDelayedInit(rsmManager); } - public RemoteLogMetadataManager createRemoteLogMetadataManager() { - return remoteLogMetadataManager; + public Plugin createRemoteLogMetadataManagerPlugin() { + return Plugin.wrapInstanceDelayedInit(remoteLogMetadataManager); } @Override @@ -3219,11 +3220,11 @@ public void testReadForFirstBatchMoreThanMaxFetchBytes(boolean minOneMessage) th (topicPartition, offset) -> { }, brokerTopicStats, metrics) { - public RemoteStorageManager createRemoteStorageManager() { - return rsmManager; + public Plugin createRemoteStorageManagerPlugin() { + return Plugin.wrapInstanceDelayedInit(rsmManager); } - public RemoteLogMetadataManager createRemoteLogMetadataManager() { - return remoteLogMetadataManager; + public Plugin createRemoteLogMetadataManagerPlugin() { + return Plugin.wrapInstanceDelayedInit(remoteLogMetadataManager); } public Optional fetchRemoteLogSegmentMetadata(TopicPartition topicPartition, @@ -3305,12 +3306,12 @@ public void testReadForFirstBatchInLogCompaction() throws RemoteStorageException }, brokerTopicStats, metrics) { - public RemoteStorageManager createRemoteStorageManager() { - return rsmManager; + public Plugin createRemoteStorageManagerPlugin() { + return Plugin.wrapInstanceDelayedInit(remoteStorageManager); } - public RemoteLogMetadataManager createRemoteLogMetadataManager() { - return remoteLogMetadataManager; + public Plugin createRemoteLogMetadataManagerPlugin() { + return Plugin.wrapInstanceDelayedInit(remoteLogMetadataManager); } public Optional fetchRemoteLogSegmentMetadata(TopicPartition topicPartition, @@ -3655,11 +3656,11 @@ public void testRemoteReadFetchDataInfo() throws RemoteStorageException, IOExcep tp -> Optional.of(mockLog), (topicPartition, offset) -> currentLogStartOffset.set(offset), brokerTopicStats, metrics) { - public RemoteStorageManager createRemoteStorageManager() { - return remoteStorageManager; + public Plugin createRemoteStorageManagerPlugin() { + return Plugin.wrapInstanceDelayedInit(remoteStorageManager); } - public RemoteLogMetadataManager createRemoteLogMetadataManager() { - return remoteLogMetadataManager; + public Plugin createRemoteLogMetadataManagerPlugin() { + return Plugin.wrapInstanceDelayedInit(remoteLogMetadataManager); } int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) { return 0; @@ -3698,11 +3699,11 @@ public void testRLMOpsWhenMetadataIsNotReady() throws InterruptedException, IOEx tp -> Optional.of(mockLog), (topicPartition, offset) -> currentLogStartOffset.set(offset), brokerTopicStats, metrics) { - public RemoteStorageManager createRemoteStorageManager() { - return remoteStorageManager; + public Plugin createRemoteStorageManagerPlugin() { + return Plugin.wrapInstanceDelayedInit(remoteStorageManager); } - public RemoteLogMetadataManager createRemoteLogMetadataManager() { - return remoteLogMetadataManager; + public Plugin createRemoteLogMetadataManagerPlugin() { + return Plugin.wrapInstanceDelayedInit(remoteLogMetadataManager); } public RLMQuotaManager createRLMCopyQuotaManager() { return rlmCopyQuotaManager; From 518f03a1366c3426963d5bcec10e21931b02eb11 Mon Sep 17 00:00:00 2001 From: TaiJu Wu Date: Wed, 26 Mar 2025 05:01:34 +0000 Subject: [PATCH 11/24] refactor --- .../java/org/apache/kafka/common/internals/Plugin.java | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/internals/Plugin.java b/clients/src/main/java/org/apache/kafka/common/internals/Plugin.java index c32105ec169f8..62b3c2f5070bb 100644 --- a/clients/src/main/java/org/apache/kafka/common/internals/Plugin.java +++ b/clients/src/main/java/org/apache/kafka/common/internals/Plugin.java @@ -73,15 +73,9 @@ public static Plugin wrapInstanceDelayedInit(T instance) { } public Plugin initialize(Metrics metrics, String key) { - PluginMetricsImpl pluginMetrics = null; - if (instance instanceof Monitorable && metrics != null) { - pluginMetrics = new PluginMetricsImpl(metrics, tags(key, instance)); - ((Monitorable) instance).withPluginMetrics(pluginMetrics); - } - return this; + return wrapInstance(instance, metrics, key); } - @Override public T get() { return instance; From d4ef41dc896e4d5670b352bc2370e1a188195908 Mon Sep 17 00:00:00 2001 From: TaiJu Wu Date: Wed, 26 Mar 2025 07:02:28 +0000 Subject: [PATCH 12/24] improve test --- .../java/kafka/log/remote/RemoteLogManagerTest.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java index 75718029b9a5b..9fcbc2900f15b 100644 --- a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java +++ b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java @@ -3763,10 +3763,15 @@ public void testMonitorableRemoteLogStorageManager() throws IOException { (topicPartition, offset) -> { }, brokerTopicStats, metrics)) { - // We need to call startup for call config and wrap instance - remoteLogManager.startup(); assertInstanceOf(MonitorableNoOpRemoteStorageManager.class, remoteLogManager.storageManager()); assertInstanceOf(MonitorableNoOpRemoteLogMetadataManager.class, remoteLogManager.remoteLogMetadataManager()); + MonitorableNoOpRemoteStorageManager rsm = (MonitorableNoOpRemoteStorageManager) remoteLogManager.storageManager(); + MonitorableNoOpRemoteLogMetadataManager rlm = (MonitorableNoOpRemoteLogMetadataManager) remoteLogManager.remoteLogMetadataManager(); + + assertEquals(false, rsm.pluginMetrics); + assertEquals(false, rlm.pluginMetrics); + // We need to call startup for call config() and call withPluginMetrics() + remoteLogManager.startup(); assertEquals(true, ((MonitorableNoOpRemoteStorageManager) remoteLogManager.storageManager()).pluginMetrics); assertEquals(true, ((MonitorableNoOpRemoteLogMetadataManager) remoteLogManager.remoteLogMetadataManager()).pluginMetrics); } @@ -3819,8 +3824,6 @@ private void appendRLMConfig(Properties props) { public static class MonitorableNoOpRemoteStorageManager extends NoOpRemoteStorageManager implements Monitorable { public boolean pluginMetrics = false; - public MonitorableNoOpRemoteStorageManager() { } - @Override public void withPluginMetrics(PluginMetrics metrics) { pluginMetrics = true; From 4f07357a6d095d4673df1bfc75d787cf99bba0af Mon Sep 17 00:00:00 2001 From: TaiJu Wu Date: Mon, 31 Mar 2025 14:55:50 +0000 Subject: [PATCH 13/24] apply Maison's implementation --- .../apache/kafka/common/internals/Plugin.java | 8 - .../kafka/log/remote/RemoteLogManager.java | 61 ++--- .../scala/kafka/server/BrokerServer.scala | 44 ++-- .../log/remote/RemoteLogManagerTest.java | 242 +++++++++--------- .../log/remote/RemoteLogOffsetReaderTest.java | 3 +- .../scala/unit/kafka/log/UnifiedLogTest.scala | 6 +- .../kafka/server/ReplicaManagerTest.scala | 8 +- 7 files changed, 174 insertions(+), 198 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/internals/Plugin.java b/clients/src/main/java/org/apache/kafka/common/internals/Plugin.java index 62b3c2f5070bb..620cd0c07ec0f 100644 --- a/clients/src/main/java/org/apache/kafka/common/internals/Plugin.java +++ b/clients/src/main/java/org/apache/kafka/common/internals/Plugin.java @@ -68,14 +68,6 @@ public static Plugin wrapInstance(T instance, Metrics metrics, Supplier(instance, pluginMetrics); } - public static Plugin wrapInstanceDelayedInit(T instance) { - return new Plugin<>(instance, null); - } - - public Plugin initialize(Metrics metrics, String key) { - return wrapInstance(instance, metrics, key); - } - @Override public T get() { return instance; diff --git a/core/src/main/java/kafka/log/remote/RemoteLogManager.java b/core/src/main/java/kafka/log/remote/RemoteLogManager.java index baa6acf8650a5..d0b269cafcbdb 100644 --- a/core/src/main/java/kafka/log/remote/RemoteLogManager.java +++ b/core/src/main/java/kafka/log/remote/RemoteLogManager.java @@ -228,7 +228,8 @@ public RemoteLogManager(RemoteLogManagerConfig rlmConfig, Function> fetchLog, BiConsumer updateRemoteLogStartOffset, BrokerTopicStats brokerTopicStats, - Metrics metrics) throws IOException { + Metrics metrics, + Optional endpoint) throws IOException { this.rlmConfig = rlmConfig; this.brokerId = brokerId; this.logDir = logDir; @@ -238,9 +239,12 @@ public RemoteLogManager(RemoteLogManagerConfig rlmConfig, this.updateRemoteLogStartOffset = updateRemoteLogStartOffset; this.brokerTopicStats = brokerTopicStats; this.metrics = metrics; + this.endpoint = endpoint; + + remoteLogStorageManagerPlugin = configAndWrapRSM(createRemoteStorageManager()); + remoteLogMetadataManagerPlugin = configAndWrapRLMM(createRemoteLogMetadataManager()); + remoteLogManagerConfigured = true; - remoteLogStorageManagerPlugin = createRemoteStorageManagerPlugin(); - remoteLogMetadataManagerPlugin = createRemoteLogMetadataManagerPlugin(); rlmCopyQuotaManager = createRLMCopyQuotaManager(); rlmFetchQuotaManager = createRLMFetchQuotaManager(); @@ -359,48 +363,49 @@ private T createDelegate(ClassLoader classLoader, String className) { } } - Plugin createRemoteStorageManagerPlugin() { + RemoteStorageManager createRemoteStorageManager() { return SecurityManagerCompatibility.get().doPrivileged(() -> { final String classPath = rlmConfig.remoteStorageManagerClassPath(); if (classPath != null && !classPath.trim().isEmpty()) { ChildFirstClassLoader classLoader = new ChildFirstClassLoader(classPath, this.getClass().getClassLoader()); RemoteStorageManager delegate = createDelegate(classLoader, rlmConfig.remoteStorageManagerClassName()); - return Plugin.wrapInstanceDelayedInit(new ClassLoaderAwareRemoteStorageManager(delegate, classLoader)); + return (RemoteStorageManager) new ClassLoaderAwareRemoteStorageManager(delegate, classLoader); } else { - return Plugin.wrapInstanceDelayedInit(createDelegate(this.getClass().getClassLoader(), rlmConfig.remoteStorageManagerClassName())); + return createDelegate(this.getClass().getClassLoader(), rlmConfig.remoteStorageManagerClassName()); } }); } - private void configureRSM() { + private void configureRSM(RemoteStorageManager remoteLogStorageManager) { final Map rsmProps = new HashMap<>(rlmConfig.remoteStorageManagerProps()); rsmProps.put(ServerConfigs.BROKER_ID_CONFIG, brokerId); - remoteLogStorageManagerPlugin.get().configure(rsmProps); + remoteLogStorageManager.configure(rsmProps); + } + + private Plugin configAndWrapRSM(RemoteStorageManager rsm) { + configureRSM(rsm); + return Plugin.wrapInstance(rsm, metrics, RemoteLogManagerConfig.REMOTE_STORAGE_MANAGER_CLASS_NAME_PROP); } - Plugin createRemoteLogMetadataManagerPlugin() { + RemoteLogMetadataManager createRemoteLogMetadataManager() { return SecurityManagerCompatibility.get().doPrivileged(() -> { final String classPath = rlmConfig.remoteLogMetadataManagerClassPath(); if (classPath != null && !classPath.trim().isEmpty()) { ClassLoader classLoader = new ChildFirstClassLoader(classPath, this.getClass().getClassLoader()); RemoteLogMetadataManager delegate = createDelegate(classLoader, rlmConfig.remoteLogMetadataManagerClassName()); - return Plugin.wrapInstanceDelayedInit(new ClassLoaderAwareRemoteLogMetadataManager(delegate, classLoader)); + return (RemoteLogMetadataManager) new ClassLoaderAwareRemoteLogMetadataManager(delegate, classLoader); } else { - return Plugin.wrapInstanceDelayedInit(createDelegate(this.getClass().getClassLoader(), rlmConfig.remoteLogMetadataManagerClassName())); + return createDelegate(this.getClass().getClassLoader(), rlmConfig.remoteLogMetadataManagerClassName()); } }); } - - RemoteLogMetadataManager remoteLogMetadataManager() { - return remoteLogMetadataManagerPlugin.get(); + private Plugin configAndWrapRLMM(RemoteLogMetadataManager rlmm) { + configureRLMM(rlmm); + return Plugin.wrapInstance(rlmm, metrics, RemoteLogManagerConfig.REMOTE_LOG_METADATA_MANAGER_CLASS_NAME_PROP); } - public void onEndPointCreated(Endpoint endpoint) { - this.endpoint = Optional.of(endpoint); - } - - private void configureRLMM() { + private void configureRLMM(RemoteLogMetadataManager remoteLogMetadataManager) { final Map rlmmProps = new HashMap<>(); endpoint.ifPresent(e -> { rlmmProps.put(REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX + "bootstrap.servers", e.host() + ":" + e.port()); @@ -413,22 +418,12 @@ private void configureRLMM() { rlmmProps.put(LOG_DIR_CONFIG, logDir); rlmmProps.put("cluster.id", clusterId); - remoteLogMetadataManagerPlugin.get().configure(rlmmProps); + remoteLogMetadataManager.configure(rlmmProps); } - public void startup() { - // Initialize and configure RSM and RLMM. This will start RSM, RLMM resources which may need to start resources - // in connecting to the brokers or remote storages. - configureRSM(); - configureRLMM(); - // the withPluginMetrics() method will be called when the plugin is instantiated (after configure() if the plugin also implements Configurable) - remoteLogStorageManagerPlugin.initialize(metrics, rlmConfig.remoteStorageManagerClassName()); - remoteLogMetadataManagerPlugin.initialize(metrics, rlmConfig.remoteLogMetadataManagerClassName()); - remoteLogManagerConfigured = true; - } - private boolean isRemoteLogManagerConfigured() { - return this.remoteLogManagerConfigured; + RemoteLogMetadataManager remoteLogMetadataManager() { + return remoteLogMetadataManagerPlugin.get(); } public RemoteStorageManager storageManager() { @@ -462,7 +457,7 @@ public void onLeadershipChange(Set partitionsBecomeLeader, Map topicIds) { LOGGER.debug("Received leadership changes for leaders: {} and followers: {}", partitionsBecomeLeader, partitionsBecomeFollower); - if (rlmConfig.isRemoteStorageSystemEnabled() && !isRemoteLogManagerConfigured()) { + if (rlmConfig.isRemoteStorageSystemEnabled() && !this.remoteLogManagerConfigured) { throw new KafkaException("RemoteLogManager is not configured when remote storage system is enabled"); } diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index ace63454f93ac..f52ced9babf3a 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -220,8 +220,6 @@ class BrokerServer( brokerTopicStats, logDirFailureChannel) - remoteLogManagerOpt = createRemoteLogManager() - lifecycleManager = new BrokerLifecycleManager(config, time, s"broker-${config.nodeId}-", @@ -280,6 +278,8 @@ class BrokerServer( withWildcardHostnamesResolved(). withEphemeralPortsCorrected(name => socketServer.boundPort(new ListenerName(name))) + remoteLogManagerOpt = createRemoteLogManager(listenerInfo) + alterPartitionManager = AlterPartitionManager( config, scheduler = kafkaScheduler, @@ -473,22 +473,7 @@ class BrokerServer( config.numIoThreads, s"${DataPlaneAcceptor.MetricPrefix}RequestHandlerAvgIdlePercent", DataPlaneAcceptor.ThreadPrefix) - // Start RemoteLogManager before initializing broker metadata publishers. - remoteLogManagerOpt.foreach { rlm => - val listenerName = config.remoteLogManagerConfig.remoteLogMetadataManagerListenerName() - if (listenerName != null) { - val endpoint = listenerInfo.listeners().values().stream - .filter(e => - e.listenerName().isPresent && - ListenerName.normalised(e.listenerName().get()).equals(ListenerName.normalised(listenerName)) - ) - .findFirst() - .orElseThrow(() => new ConfigException(RemoteLogManagerConfig.REMOTE_LOG_METADATA_MANAGER_LISTENER_NAME_PROP, - listenerName, "Should be set as a listener name within valid broker listener name list: " + listenerInfo.listeners().values())) - rlm.onEndPointCreated(endpoint) - } - rlm.startup() - } + metadataPublishers.add(new MetadataVersionConfigValidator(config, sharedServer.metadataPublishingFaultHandler)) brokerMetadataPublisher = new BrokerMetadataPublisher(config, @@ -714,16 +699,31 @@ class BrokerServer( } } - protected def createRemoteLogManager(): Option[RemoteLogManager] = { - if (config.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) { - Some(new RemoteLogManager(config.remoteLogManagerConfig, config.brokerId, config.logDirs.head, clusterId, time, + protected def createRemoteLogManager(listenerInfo: ListenerInfo): Option[RemoteLogManager] = { + if (config.remoteLogManagerConfig.isRemoteStorageSystemEnabled) { + val listenerName = config.remoteLogManagerConfig.remoteLogMetadataManagerListenerName() + val endpoint = if (listenerName != null) { + Some(listenerInfo.listeners().values().stream + .filter(e => + e.listenerName().isPresent && + ListenerName.normalised(e.listenerName().get()).equals(ListenerName.normalised(listenerName)) + ) + .findFirst() + .orElseThrow(() => new ConfigException(RemoteLogManagerConfig.REMOTE_LOG_METADATA_MANAGER_LISTENER_NAME_PROP, + listenerName, "Should be set as a listener name within valid broker listener name list: " + listenerInfo.listeners().values()))) + } else { + None + } + + val rlm = new RemoteLogManager(config.remoteLogManagerConfig, config.brokerId, config.logDirs.head, clusterId, time, (tp: TopicPartition) => logManager.getLog(tp).toJava, (tp: TopicPartition, remoteLogStartOffset: java.lang.Long) => { logManager.getLog(tp).foreach { log => log.updateLogStartOffsetFromRemoteTier(remoteLogStartOffset) } }, - brokerTopicStats, metrics)) + brokerTopicStats, metrics, endpoint.toJava) + Some(rlm) } else { None } diff --git a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java index 9fcbc2900f15b..9b78910197a31 100644 --- a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java +++ b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java @@ -26,7 +26,6 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.compress.Compression; import org.apache.kafka.common.errors.ReplicaNotAvailableException; -import org.apache.kafka.common.internals.Plugin; import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Monitorable; @@ -233,6 +232,11 @@ public class RemoteLogManagerTest { private final MockScheduler scheduler = new MockScheduler(time); private final Properties brokerConfig = kafka.utils.TestUtils.createDummyBrokerConfig(); + private final String host = "localhost"; + private final int port = 1234; + private final String securityProtocol = "PLAINTEXT"; + private final Optional endPoint = Optional.of(new Endpoint(securityProtocol, SecurityProtocol.PLAINTEXT, host, port)); + @BeforeEach void setUp() throws Exception { checkpoint = new LeaderEpochCheckpointFile(TestUtils.tempFile(), new LogDirFailureChannel(1)); @@ -248,16 +252,20 @@ void setUp() throws Exception { remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), brokerId, logDir, clusterId, time, tp -> Optional.of(mockLog), (topicPartition, offset) -> currentLogStartOffset.set(offset), - brokerTopicStats, metrics) { - public Plugin createRemoteStorageManagerPlugin() { - return Plugin.wrapInstanceDelayedInit(remoteStorageManager); + brokerTopicStats, metrics, endPoint) { + @Override + public RemoteStorageManager createRemoteStorageManager() { + return remoteStorageManager; } - public Plugin createRemoteLogMetadataManagerPlugin() { - return Plugin.wrapInstanceDelayedInit(remoteLogMetadataManager); + @Override + public RemoteLogMetadataManager createRemoteLogMetadataManager() { + return remoteLogMetadataManager; } + @Override public RLMQuotaManager createRLMCopyQuotaManager() { return rlmCopyQuotaManager; } + @Override public Duration quotaTimeout() { return Duration.ofMillis(100); } @@ -385,13 +393,6 @@ void testRemoteStorageManagerWithUserDefinedConfigs() { @Test void testRemoteLogMetadataManagerWithEndpointConfig() { - String host = "localhost"; - int port = 1234; - String securityProtocol = "PLAINTEXT"; - Endpoint endPoint = new Endpoint(securityProtocol, SecurityProtocol.PLAINTEXT, host, port); - remoteLogManager.onEndPointCreated(endPoint); - remoteLogManager.startup(); - ArgumentCaptor> capture = ArgumentCaptor.forClass(Map.class); verify(remoteLogMetadataManager, times(1)).configure(capture.capture()); assertEquals(host + ":" + port, capture.getValue().get(REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX + "bootstrap.servers")); @@ -417,24 +418,19 @@ void testRemoteLogMetadataManagerWithEndpointConfigOverridden() throws IOExcepti tp -> Optional.of(mockLog), (topicPartition, offset) -> { }, brokerTopicStats, - metrics) { - public Plugin createRemoteStorageManagerPlugin() { - return Plugin.wrapInstanceDelayedInit(remoteStorageManager); + metrics, + endPoint) { + @Override + public RemoteStorageManager createRemoteStorageManager() { + return remoteStorageManager; } - public Plugin createRemoteLogMetadataManagerPlugin() { - return Plugin.wrapInstanceDelayedInit(remoteLogMetadataManager); + @Override + public RemoteLogMetadataManager createRemoteLogMetadataManager() { + return remoteLogMetadataManager; } }) { - - String host = "localhost"; - int port = 1234; - String securityProtocol = "PLAINTEXT"; - Endpoint endpoint = new Endpoint(securityProtocol, SecurityProtocol.PLAINTEXT, host, port); - remoteLogManager.onEndPointCreated(endpoint); - remoteLogManager.startup(); - ArgumentCaptor> capture = ArgumentCaptor.forClass(Map.class); - verify(remoteLogMetadataManager, times(1)).configure(capture.capture()); + verify(remoteLogMetadataManager, times(2)).configure(capture.capture()); assertEquals(host + ":" + port, capture.getValue().get(REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX + "bootstrap.servers")); // should be overridden as SSL assertEquals("SSL", capture.getValue().get(REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX + "security.protocol")); @@ -445,7 +441,6 @@ public Plugin createRemoteLogMetadataManagerPlugin() { @Test void testStartup() { - remoteLogManager.startup(); ArgumentCaptor> capture = ArgumentCaptor.forClass(Map.class); verify(remoteStorageManager, times(1)).configure(capture.capture()); assertEquals(brokerId, capture.getValue().get("broker.id")); @@ -780,17 +775,8 @@ void testFailedCopyShouldDeleteTheDanglingSegment() throws Exception { assertEquals(1, brokerTopicStats.allTopicsStats().failedRemoteCopyRequestRate().count()); } - @Test - void testLeadershipChangesWithoutRemoteLogManagerConfiguring() { - assertThrows(KafkaException.class, () -> - remoteLogManager.onLeadershipChange( - Collections.singleton(mockPartition(leaderTopicIdPartition)), Collections.singleton(mockPartition(followerTopicIdPartition)), topicIds), - "RemoteLogManager is not configured when remote storage system is enabled"); - } - @Test void testRemoteLogManagerTasksAvgIdlePercentAndMetadataCountMetrics() throws Exception { - remoteLogManager.startup(); long oldSegmentStartOffset = 0L; long nextSegmentStartOffset = 150L; int segmentCount = 3; @@ -909,7 +895,6 @@ void testRemoteLogManagerTasksAvgIdlePercentAndMetadataCountMetrics() throws Exc @Test void testRemoteLogTaskUpdateRemoteLogSegmentMetadataAfterLogDirChanged() throws Exception { - remoteLogManager.startup(); long oldSegmentStartOffset = 0L; long nextSegmentStartOffset = 150L; int segmentCount = 3; @@ -1024,7 +1009,6 @@ void testRemoteLogTaskUpdateRemoteLogSegmentMetadataAfterLogDirChanged() throws @Test void testRemoteLogManagerRemoteMetrics() throws Exception { - remoteLogManager.startup(); long oldestSegmentStartOffset = 0L; long olderSegmentStartOffset = 75L; long nextSegmentStartOffset = 150L; @@ -1363,9 +1347,10 @@ void testGetClassLoaderAwareRemoteStorageManager() throws Exception { new RemoteLogManager(config.remoteLogManagerConfig(), brokerId, logDir, clusterId, time, t -> Optional.empty(), (topicPartition, offset) -> { }, - brokerTopicStats, metrics) { - public Plugin createRemoteStorageManagerPlugin() { - return Plugin.wrapInstanceDelayedInit(rsmManager); + brokerTopicStats, metrics, endPoint) { + @Override + public RemoteStorageManager createRemoteStorageManager() { + return rsmManager; } } ) { @@ -1388,7 +1373,6 @@ private void verifyNotInCache(TopicIdPartition... topicIdPartitions) { @Test void testTopicIdCacheUpdates() throws RemoteStorageException { - remoteLogManager.startup(); Partition mockLeaderPartition = mockPartition(leaderTopicIdPartition); Partition mockFollowerPartition = mockPartition(followerTopicIdPartition); @@ -1413,7 +1397,6 @@ void testTopicIdCacheUpdates() throws RemoteStorageException { @Test void testFetchRemoteLogSegmentMetadata() throws RemoteStorageException { - remoteLogManager.startup(); remoteLogManager.onLeadershipChange( Collections.singleton(mockPartition(leaderTopicIdPartition)), Collections.singleton(mockPartition(followerTopicIdPartition)), topicIds); remoteLogManager.fetchRemoteLogSegmentMetadata(leaderTopicIdPartition.topicPartition(), 10, 100L); @@ -1427,7 +1410,6 @@ void testFetchRemoteLogSegmentMetadata() throws RemoteStorageException { @Test public void testFetchNextSegmentWithTxnIndex() throws RemoteStorageException { - remoteLogManager.startup(); remoteLogManager.onLeadershipChange( Collections.singleton(mockPartition(leaderTopicIdPartition)), Collections.singleton(mockPartition(followerTopicIdPartition)), topicIds); remoteLogManager.fetchNextSegmentWithTxnIndex(leaderTopicIdPartition.topicPartition(), 10, 100L); @@ -1460,7 +1442,6 @@ public void testFindNextSegmentWithTxnIndex() throws RemoteStorageException { return Optional.of(metadata); }); - remoteLogManager.startup(); remoteLogManager.onLeadershipChange( Collections.singleton(mockPartition(leaderTopicIdPartition)), Collections.singleton(mockPartition(followerTopicIdPartition)), topicIds); @@ -1495,7 +1476,6 @@ public void testFindNextSegmentWithTxnIndexTraversesNextEpoch() throws RemoteSto return metadataOpt; }); - remoteLogManager.startup(); remoteLogManager.onLeadershipChange( Collections.singleton(mockPartition(leaderTopicIdPartition)), Collections.singleton(mockPartition(followerTopicIdPartition)), topicIds); @@ -1513,7 +1493,6 @@ public void testFindNextSegmentWithTxnIndexTraversesNextEpoch() throws RemoteSto @Test void testOnLeadershipChangeWillInvokeHandleLeaderOrFollowerPartitions() { - remoteLogManager.startup(); RemoteLogManager spyRemoteLogManager = spy(remoteLogManager); spyRemoteLogManager.onLeadershipChange( Collections.emptySet(), Collections.singleton(mockPartition(followerTopicIdPartition)), topicIds); @@ -1538,7 +1517,6 @@ private MemoryRecords records(long timestamp, @Test void testFindOffsetByTimestamp() throws IOException, RemoteStorageException { - remoteLogManager.startup(); TopicPartition tp = leaderTopicIdPartition.topicPartition(); long ts = time.milliseconds(); @@ -1572,7 +1550,6 @@ void testFindOffsetByTimestamp() throws IOException, RemoteStorageException { @Test void testFindOffsetByTimestampWithInvalidEpochSegments() throws IOException, RemoteStorageException { - remoteLogManager.startup(); TopicPartition tp = leaderTopicIdPartition.topicPartition(); long ts = time.milliseconds(); @@ -1604,7 +1581,6 @@ void testFindOffsetByTimestampWithInvalidEpochSegments() throws IOException, Rem @Test void testFindOffsetByTimestampWithSegmentNotReady() throws IOException, RemoteStorageException { - remoteLogManager.startup(); TopicPartition tp = leaderTopicIdPartition.topicPartition(); long ts = time.milliseconds(); @@ -1754,17 +1730,16 @@ void testFetchOffsetByTimestampWithTieredStorageDoesNotFetchIndexWhenExistsLocal remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), brokerId, logDir, clusterId, time, partition -> Optional.of(mockLog), (topicPartition, offset) -> currentLogStartOffset.set(offset), - brokerTopicStats, metrics) { + brokerTopicStats, metrics, endPoint) { @Override - public Plugin createRemoteLogMetadataManagerPlugin() { - return Plugin.wrapInstanceDelayedInit(remoteLogMetadataManager); + public RemoteLogMetadataManager createRemoteLogMetadataManager() { + return remoteLogMetadataManager; } @Override Optional lookupTimestamp(RemoteLogSegmentMetadata rlsMetadata, long timestamp, long startingOffset) { return Optional.of(expectedRemoteResult); } }; - remoteLogManager.startup(); remoteLogManager.onLeadershipChange(Collections.emptySet(), Collections.singleton(mockFollowerPartition), topicIds); // Read the offset from the remote storage, since the local-log starts from offset 50L and the message with `timestamp` does not exist in the local log @@ -1811,13 +1786,14 @@ public void testRemoveMetricsOnClose() throws IOException { try (MockedConstruction mockMetricsGroupCtor = mockConstruction(KafkaMetricsGroup.class)) { RemoteLogManager remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), brokerId, logDir, clusterId, time, tp -> Optional.of(mockLog), (topicPartition, offset) -> { - }, brokerTopicStats, metrics) { - public Plugin createRemoteStorageManagerPlugin() { - return Plugin.wrapInstanceDelayedInit(remoteStorageManager); + }, brokerTopicStats, metrics, endPoint) { + @Override + public RemoteStorageManager createRemoteStorageManager() { + return remoteStorageManager; } - - public Plugin createRemoteLogMetadataManagerPlugin() { - return Plugin.wrapInstanceDelayedInit(remoteLogMetadataManager); + @Override + public RemoteLogMetadataManager createRemoteLogMetadataManager() { + return remoteLogMetadataManager; } }; // Close RemoteLogManager so that metrics are removed @@ -2121,7 +2097,6 @@ public void testRemoteSizeTime() { @Test public void testStopPartitionsWithoutDeletion() throws RemoteStorageException { - remoteLogManager.startup(); BiConsumer errorHandler = (topicPartition, throwable) -> fail("shouldn't be called"); Set partitions = new HashSet<>(); partitions.add(new StopPartition(leaderTopicIdPartition.topicPartition(), true, false, false)); @@ -2143,7 +2118,6 @@ public void testStopPartitionsWithoutDeletion() throws RemoteStorageException { @Test public void testStopPartitionsWithDeletion() throws RemoteStorageException { - remoteLogManager.startup(); BiConsumer errorHandler = (topicPartition, ex) -> fail("shouldn't be called: " + ex); Set partitions = new HashSet<>(); @@ -2209,9 +2183,10 @@ public void testFindLogStartOffset() throws RemoteStorageException, IOException try (RemoteLogManager remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), brokerId, logDir, clusterId, time, tp -> Optional.of(mockLog), (topicPartition, offset) -> { }, - brokerTopicStats, metrics) { - public Plugin createRemoteLogMetadataManagerPlugin() { - return Plugin.wrapInstanceDelayedInit(remoteLogMetadataManager); + brokerTopicStats, metrics, endPoint) { + @Override + public RemoteLogMetadataManager createRemoteLogMetadataManager() { + return remoteLogMetadataManager; } }) { assertEquals(500L, remoteLogManager.findLogStartOffset(leaderTopicIdPartition, mockLog)); @@ -2234,9 +2209,10 @@ public void testFindLogStartOffsetFallbackToLocalLogStartOffsetWhenRemoteIsEmpty try (RemoteLogManager remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), brokerId, logDir, clusterId, time, tp -> Optional.of(mockLog), (topicPartition, offset) -> { }, - brokerTopicStats, metrics) { - public Plugin createRemoteLogMetadataManagerPlugin() { - return Plugin.wrapInstanceDelayedInit(remoteLogMetadataManager); + brokerTopicStats, metrics, endPoint) { + @Override + public RemoteLogMetadataManager createRemoteLogMetadataManager() { + return remoteLogMetadataManager; } }) { assertEquals(250L, remoteLogManager.findLogStartOffset(leaderTopicIdPartition, mockLog)); @@ -2268,9 +2244,10 @@ public void testLogStartOffsetUpdatedOnStartup() throws RemoteStorageException, try (RemoteLogManager remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), brokerId, logDir, clusterId, time, tp -> Optional.of(mockLog), (topicPartition, offset) -> logStartOffset.set(offset), - brokerTopicStats, metrics) { - public Plugin createRemoteLogMetadataManagerPlugin() { - return Plugin.wrapInstanceDelayedInit(remoteLogMetadataManager); + brokerTopicStats, metrics, endPoint) { + @Override + public RemoteLogMetadataManager createRemoteLogMetadataManager() { + return remoteLogMetadataManager; } }) { RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, 128); @@ -2322,16 +2299,20 @@ public synchronized Iterator listRemoteLogSegments(Top remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), brokerId, logDir, clusterId, time, tp -> Optional.of(mockLog), (topicPartition, offset) -> currentLogStartOffset.set(offset), - brokerTopicStats, metrics) { - public Plugin createRemoteStorageManagerPlugin() { - return Plugin.wrapInstanceDelayedInit(remoteStorageManager); + brokerTopicStats, metrics, endPoint) { + @Override + public RemoteStorageManager createRemoteStorageManager() { + return remoteStorageManager; } - public Plugin createRemoteLogMetadataManagerPlugin() { - return Plugin.wrapInstanceDelayedInit(remoteLogMetadataManager); + @Override + public RemoteLogMetadataManager createRemoteLogMetadataManager() { + return remoteLogMetadataManager; } + @Override public RLMQuotaManager createRLMCopyQuotaManager() { return rlmCopyQuotaManager; } + @Override public Duration quotaTimeout() { return Duration.ofMillis(100); } @@ -2976,12 +2957,12 @@ public void testDeleteRetentionMsOnExpiredSegment() throws RemoteStorageExceptio try (RemoteLogManager remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), brokerId, logDir, clusterId, time, tp -> Optional.of(mockLog), (topicPartition, offset) -> logStartOffset.set(offset), - brokerTopicStats, metrics) { - public Plugin createRemoteStorageManagerPlugin() { - return Plugin.wrapInstanceDelayedInit(remoteStorageManager); + brokerTopicStats, metrics, endPoint) { + public RemoteStorageManager createRemoteStorageManager() { + return remoteStorageManager; } - public Plugin createRemoteLogMetadataManagerPlugin() { - return Plugin.wrapInstanceDelayedInit(remoteLogMetadataManager); + public RemoteLogMetadataManager createRemoteLogMetadataManager() { + return remoteLogMetadataManager; } }) { RemoteLogManager.RLMExpirationTask task = remoteLogManager.new RLMExpirationTask(leaderTopicIdPartition); @@ -3015,7 +2996,6 @@ public Plugin createRemoteLogMetadataManagerPlugin() { task.cleanupExpiredRemoteLogSegments(); - verifyNoMoreInteractions(remoteStorageManager); assertEquals(0L, logStartOffset.get()); } catch (ExecutionException | InterruptedException e) { throw new RuntimeException(e); @@ -3142,12 +3122,15 @@ public void testReadForMissingFirstBatchInRemote() throws RemoteStorageException tp -> Optional.of(mockLog), (topicPartition, offset) -> { }, brokerTopicStats, - metrics) { - public Plugin createRemoteStorageManagerPlugin() { - return Plugin.wrapInstanceDelayedInit(rsmManager); + metrics, + endPoint) { + @Override + public RemoteStorageManager createRemoteStorageManager() { + return rsmManager; } - public Plugin createRemoteLogMetadataManagerPlugin() { - return Plugin.wrapInstanceDelayedInit(remoteLogMetadataManager); + @Override + public RemoteLogMetadataManager createRemoteLogMetadataManager() { + return remoteLogMetadataManager; } @Override @@ -3219,23 +3202,25 @@ public void testReadForFirstBatchMoreThanMaxFetchBytes(boolean minOneMessage) th tp -> Optional.of(mockLog), (topicPartition, offset) -> { }, brokerTopicStats, - metrics) { - public Plugin createRemoteStorageManagerPlugin() { - return Plugin.wrapInstanceDelayedInit(rsmManager); + metrics, + endPoint) { + @Override + public RemoteStorageManager createRemoteStorageManager() { + return rsmManager; } - public Plugin createRemoteLogMetadataManagerPlugin() { - return Plugin.wrapInstanceDelayedInit(remoteLogMetadataManager); + @Override + public RemoteLogMetadataManager createRemoteLogMetadataManager() { + return remoteLogMetadataManager; } - - public Optional fetchRemoteLogSegmentMetadata(TopicPartition topicPartition, - int epochForOffset, long offset) { + @Override + public Optional fetchRemoteLogSegmentMetadata(TopicPartition topicPartition, int epochForOffset, long offset) { return Optional.of(segmentMetadata); } - + @Override int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) { return 1; } - + @Override EnrichedRecordBatch findFirstBatch(RemoteLogInputStream remoteLogInputStream, long offset) { when(firstBatch.sizeInBytes()).thenReturn(recordBatchSizeInBytes); doNothing().when(firstBatch).writeTo(capture.capture()); @@ -3305,23 +3290,25 @@ public void testReadForFirstBatchInLogCompaction() throws RemoteStorageException (topicPartition, offset) -> { }, brokerTopicStats, - metrics) { - public Plugin createRemoteStorageManagerPlugin() { - return Plugin.wrapInstanceDelayedInit(remoteStorageManager); + metrics, + endPoint) { + @Override + public RemoteStorageManager createRemoteStorageManager() { + return rsmManager; } - - public Plugin createRemoteLogMetadataManagerPlugin() { - return Plugin.wrapInstanceDelayedInit(remoteLogMetadataManager); + @Override + public RemoteLogMetadataManager createRemoteLogMetadataManager() { + return remoteLogMetadataManager; } - - public Optional fetchRemoteLogSegmentMetadata(TopicPartition topicPartition, - int epochForOffset, long offset) { + @Override + public Optional fetchRemoteLogSegmentMetadata(TopicPartition topicPartition, int epochForOffset, long offset) { return Optional.of(segmentMetadata); } + @Override public RemoteLogInputStream getRemoteLogInputStream(InputStream in) { return remoteLogInputStream; } - + @Override int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) { return 1; } @@ -3451,7 +3438,6 @@ public void testCopyQuota(boolean quotaExceeded) throws Exception { @Test public void testRLMShutdownDuringQuotaExceededScenario() throws Exception { - remoteLogManager.startup(); setupRLMTask(true); remoteLogManager.onLeadershipChange( Collections.singleton(mockPartition(leaderTopicIdPartition)), Collections.emptySet(), topicIds); @@ -3612,7 +3598,6 @@ public void testCopyThrottling() throws Exception { @Test public void testTierLagResetsToZeroOnBecomingFollower() { - remoteLogManager.startup(); remoteLogManager.onLeadershipChange( Collections.singleton(mockPartition(leaderTopicIdPartition)), Collections.emptySet(), topicIds); RemoteLogManager.RLMCopyTask rlmTask = (RemoteLogManager.RLMCopyTask) remoteLogManager.rlmCopyTask(leaderTopicIdPartition); @@ -3655,18 +3640,20 @@ public void testRemoteReadFetchDataInfo() throws RemoteStorageException, IOExcep RemoteLogManager remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), brokerId, logDir, clusterId, time, tp -> Optional.of(mockLog), (topicPartition, offset) -> currentLogStartOffset.set(offset), - brokerTopicStats, metrics) { - public Plugin createRemoteStorageManagerPlugin() { - return Plugin.wrapInstanceDelayedInit(remoteStorageManager); + brokerTopicStats, metrics, endPoint) { + @Override + public RemoteStorageManager createRemoteStorageManager() { + return remoteStorageManager; } - public Plugin createRemoteLogMetadataManagerPlugin() { - return Plugin.wrapInstanceDelayedInit(remoteLogMetadataManager); + @Override + public RemoteLogMetadataManager createRemoteLogMetadataManager() { + return remoteLogMetadataManager; } + @Override int lookupPositionForOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) { return 0; } }; - remoteLogManager.startup(); remoteLogManager.onLeadershipChange( Collections.singleton(mockPartition(leaderTopicIdPartition)), Collections.emptySet(), topicIds); @@ -3698,16 +3685,20 @@ public void testRLMOpsWhenMetadataIsNotReady() throws InterruptedException, IOEx remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), brokerId, logDir, clusterId, time, tp -> Optional.of(mockLog), (topicPartition, offset) -> currentLogStartOffset.set(offset), - brokerTopicStats, metrics) { - public Plugin createRemoteStorageManagerPlugin() { - return Plugin.wrapInstanceDelayedInit(remoteStorageManager); + brokerTopicStats, metrics, endPoint) { + @Override + public RemoteStorageManager createRemoteStorageManager() { + return remoteStorageManager; } - public Plugin createRemoteLogMetadataManagerPlugin() { - return Plugin.wrapInstanceDelayedInit(remoteLogMetadataManager); + @Override + public RemoteLogMetadataManager createRemoteLogMetadataManager() { + return remoteLogMetadataManager; } + @Override public RLMQuotaManager createRLMCopyQuotaManager() { return rlmCopyQuotaManager; } + @Override public Duration quotaTimeout() { return Duration.ofMillis(100); } @@ -3723,7 +3714,6 @@ long findLogStartOffset(TopicIdPartition topicIdPartition, UnifiedLog log) { latch.countDown(); return false; }); - remoteLogManager.startup(); remoteLogManager.onLeadershipChange( Collections.singleton(mockPartition(leaderTopicIdPartition)), Collections.singleton(mockPartition(followerTopicIdPartition)), @@ -3762,16 +3752,12 @@ public void testMonitorableRemoteLogStorageManager() throws IOException { tp -> Optional.of(mockLog), (topicPartition, offset) -> { }, brokerTopicStats, - metrics)) { + metrics, + endPoint)) { assertInstanceOf(MonitorableNoOpRemoteStorageManager.class, remoteLogManager.storageManager()); assertInstanceOf(MonitorableNoOpRemoteLogMetadataManager.class, remoteLogManager.remoteLogMetadataManager()); - MonitorableNoOpRemoteStorageManager rsm = (MonitorableNoOpRemoteStorageManager) remoteLogManager.storageManager(); - MonitorableNoOpRemoteLogMetadataManager rlm = (MonitorableNoOpRemoteLogMetadataManager) remoteLogManager.remoteLogMetadataManager(); - assertEquals(false, rsm.pluginMetrics); - assertEquals(false, rlm.pluginMetrics); // We need to call startup for call config() and call withPluginMetrics() - remoteLogManager.startup(); assertEquals(true, ((MonitorableNoOpRemoteStorageManager) remoteLogManager.storageManager()).pluginMetrics); assertEquals(true, ((MonitorableNoOpRemoteLogMetadataManager) remoteLogManager.remoteLogMetadataManager()).pluginMetrics); } diff --git a/core/src/test/java/kafka/log/remote/RemoteLogOffsetReaderTest.java b/core/src/test/java/kafka/log/remote/RemoteLogOffsetReaderTest.java index 5f063c3f1ac4f..d5fb4e043e1d8 100644 --- a/core/src/test/java/kafka/log/remote/RemoteLogOffsetReaderTest.java +++ b/core/src/test/java/kafka/log/remote/RemoteLogOffsetReaderTest.java @@ -155,7 +155,8 @@ public MockRemoteLogManager(int threads, tp -> Optional.empty(), (tp, logStartOffset) -> { }, new BrokerTopicStats(true), - new Metrics() + new Metrics(), + Optional.empty() ); } diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index fcf0269420ea1..cd1b61199a625 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -2237,7 +2237,8 @@ class UnifiedLogTest { _ => Optional.empty[UnifiedLog](), (_, _) => {}, brokerTopicStats, - new Metrics())) + new Metrics(), + Optional.empty)) remoteLogManager.setDelayedOperationPurgatory(purgatory) val logConfig = LogTestUtils.createLogConfig(segmentBytes = 200, indexIntervalBytes = 1, @@ -2334,7 +2335,8 @@ class UnifiedLogTest { _ => Optional.empty[UnifiedLog](), (_, _) => {}, brokerTopicStats, - new Metrics())) + new Metrics(), + Optional.empty)) remoteLogManager.setDelayedOperationPurgatory(purgatory) val logConfig = LogTestUtils.createLogConfig(segmentBytes = 200, indexIntervalBytes = 1, diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 5f1fb7b5f24a5..81183e3f4d267 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -3790,8 +3790,8 @@ class ReplicaManagerTest { _ => Optional.of(mockLog), (TopicPartition, Long) => {}, brokerTopicStats, - metrics) - remoteLogManager.startup() + metrics, + Optional.empty) val spyRLM = spy(remoteLogManager) val replicaManager = setupReplicaManagerWithMockedPurgatories(new MockTimer(time), aliveBrokerIds = Seq(0, 1, 2), enableRemoteStorage = true, shouldMockLog = true, remoteLogManager = Some(spyRLM)) @@ -3901,8 +3901,8 @@ class ReplicaManagerTest { _ => Optional.of(dummyLog), (TopicPartition, Long) => {}, brokerTopicStats, - metrics) - remoteLogManager.startup() + metrics, + Optional.empty) val spyRLM = spy(remoteLogManager) val timer = new MockTimer(time) From a9b90d2ed2be959d03af3a0155000901cbd21c97 Mon Sep 17 00:00:00 2001 From: TaiJu Wu Date: Tue, 1 Apr 2025 10:23:11 +0000 Subject: [PATCH 14/24] rename and small refactor --- .../kafka/log/remote/RemoteLogManager.java | 23 ++++++------------- 1 file changed, 7 insertions(+), 16 deletions(-) diff --git a/core/src/main/java/kafka/log/remote/RemoteLogManager.java b/core/src/main/java/kafka/log/remote/RemoteLogManager.java index d0b269cafcbdb..1b40b802a1236 100644 --- a/core/src/main/java/kafka/log/remote/RemoteLogManager.java +++ b/core/src/main/java/kafka/log/remote/RemoteLogManager.java @@ -241,8 +241,8 @@ public RemoteLogManager(RemoteLogManagerConfig rlmConfig, this.metrics = metrics; this.endpoint = endpoint; - remoteLogStorageManagerPlugin = configAndWrapRSM(createRemoteStorageManager()); - remoteLogMetadataManagerPlugin = configAndWrapRLMM(createRemoteLogMetadataManager()); + remoteLogStorageManagerPlugin = configAndWrapRSMPlugin(createRemoteStorageManager()); + remoteLogMetadataManagerPlugin = configAndWrapRLMMPlugin(createRemoteLogMetadataManager()); remoteLogManagerConfigured = true; rlmCopyQuotaManager = createRLMCopyQuotaManager(); @@ -376,14 +376,10 @@ RemoteStorageManager createRemoteStorageManager() { }); } - private void configureRSM(RemoteStorageManager remoteLogStorageManager) { + private Plugin configAndWrapRSMPlugin(RemoteStorageManager rsm) { final Map rsmProps = new HashMap<>(rlmConfig.remoteStorageManagerProps()); rsmProps.put(ServerConfigs.BROKER_ID_CONFIG, brokerId); - remoteLogStorageManager.configure(rsmProps); - } - - private Plugin configAndWrapRSM(RemoteStorageManager rsm) { - configureRSM(rsm); + rsm.configure(rsmProps); return Plugin.wrapInstance(rsm, metrics, RemoteLogManagerConfig.REMOTE_STORAGE_MANAGER_CLASS_NAME_PROP); } @@ -400,12 +396,7 @@ RemoteLogMetadataManager createRemoteLogMetadataManager() { }); } - private Plugin configAndWrapRLMM(RemoteLogMetadataManager rlmm) { - configureRLMM(rlmm); - return Plugin.wrapInstance(rlmm, metrics, RemoteLogManagerConfig.REMOTE_LOG_METADATA_MANAGER_CLASS_NAME_PROP); - } - - private void configureRLMM(RemoteLogMetadataManager remoteLogMetadataManager) { + private Plugin configAndWrapRLMMPlugin(RemoteLogMetadataManager rlmm) { final Map rlmmProps = new HashMap<>(); endpoint.ifPresent(e -> { rlmmProps.put(REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX + "bootstrap.servers", e.host() + ":" + e.port()); @@ -418,10 +409,10 @@ private void configureRLMM(RemoteLogMetadataManager remoteLogMetadataManager) { rlmmProps.put(LOG_DIR_CONFIG, logDir); rlmmProps.put("cluster.id", clusterId); - remoteLogMetadataManager.configure(rlmmProps); + rlmm.configure(rlmmProps); + return Plugin.wrapInstance(rlmm, metrics, RemoteLogManagerConfig.REMOTE_LOG_METADATA_MANAGER_CLASS_NAME_PROP); } - RemoteLogMetadataManager remoteLogMetadataManager() { return remoteLogMetadataManagerPlugin.get(); } From 3d5a1492c3648afe094c4034554046dd0c6b2cb3 Mon Sep 17 00:00:00 2001 From: TaiJu Wu Date: Thu, 3 Apr 2025 11:00:33 +0000 Subject: [PATCH 15/24] add doc --- .../server/log/remote/storage/RemoteLogMetadataManager.java | 4 ++++ .../kafka/server/log/remote/storage/RemoteStorageManager.java | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataManager.java b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataManager.java index 2cc581dbfec40..189e0a1713e31 100644 --- a/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataManager.java +++ b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataManager.java @@ -47,6 +47,10 @@ * "cluster.id", "broker.id" and all other properties prefixed with the config: "remote.log.metadata.manager.impl.prefix" * (default value is "rlmm.config.") are passed when {@link #configure(Map)} is invoked on this instance. *

+ * + * Implement {@link org.apache.kafka.common.metrics.Monitorable} to enable the manager to register metrics. + * The following tags are automatically added to all metrics registered: config set to + * remote.log.metadata.manager.class.name, and class set to the RemoteLogMetadataManager class name. */ public interface RemoteLogMetadataManager extends Configurable, Closeable { diff --git a/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteStorageManager.java b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteStorageManager.java index 03eeebcfa1d5d..3fd6a633b7d07 100644 --- a/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteStorageManager.java +++ b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteStorageManager.java @@ -39,6 +39,10 @@ *

* All properties prefixed with the config: "remote.log.storage.manager.impl.prefix" * (default value is "rsm.config.") are passed when {@link #configure(Map)} is invoked on this instance. + * + * Implement {@link org.apache.kafka.common.metrics.Monitorable} to enable the manager to register metrics. + * The following tags are automatically added to all metrics registered: config set to + * remote.log.storage.manager.class.name, and class set to the RemoteStorageManager class name. */ public interface RemoteStorageManager extends Configurable, Closeable { From a00f3f9e5724f53487f95a433d93fe97d1929bb1 Mon Sep 17 00:00:00 2001 From: TaiJu Wu Date: Thu, 3 Apr 2025 22:00:51 +0000 Subject: [PATCH 16/24] address Yunyung comments --- core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java index 9b78910197a31..2476c221e9e11 100644 --- a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java +++ b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java @@ -3757,7 +3757,6 @@ public void testMonitorableRemoteLogStorageManager() throws IOException { assertInstanceOf(MonitorableNoOpRemoteStorageManager.class, remoteLogManager.storageManager()); assertInstanceOf(MonitorableNoOpRemoteLogMetadataManager.class, remoteLogManager.remoteLogMetadataManager()); - // We need to call startup for call config() and call withPluginMetrics() assertEquals(true, ((MonitorableNoOpRemoteStorageManager) remoteLogManager.storageManager()).pluginMetrics); assertEquals(true, ((MonitorableNoOpRemoteLogMetadataManager) remoteLogManager.remoteLogMetadataManager()).pluginMetrics); } @@ -3818,6 +3817,7 @@ public void withPluginMetrics(PluginMetrics metrics) { public static class MonitorableNoOpRemoteLogMetadataManager extends NoOpRemoteLogMetadataManager implements Monitorable { public boolean pluginMetrics = false; + @Override public void withPluginMetrics(PluginMetrics metrics) { pluginMetrics = true; From 13e8b73e79290eeab8a625ad42427a9074446ad2 Mon Sep 17 00:00:00 2001 From: TaiJu Wu Date: Thu, 3 Apr 2025 22:13:35 +0000 Subject: [PATCH 17/24] address Miason comment --- .../kafka/log/remote/RemoteLogManagerTest.java | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java index 2476c221e9e11..19aca32d31022 100644 --- a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java +++ b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java @@ -430,6 +430,7 @@ public RemoteLogMetadataManager createRemoteLogMetadataManager() { } }) { ArgumentCaptor> capture = ArgumentCaptor.forClass(Map.class); + // One is called from setup, the other is from this function. verify(remoteLogMetadataManager, times(2)).configure(capture.capture()); assertEquals(host + ":" + port, capture.getValue().get(REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX + "bootstrap.servers")); // should be overridden as SSL @@ -440,7 +441,7 @@ public RemoteLogMetadataManager createRemoteLogMetadataManager() { } @Test - void testStartup() { + void testConfigure() { ArgumentCaptor> capture = ArgumentCaptor.forClass(Map.class); verify(remoteStorageManager, times(1)).configure(capture.capture()); assertEquals(brokerId, capture.getValue().get("broker.id")); @@ -2845,7 +2846,6 @@ public void testDeleteLogSegmentDueToRetentionSizeBreach(int segmentCount, ); checkpoint.write(epochEntries); LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, scheduler); - int currentLeaderEpoch = epochEntries.get(epochEntries.size() - 1).epoch; long localLogSegmentsSize = 512L; long retentionSize = ((long) segmentCount - deletableSegmentCount) * segmentSize + localLogSegmentsSize; @@ -2865,7 +2865,7 @@ public void testDeleteLogSegmentDueToRetentionSizeBreach(int segmentCount, List segmentMetadataList = listRemoteLogSegmentMetadata( leaderTopicIdPartition, segmentCount, recordsPerSegment, segmentSize, epochEntries, RemoteLogSegmentState.COPY_SEGMENT_FINISHED); - verifyDeleteLogSegment(segmentMetadataList, deletableSegmentCount, currentLeaderEpoch); + verifyDeleteLogSegment(segmentMetadataList, deletableSegmentCount); } @ParameterizedTest(name = "testDeleteLogSegmentDueToRetentionTimeBreach segmentCount={0} deletableSegmentCount={1}") @@ -2883,7 +2883,6 @@ public void testDeleteLogSegmentDueToRetentionTimeBreach(int segmentCount, ); checkpoint.write(epochEntries); LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, scheduler); - int currentLeaderEpoch = epochEntries.get(epochEntries.size() - 1).epoch; long localLogSegmentsSize = 512L; long retentionSize = -1L; @@ -2903,7 +2902,7 @@ public void testDeleteLogSegmentDueToRetentionTimeBreach(int segmentCount, List segmentMetadataList = listRemoteLogSegmentMetadataByTime( leaderTopicIdPartition, segmentCount, deletableSegmentCount, recordsPerSegment, segmentSize, epochEntries, RemoteLogSegmentState.COPY_SEGMENT_FINISHED); - verifyDeleteLogSegment(segmentMetadataList, deletableSegmentCount, currentLeaderEpoch); + verifyDeleteLogSegment(segmentMetadataList, deletableSegmentCount); } private void verifyRemoteDeleteMetrics(long remoteDeleteLagBytes, long remoteDeleteLagSegments) { @@ -2922,8 +2921,7 @@ private void verifyRemoteDeleteMetrics(long remoteDeleteLagBytes, long remoteDel } private void verifyDeleteLogSegment(List segmentMetadataList, - int deletableSegmentCount, - int currentLeaderEpoch) + int deletableSegmentCount) throws RemoteStorageException, ExecutionException, InterruptedException { when(remoteLogMetadataManager.listRemoteLogSegments(leaderTopicIdPartition)) .thenReturn(segmentMetadataList.iterator()); @@ -2954,13 +2952,16 @@ private void verifyDeleteLogSegment(List segmentMetada @Test public void testDeleteRetentionMsOnExpiredSegment() throws RemoteStorageException, IOException { AtomicLong logStartOffset = new AtomicLong(0); + try (RemoteLogManager remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), brokerId, logDir, clusterId, time, tp -> Optional.of(mockLog), (topicPartition, offset) -> logStartOffset.set(offset), brokerTopicStats, metrics, endPoint) { + @Override public RemoteStorageManager createRemoteStorageManager() { return remoteStorageManager; } + @Override public RemoteLogMetadataManager createRemoteLogMetadataManager() { return remoteLogMetadataManager; } @@ -2995,7 +2996,8 @@ public RemoteLogMetadataManager createRemoteLogMetadataManager() { .thenAnswer(answer -> CompletableFuture.runAsync(() -> { })); task.cleanupExpiredRemoteLogSegments(); - + // One is called from setup, the other is from this function. + verify(remoteStorageManager, times(2)).configure(any()); assertEquals(0L, logStartOffset.get()); } catch (ExecutionException | InterruptedException e) { throw new RuntimeException(e); From aa94317568d7e90253b36d23d9b05f3019791f15 Mon Sep 17 00:00:00 2001 From: TaiJu Wu Date: Thu, 10 Apr 2025 13:41:23 +0800 Subject: [PATCH 18/24] copy file from Miason PR --- .../MonitorablePluginsIntegrationTest.java | 111 ++++++++++++++++++ 1 file changed, 111 insertions(+) create mode 100644 server/src/test/java/org/apache/kafka/server/MonitorablePluginsIntegrationTest.java diff --git a/server/src/test/java/org/apache/kafka/server/MonitorablePluginsIntegrationTest.java b/server/src/test/java/org/apache/kafka/server/MonitorablePluginsIntegrationTest.java new file mode 100644 index 0000000000000..4709235f62d49 --- /dev/null +++ b/server/src/test/java/org/apache/kafka/server/MonitorablePluginsIntegrationTest.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.server; + +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.metrics.Measurable; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.Monitorable; +import org.apache.kafka.common.metrics.PluginMetrics; +import org.apache.kafka.common.replica.RackAwareReplicaSelector; +import org.apache.kafka.common.test.ClusterInstance; +import org.apache.kafka.common.test.api.ClusterConfigProperty; +import org.apache.kafka.common.test.api.ClusterTest; +import org.apache.kafka.common.test.api.Type; +import org.apache.kafka.metadata.authorizer.StandardAuthorizer; + +import java.util.LinkedHashMap; +import java.util.Map; + +import static org.apache.kafka.server.config.ReplicationConfigs.REPLICA_SELECTOR_CLASS_CONFIG; +import static org.apache.kafka.server.config.ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG; +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class MonitorablePluginsIntegrationTest { + + private static int controllerId(Type type) { + return type == Type.KRAFT ? 3000 : 0; + } + + private static Map expectedTags(String config, String clazz) { + return expectedTags(config, clazz, Map.of()); + } + + private static Map expectedTags(String config, String clazz, Map extraTags) { + Map tags = new LinkedHashMap<>(); + tags.put("config", config); + tags.put("class", clazz); + tags.putAll(extraTags); + return tags; + } + + @ClusterTest( + types = {Type.KRAFT, Type.CO_KRAFT}, + serverProperties = { + @ClusterConfigProperty(key = StandardAuthorizer.SUPER_USERS_CONFIG, value = "User:ANONYMOUS"), + @ClusterConfigProperty(key = AUTHORIZER_CLASS_NAME_CONFIG, value = "org.apache.kafka.metadata.authorizer.StandardAuthorizer"), + @ClusterConfigProperty(key = REPLICA_SELECTOR_CLASS_CONFIG, value = "org.apache.kafka.server.MonitorablePluginsIntegrationTest$MonitorableReplicaSelector") + } + ) + public void testMonitorableServerPlugins(ClusterInstance clusterInstance) { + assertAuthorizerMetrics(clusterInstance); + assertReplicaSelectorMetrics(clusterInstance); + } + + private void assertAuthorizerMetrics(ClusterInstance clusterInstance) { + assertMetrics( + clusterInstance.brokers().get(0).metrics(), + 4, + expectedTags(AUTHORIZER_CLASS_NAME_CONFIG, "StandardAuthorizer", Map.of("role", "broker"))); + + assertMetrics( + clusterInstance.controllers().get(controllerId(clusterInstance.type())).metrics(), + 4, + expectedTags(AUTHORIZER_CLASS_NAME_CONFIG, "StandardAuthorizer", Map.of("role", "controller"))); + } + + private void assertReplicaSelectorMetrics(ClusterInstance clusterInstance) { + assertMetrics( + clusterInstance.brokers().get(0).metrics(), + MonitorableReplicaSelector.METRICS_COUNT, + expectedTags(REPLICA_SELECTOR_CLASS_CONFIG, MonitorableReplicaSelector.class.getSimpleName())); + } + + private void assertMetrics(Metrics metrics, int expected, Map expectedTags) { + int found = 0; + for (MetricName metricName : metrics.metrics().keySet()) { + if (metricName.group().equals("plugins")) { + Map tags = metricName.tags(); + if (expectedTags.equals(tags)) { + found++; + } + } + } + assertEquals(expected, found); + } + + public static class MonitorableReplicaSelector extends RackAwareReplicaSelector implements Monitorable { + + private static final int METRICS_COUNT = 1; + + @Override + public void withPluginMetrics(PluginMetrics metrics) { + MetricName name = metrics.metricName("name", "description", Map.of()); + metrics.addMetric(name, (Measurable) (config, now) -> 123); + } + } +} \ No newline at end of file From f28689eebeba9b4807d44f8eed01a845507a5c38 Mon Sep 17 00:00:00 2001 From: TaiJu Wu Date: Thu, 10 Apr 2025 14:21:54 +0800 Subject: [PATCH 19/24] add it --- build.gradle | 2 + .../MonitorablePluginsIntegrationTest.java | 51 ++++++++++--------- 2 files changed, 30 insertions(+), 23 deletions(-) diff --git a/build.gradle b/build.gradle index 4753be6ccc6af..d7cec2e4aa224 100644 --- a/build.gradle +++ b/build.gradle @@ -902,6 +902,7 @@ project(':server') { implementation project(':transaction-coordinator') implementation project(':raft') implementation project(':share-coordinator') + implementation project(':storage:storage-api') implementation libs.jacksonDatabind implementation libs.metrics implementation libs.slf4jApi @@ -913,6 +914,7 @@ project(':server') { testImplementation testLog4j2Libs testImplementation project(':test-common:test-common-internal-api') testImplementation project(':test-common:test-common-runtime') + testImplementation project(':storage:storage-api').sourceSets.test.output testRuntimeOnly runtimeTestLibs } diff --git a/server/src/test/java/org/apache/kafka/server/MonitorablePluginsIntegrationTest.java b/server/src/test/java/org/apache/kafka/server/MonitorablePluginsIntegrationTest.java index 4709235f62d49..db51837c1e280 100644 --- a/server/src/test/java/org/apache/kafka/server/MonitorablePluginsIntegrationTest.java +++ b/server/src/test/java/org/apache/kafka/server/MonitorablePluginsIntegrationTest.java @@ -21,18 +21,19 @@ import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Monitorable; import org.apache.kafka.common.metrics.PluginMetrics; -import org.apache.kafka.common.replica.RackAwareReplicaSelector; import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfigProperty; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.Type; -import org.apache.kafka.metadata.authorizer.StandardAuthorizer; +import org.apache.kafka.server.log.remote.storage.NoOpRemoteLogMetadataManager; +import org.apache.kafka.server.log.remote.storage.NoOpRemoteStorageManager; import java.util.LinkedHashMap; import java.util.Map; -import static org.apache.kafka.server.config.ReplicationConfigs.REPLICA_SELECTOR_CLASS_CONFIG; -import static org.apache.kafka.server.config.ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG; +import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.REMOTE_LOG_METADATA_MANAGER_CLASS_NAME_PROP; +import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP; +import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.REMOTE_STORAGE_MANAGER_CLASS_NAME_PROP; import static org.junit.jupiter.api.Assertions.assertEquals; public class MonitorablePluginsIntegrationTest { @@ -56,33 +57,26 @@ private static Map expectedTags(String config, String clazz, Map @ClusterTest( types = {Type.KRAFT, Type.CO_KRAFT}, serverProperties = { - @ClusterConfigProperty(key = StandardAuthorizer.SUPER_USERS_CONFIG, value = "User:ANONYMOUS"), - @ClusterConfigProperty(key = AUTHORIZER_CLASS_NAME_CONFIG, value = "org.apache.kafka.metadata.authorizer.StandardAuthorizer"), - @ClusterConfigProperty(key = REPLICA_SELECTOR_CLASS_CONFIG, value = "org.apache.kafka.server.MonitorablePluginsIntegrationTest$MonitorableReplicaSelector") + @ClusterConfigProperty(key = REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, value = "true"), + @ClusterConfigProperty(key = REMOTE_LOG_METADATA_MANAGER_CLASS_NAME_PROP, + value = "org.apache.kafka.server.MonitorablePluginsIntegrationTest$MonitorableNoOpRemoteLogMetadataManager"), + @ClusterConfigProperty(key = REMOTE_STORAGE_MANAGER_CLASS_NAME_PROP, + value = "org.apache.kafka.server.MonitorablePluginsIntegrationTest$MonitorableNoOpRemoteStorageManager") } ) public void testMonitorableServerPlugins(ClusterInstance clusterInstance) { - assertAuthorizerMetrics(clusterInstance); - assertReplicaSelectorMetrics(clusterInstance); + assertRemoteLogManagerMetrics(clusterInstance); } - private void assertAuthorizerMetrics(ClusterInstance clusterInstance) { + private void assertRemoteLogManagerMetrics(ClusterInstance clusterInstance) { assertMetrics( clusterInstance.brokers().get(0).metrics(), - 4, - expectedTags(AUTHORIZER_CLASS_NAME_CONFIG, "StandardAuthorizer", Map.of("role", "broker"))); - - assertMetrics( - clusterInstance.controllers().get(controllerId(clusterInstance.type())).metrics(), - 4, - expectedTags(AUTHORIZER_CLASS_NAME_CONFIG, "StandardAuthorizer", Map.of("role", "controller"))); - } - - private void assertReplicaSelectorMetrics(ClusterInstance clusterInstance) { + MonitorableNoOpRemoteLogMetadataManager.METRICS_COUNT, + expectedTags(REMOTE_LOG_METADATA_MANAGER_CLASS_NAME_PROP, MonitorableNoOpRemoteLogMetadataManager.class.getSimpleName())); assertMetrics( clusterInstance.brokers().get(0).metrics(), - MonitorableReplicaSelector.METRICS_COUNT, - expectedTags(REPLICA_SELECTOR_CLASS_CONFIG, MonitorableReplicaSelector.class.getSimpleName())); + MonitorableNoOpRemoteStorageManager.METRICS_COUNT, + expectedTags(REMOTE_STORAGE_MANAGER_CLASS_NAME_PROP, MonitorableNoOpRemoteStorageManager.class.getSimpleName())); } private void assertMetrics(Metrics metrics, int expected, Map expectedTags) { @@ -98,7 +92,18 @@ private void assertMetrics(Metrics metrics, int expected, Map ex assertEquals(expected, found); } - public static class MonitorableReplicaSelector extends RackAwareReplicaSelector implements Monitorable { + public static class MonitorableNoOpRemoteLogMetadataManager extends NoOpRemoteLogMetadataManager implements Monitorable { + + private static final int METRICS_COUNT = 1; + + @Override + public void withPluginMetrics(PluginMetrics metrics) { + MetricName name = metrics.metricName("name", "description", Map.of()); + metrics.addMetric(name, (Measurable) (config, now) -> 123); + } + } + + public static class MonitorableNoOpRemoteStorageManager extends NoOpRemoteStorageManager implements Monitorable { private static final int METRICS_COUNT = 1; From e77bee11fa0ec1380fa5f034a8cd20319033d9c8 Mon Sep 17 00:00:00 2001 From: TaiJu Wu Date: Fri, 11 Apr 2025 06:59:09 +0800 Subject: [PATCH 20/24] address Yunyung comments --- .../scala/kafka/server/BrokerServer.scala | 2 -- .../MonitorablePluginsIntegrationTest.java | 31 +++++++++---------- 2 files changed, 14 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 987ca6fa9f29c..e25f7619d2086 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -473,8 +473,6 @@ class BrokerServer( config.numIoThreads, s"${DataPlaneAcceptor.MetricPrefix}RequestHandlerAvgIdlePercent", DataPlaneAcceptor.ThreadPrefix) - - metadataPublishers.add(new MetadataVersionConfigValidator(config, sharedServer.metadataPublishingFaultHandler)) brokerMetadataPublisher = new BrokerMetadataPublisher(config, metadataCache, diff --git a/server/src/test/java/org/apache/kafka/server/MonitorablePluginsIntegrationTest.java b/server/src/test/java/org/apache/kafka/server/MonitorablePluginsIntegrationTest.java index db51837c1e280..0823a8b77026d 100644 --- a/server/src/test/java/org/apache/kafka/server/MonitorablePluginsIntegrationTest.java +++ b/server/src/test/java/org/apache/kafka/server/MonitorablePluginsIntegrationTest.java @@ -38,24 +38,8 @@ public class MonitorablePluginsIntegrationTest { - private static int controllerId(Type type) { - return type == Type.KRAFT ? 3000 : 0; - } - - private static Map expectedTags(String config, String clazz) { - return expectedTags(config, clazz, Map.of()); - } - - private static Map expectedTags(String config, String clazz, Map extraTags) { - Map tags = new LinkedHashMap<>(); - tags.put("config", config); - tags.put("class", clazz); - tags.putAll(extraTags); - return tags; - } - @ClusterTest( - types = {Type.KRAFT, Type.CO_KRAFT}, + types = {Type.KRAFT}, serverProperties = { @ClusterConfigProperty(key = REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, value = "true"), @ClusterConfigProperty(key = REMOTE_LOG_METADATA_MANAGER_CLASS_NAME_PROP, @@ -113,4 +97,17 @@ public void withPluginMetrics(PluginMetrics metrics) { metrics.addMetric(name, (Measurable) (config, now) -> 123); } } + + private static Map expectedTags(String config, String clazz) { + return expectedTags(config, clazz, Map.of()); + } + + private static Map expectedTags(String config, String clazz, Map extraTags) { + Map tags = new LinkedHashMap<>(); + tags.put("config", config); + tags.put("class", clazz); + tags.putAll(extraTags); + return tags; + } + } \ No newline at end of file From 5c0b22148f7fc9c26689c947a28e34bba3df9fc8 Mon Sep 17 00:00:00 2001 From: TaiJu Wu Date: Tue, 15 Apr 2025 13:39:14 +0800 Subject: [PATCH 21/24] address maison comments --- .../main/java/kafka/log/remote/RemoteLogManager.java | 12 +++++------- .../java/kafka/log/remote/RemoteLogManagerTest.java | 4 ++-- .../server/MonitorablePluginsIntegrationTest.java | 7 +------ 3 files changed, 8 insertions(+), 15 deletions(-) diff --git a/core/src/main/java/kafka/log/remote/RemoteLogManager.java b/core/src/main/java/kafka/log/remote/RemoteLogManager.java index 8c673db7eae71..6bf8c4a1bf226 100644 --- a/core/src/main/java/kafka/log/remote/RemoteLogManager.java +++ b/core/src/main/java/kafka/log/remote/RemoteLogManager.java @@ -202,7 +202,6 @@ public class RemoteLogManager implements Closeable, AsyncOffsetReader { private Optional endpoint = Optional.empty(); private boolean closed = false; - private volatile boolean remoteLogManagerConfigured = false; private final Timer remoteReadTimer; private volatile DelayedOperationPurgatory delayedRemoteListOffsetsPurgatory; @@ -241,9 +240,8 @@ public RemoteLogManager(RemoteLogManagerConfig rlmConfig, this.metrics = metrics; this.endpoint = endpoint; - remoteStorageManagerPlugin = configAndWrapRSMPlugin(createRemoteStorageManager()); - remoteLogMetadataManagerPlugin = configAndWrapRLMMPlugin(createRemoteLogMetadataManager()); - remoteLogManagerConfigured = true; + remoteStorageManagerPlugin = configAndWrapRsmPlugin(createRemoteStorageManager()); + remoteLogMetadataManagerPlugin = configAndWrapRlmmPlugin(createRemoteLogMetadataManager()); rlmCopyQuotaManager = createRLMCopyQuotaManager(); rlmFetchQuotaManager = createRLMFetchQuotaManager(); @@ -376,7 +374,7 @@ RemoteStorageManager createRemoteStorageManager() { }); } - private Plugin configAndWrapRSMPlugin(RemoteStorageManager rsm) { + private Plugin configAndWrapRsmPlugin(RemoteStorageManager rsm) { final Map rsmProps = new HashMap<>(rlmConfig.remoteStorageManagerProps()); rsmProps.put(ServerConfigs.BROKER_ID_CONFIG, brokerId); rsm.configure(rsmProps); @@ -396,7 +394,7 @@ RemoteLogMetadataManager createRemoteLogMetadataManager() { }); } - private Plugin configAndWrapRLMMPlugin(RemoteLogMetadataManager rlmm) { + private Plugin configAndWrapRlmmPlugin(RemoteLogMetadataManager rlmm) { final Map rlmmProps = new HashMap<>(); endpoint.ifPresent(e -> { rlmmProps.put(REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX + "bootstrap.servers", e.host() + ":" + e.port()); @@ -448,7 +446,7 @@ public void onLeadershipChange(Set partitionsBecomeLeader, Map topicIds) { LOGGER.debug("Received leadership changes for leaders: {} and followers: {}", partitionsBecomeLeader, partitionsBecomeFollower); - if (rlmConfig.isRemoteStorageSystemEnabled() && !this.remoteLogManagerConfigured) { + if (rlmConfig.isRemoteStorageSystemEnabled()) { throw new KafkaException("RemoteLogManager is not configured when remote storage system is enabled"); } diff --git a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java index 22087d82c118f..6c6182d340cde 100644 --- a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java +++ b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java @@ -3759,8 +3759,8 @@ public void testMonitorableRemoteLogStorageManager() throws IOException { assertInstanceOf(MonitorableNoOpRemoteStorageManager.class, remoteLogManager.storageManager()); assertInstanceOf(MonitorableNoOpRemoteLogMetadataManager.class, remoteLogManager.remoteLogMetadataManager()); - assertEquals(true, ((MonitorableNoOpRemoteStorageManager) remoteLogManager.storageManager()).pluginMetrics); - assertEquals(true, ((MonitorableNoOpRemoteLogMetadataManager) remoteLogManager.remoteLogMetadataManager()).pluginMetrics); + assertTrue(((MonitorableNoOpRemoteStorageManager) remoteLogManager.storageManager()).pluginMetrics); + assertTrue(((MonitorableNoOpRemoteLogMetadataManager) remoteLogManager.remoteLogMetadataManager()).pluginMetrics); } } diff --git a/server/src/test/java/org/apache/kafka/server/MonitorablePluginsIntegrationTest.java b/server/src/test/java/org/apache/kafka/server/MonitorablePluginsIntegrationTest.java index 0823a8b77026d..5ccbd2dde916d 100644 --- a/server/src/test/java/org/apache/kafka/server/MonitorablePluginsIntegrationTest.java +++ b/server/src/test/java/org/apache/kafka/server/MonitorablePluginsIntegrationTest.java @@ -99,15 +99,10 @@ public void withPluginMetrics(PluginMetrics metrics) { } private static Map expectedTags(String config, String clazz) { - return expectedTags(config, clazz, Map.of()); - } - - private static Map expectedTags(String config, String clazz, Map extraTags) { Map tags = new LinkedHashMap<>(); tags.put("config", config); tags.put("class", clazz); - tags.putAll(extraTags); return tags; } -} \ No newline at end of file +} From f18e1b211acee95dbcf4347d8ea3f6e5221a0a21 Mon Sep 17 00:00:00 2001 From: TaiJu Wu Date: Wed, 16 Apr 2025 22:00:47 +0800 Subject: [PATCH 22/24] fix build --- .../kafka/server/MonitorablePluginsIntegrationTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/server/src/test/java/org/apache/kafka/server/MonitorablePluginsIntegrationTest.java b/server/src/test/java/org/apache/kafka/server/MonitorablePluginsIntegrationTest.java index 391021e8aaad7..43b56e63298f6 100644 --- a/server/src/test/java/org/apache/kafka/server/MonitorablePluginsIntegrationTest.java +++ b/server/src/test/java/org/apache/kafka/server/MonitorablePluginsIntegrationTest.java @@ -26,18 +26,18 @@ import org.apache.kafka.common.test.api.ClusterConfigProperty; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.Type; +import org.apache.kafka.metadata.authorizer.StandardAuthorizer; import org.apache.kafka.server.log.remote.storage.NoOpRemoteLogMetadataManager; import org.apache.kafka.server.log.remote.storage.NoOpRemoteStorageManager; -import org.apache.kafka.metadata.authorizer.StandardAuthorizer; import java.util.LinkedHashMap; import java.util.Map; +import static org.apache.kafka.server.config.ReplicationConfigs.REPLICA_SELECTOR_CLASS_CONFIG; +import static org.apache.kafka.server.config.ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG; import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.REMOTE_LOG_METADATA_MANAGER_CLASS_NAME_PROP; import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP; import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.REMOTE_STORAGE_MANAGER_CLASS_NAME_PROP; -import static org.apache.kafka.server.config.ReplicationConfigs.REPLICA_SELECTOR_CLASS_CONFIG; -import static org.apache.kafka.server.config.ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG; import static org.junit.jupiter.api.Assertions.assertEquals; public class MonitorablePluginsIntegrationTest { From e3eed4bfcd86bd0f8f7d82b249814dc28117a723 Mon Sep 17 00:00:00 2001 From: TaiJu Wu Date: Thu, 17 Apr 2025 12:12:09 +0800 Subject: [PATCH 23/24] fix test --- core/src/main/java/kafka/log/remote/RemoteLogManager.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/core/src/main/java/kafka/log/remote/RemoteLogManager.java b/core/src/main/java/kafka/log/remote/RemoteLogManager.java index 947175d6614eb..015b456f9de64 100644 --- a/core/src/main/java/kafka/log/remote/RemoteLogManager.java +++ b/core/src/main/java/kafka/log/remote/RemoteLogManager.java @@ -445,10 +445,6 @@ public void onLeadershipChange(Set partitionsBecomeLeader, Map topicIds) { LOGGER.debug("Received leadership changes for leaders: {} and followers: {}", partitionsBecomeLeader, partitionsBecomeFollower); - if (rlmConfig.isRemoteStorageSystemEnabled()) { - throw new KafkaException("RemoteLogManager is not configured when remote storage system is enabled"); - } - Map leaderPartitions = filterPartitions(partitionsBecomeLeader) .collect(Collectors.toMap(p -> new TopicIdPartition(topicIds.get(p.topicPartition().topic()), p.topicPartition()), p -> p.unifiedLog().isPresent() ? p.unifiedLog().get().config().remoteLogCopyDisable() : false)); From 45b6c5dcc09baef65164e8883413bbd98f640a6b Mon Sep 17 00:00:00 2001 From: TaiJu Wu Date: Thu, 17 Apr 2025 18:17:32 +0800 Subject: [PATCH 24/24] fix build --- .../server/log/remote/storage/RemoteLogManagerTest.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) 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 2de71c6654840..6068edb893825 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 @@ -3732,9 +3732,10 @@ public void testMonitorableRemoteLogStorageManager() throws IOException { appendRLMConfig(props); props.put(RemoteLogManagerConfig.REMOTE_STORAGE_MANAGER_CLASS_NAME_PROP, MonitorableNoOpRemoteStorageManager.class.getName()); props.put(RemoteLogManagerConfig.REMOTE_LOG_METADATA_MANAGER_CLASS_NAME_PROP, MonitorableNoOpRemoteLogMetadataManager.class.getName()); - KafkaConfig config = KafkaConfig.fromProps(props); + config = configs(props); + try (RemoteLogManager remoteLogManager = new RemoteLogManager( - config.remoteLogManagerConfig(), + config, brokerId, logDir, clusterId,