kowshik commented on a change in pull request #10271: URL: https://github.com/apache/kafka/pull/10271#discussion_r625420874
########## File path: storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadata.java ########## @@ -0,0 +1,56 @@ +/* + * 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.log.remote.storage; + +import org.apache.kafka.common.annotation.InterfaceStability; + +/** + * Base class for remote log metadata objects like {@link RemoteLogSegmentMetadata}, {@link RemoteLogSegmentMetadataUpdate}, + * and {@link RemotePartitionDeleteMetadata}. + */ +@InterfaceStability.Evolving +public abstract class RemoteLogMetadata { + + /** + * Broker id from which this event is generated. + */ + private final int brokerId; + + /** + * Epoch time in milli seconds at which this event is generated. + */ + private final long eventTimestampMs; + + public RemoteLogMetadata(int brokerId, long eventTimestampMs) { Review comment: should this be declared protected since this is an abstract class anyway? ########## File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogSegmentMetadataTransform.java ########## @@ -0,0 +1,98 @@ +/* + * 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.log.remote.metadata.storage.serialization; + +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.log.remote.metadata.storage.generated.RemoteLogSegmentMetadataRecord; +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.RemoteLogSegmentMetadataUpdate; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentState; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class RemoteLogSegmentMetadataTransform implements RemoteLogMetadataTransform<RemoteLogSegmentMetadata> { + + public ApiMessageAndVersion toApiMessageAndVersion(RemoteLogSegmentMetadata segmentMetadata) { + RemoteLogSegmentMetadataRecord record = new RemoteLogSegmentMetadataRecord() + .setRemoteLogSegmentId(createRemoteLogSegmentIdEntry(segmentMetadata)) + .setStartOffset(segmentMetadata.startOffset()) + .setEndOffset(segmentMetadata.endOffset()) + .setBrokerId(segmentMetadata.brokerId()) + .setEventTimestampMs(segmentMetadata.eventTimestampMs()) + .setMaxTimestampMs(segmentMetadata.maxTimestampMs()) + .setSegmentSizeInBytes(segmentMetadata.segmentSizeInBytes()) + .setSegmentLeaderEpochs(createSegmentLeaderEpochsEntry(segmentMetadata)) + .setRemoteLogSegmentState(segmentMetadata.state().id()); + + return new ApiMessageAndVersion(record, record.highestSupportedVersion()); + } + + private List<RemoteLogSegmentMetadataRecord.SegmentLeaderEpochEntry> createSegmentLeaderEpochsEntry(RemoteLogSegmentMetadata data) { + return data.segmentLeaderEpochs().entrySet().stream() + .map(entry -> new RemoteLogSegmentMetadataRecord.SegmentLeaderEpochEntry() + .setLeaderEpoch(entry.getKey()) + .setOffset(entry.getValue())) + .collect(Collectors.toList()); + } + + private RemoteLogSegmentMetadataRecord.RemoteLogSegmentIdEntry createRemoteLogSegmentIdEntry(RemoteLogSegmentMetadata data) { + return new RemoteLogSegmentMetadataRecord.RemoteLogSegmentIdEntry() + .setTopicIdPartition( + new RemoteLogSegmentMetadataRecord.TopicIdPartitionEntry() + .setId(data.remoteLogSegmentId().topicIdPartition().topicId()) + .setName(data.remoteLogSegmentId().topicIdPartition().topicPartition().topic()) + .setPartition(data.remoteLogSegmentId().topicIdPartition().topicPartition().partition())) + .setId(data.remoteLogSegmentId().id()); + } + + @Override + public RemoteLogSegmentMetadata fromApiMessageAndVersion(ApiMessageAndVersion apiMessageAndVersion) { + RemoteLogSegmentMetadataRecord record = (RemoteLogSegmentMetadataRecord) apiMessageAndVersion.message(); + RemoteLogSegmentId remoteLogSegmentId = buildRemoteLogSegmentId(record.remoteLogSegmentId()); + + Map<Integer, Long> segmentLeaderEpochs = new HashMap<>(); + for (RemoteLogSegmentMetadataRecord.SegmentLeaderEpochEntry segmentLeaderEpoch : record.segmentLeaderEpochs()) { + segmentLeaderEpochs.put(segmentLeaderEpoch.leaderEpoch(), segmentLeaderEpoch.offset()); + } + + RemoteLogSegmentMetadata remoteLogSegmentMetadata = + new RemoteLogSegmentMetadata(remoteLogSegmentId, record.startOffset(), record.endOffset(), + record.maxTimestampMs(), record.brokerId(), + record.eventTimestampMs(), record.segmentSizeInBytes(), + segmentLeaderEpochs); + RemoteLogSegmentMetadataUpdate rlsmUpdate Review comment: Maybe I'm missing something, but there seems to be a `RemoteLogSegmentMetadata` constructor overload that also allows you to pass the `RemoteLogSegmentState` as one of the parameters: https://github.com/apache/kafka/blob/trunk/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadata.java#L99 In that case, do you still need to apply the `RemoteLogSegmentMetadataUpdate` here? -- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org