-
Notifications
You must be signed in to change notification settings - Fork 15k
KAFKA-15107: Support custom metadata for remote log segment #13984
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
387c64d
621a41b
b988c9a
42abc8c
5d8a53f
34b05ad
9289d8a
a16156f
27fc3a0
6869dc9
cad75d2
473567f
3c8bac2
d564728
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,20 @@ | ||
| /* | ||
| * 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 kafka.log.remote; | ||
|
|
||
| class CustomMetadataSizeLimitExceededException extends Exception { | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -47,6 +47,7 @@ | |
| import org.apache.kafka.server.log.remote.storage.RemoteLogMetadataManager; | ||
| import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId; | ||
| import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata; | ||
| import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata.CustomMetadata; | ||
| import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadataUpdate; | ||
| import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentState; | ||
| import org.apache.kafka.server.log.remote.storage.RemoteStorageException; | ||
|
|
@@ -480,12 +481,14 @@ InMemoryLeaderEpochCheckpoint getLeaderEpochCheckpoint(UnifiedLog log, long star | |
| class RLMTask extends CancellableRunnable { | ||
|
|
||
| private final TopicIdPartition topicIdPartition; | ||
| private final int customMetadataSizeLimit; | ||
| private final Logger logger; | ||
|
|
||
| private volatile int leaderEpoch = -1; | ||
|
|
||
| public RLMTask(TopicIdPartition topicIdPartition) { | ||
| public RLMTask(TopicIdPartition topicIdPartition, int customMetadataSizeLimit) { | ||
| this.topicIdPartition = topicIdPartition; | ||
| this.customMetadataSizeLimit = customMetadataSizeLimit; | ||
| LogContext logContext = new LogContext("[RemoteLogManager=" + brokerId + " partition=" + topicIdPartition + "] "); | ||
| logger = logContext.logger(RLMTask.class); | ||
| } | ||
|
|
@@ -586,6 +589,11 @@ public void copyLogSegmentsToRemote(UnifiedLog log) throws InterruptedException | |
| } else { | ||
| logger.debug("Skipping copying segments, current read-offset:{}, and LSO:{}", copiedOffset, lso); | ||
| } | ||
| } catch (CustomMetadataSizeLimitExceededException e) { | ||
| // Only stop this task. Logging is done where the exception is thrown. | ||
| brokerTopicStats.topicStats(log.topicPartition().topic()).failedRemoteCopyRequestRate().mark(); | ||
| brokerTopicStats.allTopicsStats().failedRemoteCopyRequestRate().mark(); | ||
| this.cancel(); | ||
| } catch (InterruptedException ex) { | ||
| throw ex; | ||
| } catch (Exception ex) { | ||
|
|
@@ -597,7 +605,8 @@ public void copyLogSegmentsToRemote(UnifiedLog log) throws InterruptedException | |
| } | ||
| } | ||
|
|
||
| private void copyLogSegment(UnifiedLog log, LogSegment segment, long nextSegmentBaseOffset) throws InterruptedException, ExecutionException, RemoteStorageException, IOException { | ||
| private void copyLogSegment(UnifiedLog log, LogSegment segment, long nextSegmentBaseOffset) throws InterruptedException, ExecutionException, RemoteStorageException, IOException, | ||
| CustomMetadataSizeLimitExceededException { | ||
| File logFile = segment.log().file(); | ||
| String logFileName = logFile.getName(); | ||
|
|
||
|
|
@@ -623,10 +632,30 @@ private void copyLogSegment(UnifiedLog log, LogSegment segment, long nextSegment | |
| producerStateSnapshotFile.toPath(), leaderEpochsIndex); | ||
| brokerTopicStats.topicStats(log.topicPartition().topic()).remoteCopyRequestRate().mark(); | ||
| brokerTopicStats.allTopicsStats().remoteCopyRequestRate().mark(); | ||
| remoteLogStorageManager.copyLogSegmentData(copySegmentStartedRlsm, segmentData); | ||
| Optional<CustomMetadata> customMetadata = remoteLogStorageManager.copyLogSegmentData(copySegmentStartedRlsm, segmentData); | ||
|
|
||
| RemoteLogSegmentMetadataUpdate copySegmentFinishedRlsm = new RemoteLogSegmentMetadataUpdate(id, time.milliseconds(), | ||
| RemoteLogSegmentState.COPY_SEGMENT_FINISHED, brokerId); | ||
| customMetadata, RemoteLogSegmentState.COPY_SEGMENT_FINISHED, brokerId); | ||
showuon marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
|
||
| if (customMetadata.isPresent()) { | ||
| long customMetadataSize = customMetadata.get().value().length; | ||
| if (customMetadataSize > this.customMetadataSizeLimit) { | ||
| CustomMetadataSizeLimitExceededException e = new CustomMetadataSizeLimitExceededException(); | ||
| logger.error("Custom metadata size {} exceeds configured limit {}." + | ||
| " Copying will be stopped and copied segment will be attempted to clean." + | ||
| " Original metadata: {}", | ||
| customMetadataSize, this.customMetadataSizeLimit, copySegmentStartedRlsm, e); | ||
| try { | ||
| // For deletion, we provide back the custom metadata by creating a new metadata object from the update. | ||
| // However, the update itself will not be stored in this case. | ||
| remoteLogStorageManager.deleteLogSegmentData(copySegmentStartedRlsm.createWithUpdates(copySegmentFinishedRlsm)); | ||
showuon marked this conversation as resolved.
Show resolved
Hide resolved
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Do not we need to add respective delete_segment_started and delete_segment finished events?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Here's my reasoning why not: #13984 (comment)
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. That looks reasonable to me. |
||
| logger.info("Successfully cleaned segment after custom metadata size exceeded"); | ||
| } catch (RemoteStorageException e1) { | ||
| logger.error("Error while cleaning segment after custom metadata size exceeded, consider cleaning manually", e1); | ||
| } | ||
| throw e; | ||
| } | ||
| } | ||
|
|
||
| remoteLogMetadataManager.updateRemoteLogSegmentMetadata(copySegmentFinishedRlsm).get(); | ||
| brokerTopicStats.topicStats(log.topicPartition().topic()) | ||
|
|
@@ -883,7 +912,7 @@ void doHandleLeaderOrFollowerPartitions(TopicIdPartition topicPartition, | |
| Consumer<RLMTask> convertToLeaderOrFollower) { | ||
| RLMTaskWithFuture rlmTaskWithFuture = leaderOrFollowerTasks.computeIfAbsent(topicPartition, | ||
| topicIdPartition -> { | ||
| RLMTask task = new RLMTask(topicIdPartition); | ||
| RLMTask task = new RLMTask(topicIdPartition, this.rlmConfig.remoteLogMetadataCustomMetadataMaxBytes()); | ||
| // set this upfront when it is getting initialized instead of doing it after scheduling. | ||
| convertToLeaderOrFollower.accept(task); | ||
| LOGGER.info("Created a new task: {} and getting scheduled", task); | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.