Skip to content
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

KAFKA-15107: Support custom metadata for remote log segment #13984

Merged
merged 14 commits into from
Aug 4, 2023
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
1 change: 1 addition & 0 deletions checkstyle/suppressions.xml
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@
files="core[\\/]src[\\/](generated|generated-test)[\\/].+.java$"/>
<suppress checks="NPathComplexity" files="(ClusterTestExtensions|KafkaApisBuilder).java"/>
<suppress checks="NPathComplexity|ClassFanOutComplexity|ClassDataAbstractionCoupling" files="(RemoteLogManager|RemoteLogManagerTest).java"/>
<suppress checks="ClassFanOutComplexity" files="RemoteLogManagerTest.java"/>
<suppress checks="MethodLength"
files="(KafkaClusterTestKit).java"/>

Expand Down
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 {
ivanyu marked this conversation as resolved.
Show resolved Hide resolved
}
31 changes: 28 additions & 3 deletions core/src/main/java/kafka/log/remote/RemoteLogManager.java
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -572,6 +573,9 @@ 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.
this.cancel();
showuon marked this conversation as resolved.
Show resolved Hide resolved
} catch (InterruptedException ex) {
throw ex;
} catch (Exception ex) {
Expand All @@ -595,7 +599,8 @@ private long getNextSegmentBaseOffset(long activeSegBaseOffset, ListIterator<Log
return nextSegmentBaseOffset;
}

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();

Expand All @@ -621,10 +626,30 @@ private void copyLogSegment(UnifiedLog log, LogSegment segment, long nextSegment
producerStateSnapshotFile.toPath(), leaderEpochsIndex);
brokerTopicStats.topicStats(log.topicPartition().topic()).remoteWriteRequestRate().mark();
brokerTopicStats.allTopicsStats().remoteWriteRequestRate().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

int customMetadataSizeLimit = RemoteLogManager.this.rlmConfig.remoteLogMetadataCustomMetadataMaxSize();
showuon marked this conversation as resolved.
Show resolved Hide resolved
if (customMetadata.isPresent()) {
long customMetadataSize = customMetadata.get().value().length;
if (customMetadataSize > 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, 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
Copy link
Member

Choose a reason for hiding this comment

The 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?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Here's my reasoning why not: #13984 (comment)

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That looks reasonable to me.

} catch (RemoteStorageException e1) {
logger.error("Error while cleaning segment after custom metadata size exceeded", e1);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm not sure if we can leave the error here and not to do anything. If we failed to delete the segment and failed the RLMTask here, then how does the operator knows if we need to delete the latest segment or not? If we don't even care if the segment is deleted or not at all, why did we have to delete it anyway? Thoughts?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If this error message doesn't appear, it means the deletion was successful. However, it may be a good idea to make this explicit, so I added info logging for successful deletion and a suggestion to clean manually in case of an error.

If we don't even care if the segment is deleted or not at all, why did we have to delete it anyway?

On the high level, you're right, we don't care: the operator will fix the issue by increasing the custom metadata size limit or more radically by deleting the topic or disabling remote storage for it. In any case, there shouldn't be any garbage on the remote storage.C
leaning may matter in the case when the placement on the remote storage is nondeterministic. In this case, the subsequent write attempt will not overwrite the files but will write new ones thus leaving some garbage. So to combat this and generally as a measure of hygiene, the best-effort attempt for deletion is made.

Copy link
Collaborator

@kamalcph kamalcph Aug 5, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The approach taken looks good. Even if we fail to delete the last uploaded segment on error, it will be marked as unreferenced segment. And, when the RLM task is enabled for the topic, it will be removed in the regular segment cleanup cycle.

}
throw e;
}
}

remoteLogMetadataManager.updateRemoteLogSegmentMetadata(copySegmentFinishedRlsm).get();
brokerTopicStats.topicStats(log.topicPartition().topic())
Expand Down