satishd commented on code in PR #13984: URL: https://github.com/apache/kafka/pull/13984#discussion_r1282146991
########## core/src/main/java/kafka/log/remote/RemoteLogManager.java: ########## @@ -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); + + 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)); Review Comment: Do not we need to add respective delete_segment_started and delete_segment finished events? ########## storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadata.java: ########## @@ -66,6 +68,11 @@ public class RemoteLogSegmentMetadata extends RemoteLogMetadata { */ private final int segmentSizeInBytes; + /** + * Custom metadata. Review Comment: It is good to explain one usecase for custom metadata. ########## core/src/main/java/kafka/log/remote/CustomMetadataSizeLimitExceededException.java: ########## @@ -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 { Review Comment: Right, `storage.api` is for public API/SPI classes. It is better not to keep internal classes/exceptions in that sub module. ########## storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java: ########## @@ -81,6 +81,13 @@ public final class RemoteLogManagerConfig { public static final String REMOTE_LOG_METADATA_MANAGER_LISTENER_NAME_DOC = "Listener name of the local broker to which it should get connected if " + "needed by RemoteLogMetadataManager implementation."; + public static final String REMOTE_LOG_METADATA_CUSTOM_METADATA_MAX_SIZE_PROP = "remote.log.metadata.custom.metadata.max.size"; Review Comment: Can `remote.log.metadata.custom.metadata.max.size` be changed to `remote.log.metadata.custom.metadata.max.bytes` as we follow similar convention in other places? We need to update the KIP with this change and reply to the vote thread about this change if other reviewers are fine with this change. cc @divijvaidya @showuon @kamalcph -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org