showuon commented on code in PR #13984: URL: https://github.com/apache/kafka/pull/13984#discussion_r1280394875
########## core/src/main/java/kafka/log/remote/RemoteLogManager.java: ########## @@ -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); Review Comment: Should we create this after metadata size validation? ########## core/src/main/java/kafka/log/remote/RemoteLogManager.java: ########## @@ -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); + + int customMetadataSizeLimit = RemoteLogManager.this.rlmConfig.remoteLogMetadataCustomMetadataMaxSize(); Review Comment: I think this is a global config value, right? Then, maybe we can have a class variable for this value. ########## storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java: ########## @@ -393,6 +413,7 @@ public boolean equals(Object o) { && Objects.equals(remoteLogMetadataManagerClassName, that.remoteLogMetadataManagerClassName) && Objects.equals(remoteLogMetadataManagerClassPath, that.remoteLogMetadataManagerClassPath) && Objects.equals(remoteLogMetadataManagerListenerName, that.remoteLogMetadataManagerListenerName) + && remoteLogMetadataCustomMetadataMaxSize == that.remoteLogMetadataCustomMetadataMaxSize Review Comment: nit: Could we move this line up to L411, since this is a cheaper comparison compared with object comparison. ########## 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: Is this the public API? ########## storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java: ########## @@ -169,6 +176,11 @@ public final class RemoteLogManagerConfig { new ConfigDef.NonEmptyString(), MEDIUM, REMOTE_LOG_METADATA_MANAGER_LISTENER_NAME_DOC) + .defineInternal(REMOTE_LOG_METADATA_CUSTOM_METADATA_MAX_SIZE_PROP, INT, Review Comment: nit: `INT` should be put in anothehr line. -- 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