You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2021/04/15 09:18:50 UTC

[GitHub] [kafka] kowshik commented on a change in pull request #10271: KAFKA-12429: Added serdes for the default implementation of RLMM based on an internal topic as storage.

kowshik commented on a change in pull request #10271:
URL: https://github.com/apache/kafka/pull/10271#discussion_r613873623



##########
File path: storage/src/main/resources/message/RemoteLogSegmentMetadata.json
##########
@@ -0,0 +1,126 @@
+// 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.
+
+{
+  "apiKey": 0,
+  "type": "data",
+  "name": "RemoteLogSegmentMetadataRecord",
+  "validVersions": "0",
+  "flexibleVersions": "none",
+  "fields": [
+    {
+      "name": "RemoteLogSegmentId",
+      "type": "RemoteLogSegmentIdEntry",
+      "versions": "0+",
+      "about": "Unique representation of the remote log segment",
+      "fields": [
+        {
+          "name": "TopicIdPartition",
+          "type": "TopicIdPartitionEntry",
+          "versions": "0+",
+          "about": "Represents unique topic partition",
+          "fields": [
+            {
+              "name": "Name",
+              "type": "string",
+              "versions": "0+",
+              "about": "Topic name"
+            },
+            {
+              "name": "Id",
+              "type": "uuid",
+              "versions": "0+",
+              "about": "Unique identifier of the topic"
+            },
+            {
+              "name": "Partition",
+              "type": "int32",
+              "versions": "0+",
+              "about": "Partition number"
+            }
+          ]
+        },
+        {
+          "name": "Id",
+          "type": "uuid",
+          "versions": "0+",
+          "about": "Unique identifier of the remote log segment"
+        }
+      ]
+    },
+    {
+      "name": "StartOffset",
+      "type": "int64",
+      "versions": "0+",
+      "about": "Start offset  of the segment."
+    },
+    {
+      "name": "EndOffset",
+      "type": "int64",
+      "versions": "0+",
+      "about": "End offset  of the segment."
+    },
+    {
+      "name": "BrokerId",

Review comment:
       This field is not present in the KIP. The KIP contains a field called `LeaderEpoch`, which is not seen here.
   Should we update the KIP suitably?

##########
File path: storage/src/main/resources/message/RemoteLogSegmentMetadataUpdate.json
##########
@@ -0,0 +1,83 @@
+// 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.
+
+{
+  "apiKey": 1,
+  "type": "data",
+  "name": "RemoteLogSegmentMetadataRecordUpdate",
+  "validVersions": "0",
+  "flexibleVersions": "none",
+  "fields": [
+    {
+      "name": "RemoteLogSegmentId",
+      "type": "RemoteLogSegmentIdEntry",
+      "versions": "0+",
+      "about": "Unique representation of the remote log segment",
+      "fields": [
+        {
+          "name": "TopicIdPartition",
+          "type": "TopicIdPartitionEntry",
+          "versions": "0+",
+          "about": "Represents unique topic partition",
+          "fields": [
+            {
+              "name": "Name",
+              "type": "string",
+              "versions": "0+",
+              "about": "Topic name"
+            },
+            {
+              "name": "Id",
+              "type": "uuid",
+              "versions": "0+",
+              "about": "Unique identifier of the topic"
+            },
+            {
+              "name": "Partition",
+              "type": "int32",
+              "versions": "0+",
+              "about": "Partition number"
+            }
+          ]
+        },
+        {
+          "name": "Id",
+          "type": "uuid",
+          "versions": "0+",
+          "about": "Unique identifier of the remote log segment"
+        }
+      ]
+    },
+    {
+      "name": "BrokerId",

Review comment:
       This field is not present in the KIP. The KIP contains a field called LeaderEpoch, which is not seen here.
   Should we update the KIP suitably?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serde/RemoteLogMetadataContextSerde.java
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.serde;
+
+import org.apache.kafka.server.log.remote.metadata.storage.RemoteLogMetadataContext;
+import org.apache.kafka.common.protocol.ByteBufferAccessor;
+import org.apache.kafka.common.protocol.Message;
+import org.apache.kafka.common.protocol.ObjectSerializationCache;
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.server.log.remote.metadata.storage.generated.RemoteLogSegmentMetadataRecord;
+import org.apache.kafka.server.log.remote.metadata.storage.generated.RemoteLogSegmentMetadataRecordUpdate;
+import org.apache.kafka.server.log.remote.metadata.storage.generated.RemotePartitionDeleteMetadataRecord;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Serialization and deserialization for {@link RemoteLogMetadataContext}. This is the root serdes for the messages
+ * that are stored in internal remote log metadata topic.
+ */
+public class RemoteLogMetadataContextSerde implements Serde<RemoteLogMetadataContext> {
+
+    public static final byte REMOTE_LOG_SEGMENT_METADATA_API_KEY = (byte) new RemoteLogSegmentMetadataRecord().apiKey();
+    public static final byte REMOTE_LOG_SEGMENT_METADATA_UPDATE_API_KEY = (byte) new RemoteLogSegmentMetadataRecordUpdate().apiKey();
+    public static final byte REMOTE_PARTITION_DELETE_API_KEY = (byte) new RemotePartitionDeleteMetadataRecord().apiKey();
+
+    private final Map<Byte, RemoteLogMetadataSerdes> keyWithSerde;
+    private final Deserializer<RemoteLogMetadataContext> rootDeserializer;
+    private final Serializer<RemoteLogMetadataContext> rootSerializer;
+
+    public RemoteLogMetadataContextSerde() {
+        keyWithSerde = createInternalSerde();
+        rootSerializer = (topic, data) -> serialize(data);
+        rootDeserializer = (topic, data) -> deserialize(data);
+    }
+
+    private Map<Byte, RemoteLogMetadataSerdes> createInternalSerde() {

Review comment:
       Can this be a static function?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serde/RemoteLogMetadataContextSerde.java
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.serde;
+
+import org.apache.kafka.server.log.remote.metadata.storage.RemoteLogMetadataContext;
+import org.apache.kafka.common.protocol.ByteBufferAccessor;
+import org.apache.kafka.common.protocol.Message;
+import org.apache.kafka.common.protocol.ObjectSerializationCache;
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.server.log.remote.metadata.storage.generated.RemoteLogSegmentMetadataRecord;
+import org.apache.kafka.server.log.remote.metadata.storage.generated.RemoteLogSegmentMetadataRecordUpdate;
+import org.apache.kafka.server.log.remote.metadata.storage.generated.RemotePartitionDeleteMetadataRecord;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Serialization and deserialization for {@link RemoteLogMetadataContext}. This is the root serdes for the messages
+ * that are stored in internal remote log metadata topic.
+ */
+public class RemoteLogMetadataContextSerde implements Serde<RemoteLogMetadataContext> {
+
+    public static final byte REMOTE_LOG_SEGMENT_METADATA_API_KEY = (byte) new RemoteLogSegmentMetadataRecord().apiKey();
+    public static final byte REMOTE_LOG_SEGMENT_METADATA_UPDATE_API_KEY = (byte) new RemoteLogSegmentMetadataRecordUpdate().apiKey();
+    public static final byte REMOTE_PARTITION_DELETE_API_KEY = (byte) new RemotePartitionDeleteMetadataRecord().apiKey();
+
+    private final Map<Byte, RemoteLogMetadataSerdes> keyWithSerde;

Review comment:
       Should we call this `keyToSerde`?

##########
File path: storage/src/main/resources/message/RemotePartitionDeleteMetadata.json
##########
@@ -0,0 +1,68 @@
+// 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.
+
+{
+  "apiKey": 2,
+  "type": "data",
+  "name": "RemotePartitionDeleteMetadataRecord",
+  "validVersions": "0",
+  "flexibleVersions": "none",
+  "fields": [
+    {
+      "name": "TopicIdPartition",
+      "type": "TopicIdPartitionEntry",
+      "versions": "0+",
+      "about": "Represents unique topic partition",
+      "fields": [
+        {
+          "name": "Name",
+          "type": "string",
+          "versions": "0+",
+          "about": "Topic name"
+        },
+        {
+          "name": "Id",
+          "type": "uuid",
+          "versions": "0+",
+          "about": "Unique identifier of the topic"
+        },
+        {
+          "name": "Partition",
+          "type": "int32",
+          "versions": "0+",
+          "about": "Partition number"
+        }
+      ]
+    },
+    {
+      "name": "BrokerId",

Review comment:
       This field is not present in the KIP. The KIP contains a field called Epoch, which is not seen here.
   Should we update the KIP suitably?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serde/RemoteLogMetadataContextSerde.java
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.serde;
+
+import org.apache.kafka.server.log.remote.metadata.storage.RemoteLogMetadataContext;
+import org.apache.kafka.common.protocol.ByteBufferAccessor;
+import org.apache.kafka.common.protocol.Message;
+import org.apache.kafka.common.protocol.ObjectSerializationCache;
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.server.log.remote.metadata.storage.generated.RemoteLogSegmentMetadataRecord;
+import org.apache.kafka.server.log.remote.metadata.storage.generated.RemoteLogSegmentMetadataRecordUpdate;
+import org.apache.kafka.server.log.remote.metadata.storage.generated.RemotePartitionDeleteMetadataRecord;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Serialization and deserialization for {@link RemoteLogMetadataContext}. This is the root serdes for the messages
+ * that are stored in internal remote log metadata topic.
+ */
+public class RemoteLogMetadataContextSerde implements Serde<RemoteLogMetadataContext> {
+
+    public static final byte REMOTE_LOG_SEGMENT_METADATA_API_KEY = (byte) new RemoteLogSegmentMetadataRecord().apiKey();
+    public static final byte REMOTE_LOG_SEGMENT_METADATA_UPDATE_API_KEY = (byte) new RemoteLogSegmentMetadataRecordUpdate().apiKey();
+    public static final byte REMOTE_PARTITION_DELETE_API_KEY = (byte) new RemotePartitionDeleteMetadataRecord().apiKey();
+
+    private final Map<Byte, RemoteLogMetadataSerdes> keyWithSerde;
+    private final Deserializer<RemoteLogMetadataContext> rootDeserializer;
+    private final Serializer<RemoteLogMetadataContext> rootSerializer;
+
+    public RemoteLogMetadataContextSerde() {
+        keyWithSerde = createInternalSerde();
+        rootSerializer = (topic, data) -> serialize(data);
+        rootDeserializer = (topic, data) -> deserialize(data);
+    }
+
+    private Map<Byte, RemoteLogMetadataSerdes> createInternalSerde() {
+        Map<Byte, RemoteLogMetadataSerdes> serdes = new HashMap<>();
+        serdes.put(REMOTE_LOG_SEGMENT_METADATA_API_KEY, new RemoteLogSegmentMetadataSerde());
+        serdes.put(REMOTE_LOG_SEGMENT_METADATA_UPDATE_API_KEY, new RemoteLogSegmentMetadataUpdateSerde());
+        serdes.put(REMOTE_PARTITION_DELETE_API_KEY, new RemotePartitionDeleteMetadataSerde());
+        return serdes;
+    }
+
+    private byte[] serialize(RemoteLogMetadataContext remoteLogMetadataContext) {
+        RemoteLogMetadataSerdes serDe = keyWithSerde.get(remoteLogMetadataContext.apiKey());
+        if (serDe == null) {
+            throw new IllegalArgumentException("Serializer for apikey: " + remoteLogMetadataContext.apiKey() +
+                                               " does not exist.");
+        }
+
+        @SuppressWarnings("unchecked")
+        Message message = serDe.serialize(remoteLogMetadataContext.payload());
+
+        return transformToBytes(message, remoteLogMetadataContext.apiKey(), remoteLogMetadataContext.version());
+    }
+
+    private RemoteLogMetadataContext deserialize(byte[] data) {
+        ByteBuffer byteBuffer = ByteBuffer.wrap(data);
+        byte apiKey = byteBuffer.get();
+        byte version = byteBuffer.get();
+        RemoteLogMetadataSerdes serDe = keyWithSerde.get(apiKey);
+        if (serDe == null) {
+            throw new IllegalArgumentException("Deserializer for apikey: " + apiKey + " does not exist.");
+        }
+
+        Object deserializedObj = serDe.deserialize(version, byteBuffer);
+        return new RemoteLogMetadataContext(apiKey, version, deserializedObj);
+    }
+
+    private byte[] transformToBytes(Message message, byte apiKey, byte apiVersion) {

Review comment:
       Should we rename this function as `toBytes()` and symmetrically rename the other `deserialize()` function above as `fromBytes()`?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serde/RemoteLogSegmentMetadataSerde.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.serde;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.ByteBufferAccessor;
+import org.apache.kafka.common.protocol.Message;
+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.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class RemoteLogSegmentMetadataSerde implements RemoteLogMetadataSerdes<RemoteLogSegmentMetadata> {
+
+    public Message serialize(RemoteLogSegmentMetadata data) {
+        RemoteLogSegmentMetadataRecord record = new RemoteLogSegmentMetadataRecord()
+                .setRemoteLogSegmentId(
+                        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()))
+                .setStartOffset(data.startOffset())
+                .setEndOffset(data.endOffset())
+                .setBrokerId(data.brokerId())
+                .setEventTimestampMs(data.eventTimestampMs())
+                .setMaxTimestampMs(data.maxTimestampMs())
+                .setSegmentSizeInBytes(data.segmentSizeInBytes())
+                .setSegmentLeaderEpochs(data.segmentLeaderEpochs().entrySet().stream()
+                        .map(entry -> new RemoteLogSegmentMetadataRecord.SegmentLeaderEpochEntry()
+                                .setLeaderEpoch(entry.getKey())
+                                .setOffset(entry.getValue())).collect(Collectors.toList()))
+                .setRemoteLogSegmentState(data.state().id());
+
+        return record;
+    }
+
+    @Override
+    public RemoteLogSegmentMetadata deserialize(byte version, ByteBuffer byteBuffer) {
+        RemoteLogSegmentMetadataRecord record = new RemoteLogSegmentMetadataRecord(
+                new ByteBufferAccessor(byteBuffer), version);
+
+        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 = new RemoteLogSegmentMetadataUpdate(remoteLogSegmentId,

Review comment:
       Hmm, why do we need to create and apply an update here?

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serde/RemoteLogSegmentMetadataSerde.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.serde;
+
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.ByteBufferAccessor;
+import org.apache.kafka.common.protocol.Message;
+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.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class RemoteLogSegmentMetadataSerde implements RemoteLogMetadataSerdes<RemoteLogSegmentMetadata> {
+
+    public Message serialize(RemoteLogSegmentMetadata data) {
+        RemoteLogSegmentMetadataRecord record = new RemoteLogSegmentMetadataRecord()
+                .setRemoteLogSegmentId(
+                        new RemoteLogSegmentMetadataRecord.RemoteLogSegmentIdEntry()

Review comment:
       The nesting is getting deeper here, could we ease it out by extracting the code that creates a `RemoteLogSegmentIdEntry` from a `RemoteLogSegmentId` into a separate helper function? It seems like we could define it as a public static function in `RemoteLogSegmentId` and then we will be able to use it here and in `RemoteLogSegmentMetadataUpdateSerde.serialize(...)`.

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serde/RemoteLogMetadataContextSerde.java
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.serde;
+
+import org.apache.kafka.server.log.remote.metadata.storage.RemoteLogMetadataContext;
+import org.apache.kafka.common.protocol.ByteBufferAccessor;
+import org.apache.kafka.common.protocol.Message;
+import org.apache.kafka.common.protocol.ObjectSerializationCache;
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.server.log.remote.metadata.storage.generated.RemoteLogSegmentMetadataRecord;
+import org.apache.kafka.server.log.remote.metadata.storage.generated.RemoteLogSegmentMetadataRecordUpdate;
+import org.apache.kafka.server.log.remote.metadata.storage.generated.RemotePartitionDeleteMetadataRecord;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Serialization and deserialization for {@link RemoteLogMetadataContext}. This is the root serdes for the messages
+ * that are stored in internal remote log metadata topic.
+ */
+public class RemoteLogMetadataContextSerde implements Serde<RemoteLogMetadataContext> {
+
+    public static final byte REMOTE_LOG_SEGMENT_METADATA_API_KEY = (byte) new RemoteLogSegmentMetadataRecord().apiKey();
+    public static final byte REMOTE_LOG_SEGMENT_METADATA_UPDATE_API_KEY = (byte) new RemoteLogSegmentMetadataRecordUpdate().apiKey();
+    public static final byte REMOTE_PARTITION_DELETE_API_KEY = (byte) new RemotePartitionDeleteMetadataRecord().apiKey();
+
+    private final Map<Byte, RemoteLogMetadataSerdes> keyWithSerde;
+    private final Deserializer<RemoteLogMetadataContext> rootDeserializer;
+    private final Serializer<RemoteLogMetadataContext> rootSerializer;
+
+    public RemoteLogMetadataContextSerde() {
+        keyWithSerde = createInternalSerde();
+        rootSerializer = (topic, data) -> serialize(data);
+        rootDeserializer = (topic, data) -> deserialize(data);
+    }
+
+    private Map<Byte, RemoteLogMetadataSerdes> createInternalSerde() {
+        Map<Byte, RemoteLogMetadataSerdes> serdes = new HashMap<>();
+        serdes.put(REMOTE_LOG_SEGMENT_METADATA_API_KEY, new RemoteLogSegmentMetadataSerde());
+        serdes.put(REMOTE_LOG_SEGMENT_METADATA_UPDATE_API_KEY, new RemoteLogSegmentMetadataUpdateSerde());
+        serdes.put(REMOTE_PARTITION_DELETE_API_KEY, new RemotePartitionDeleteMetadataSerde());
+        return serdes;
+    }
+
+    private byte[] serialize(RemoteLogMetadataContext remoteLogMetadataContext) {

Review comment:
       The `ProducerRecord` can hold a key and a value. It seems like we could store the API key in `ProducerRecord.key()` and store the serialized payload in `ProducerRecord.value()`. Why not take that route instead of serializing to a single byte array here containing both key and value? 
   
   https://kafka.apache.org/26/javadoc/index.html?org/apache/kafka/clients/producer/KafkaProducer.html
   
   This is of course assuming that `RemoteLogMetadataContextSerde` will only be used for serializing/deserializing events to/from a Kafka topic (as the class level doc suggests).

##########
File path: storage/src/main/resources/message/RemoteLogSegmentMetadata.json
##########
@@ -0,0 +1,126 @@
+// 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.
+
+{
+  "apiKey": 0,
+  "type": "data",
+  "name": "RemoteLogSegmentMetadataRecord",
+  "validVersions": "0",
+  "flexibleVersions": "none",

Review comment:
       Could we set `flexibleVersions` to `0+` so that it means optional attributes are supported from initial version?
   Same question applies to other message type definitions.

##########
File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serde/RemoteLogMetadataContextSerde.java
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.serde;
+
+import org.apache.kafka.server.log.remote.metadata.storage.RemoteLogMetadataContext;
+import org.apache.kafka.common.protocol.ByteBufferAccessor;
+import org.apache.kafka.common.protocol.Message;
+import org.apache.kafka.common.protocol.ObjectSerializationCache;
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.server.log.remote.metadata.storage.generated.RemoteLogSegmentMetadataRecord;
+import org.apache.kafka.server.log.remote.metadata.storage.generated.RemoteLogSegmentMetadataRecordUpdate;
+import org.apache.kafka.server.log.remote.metadata.storage.generated.RemotePartitionDeleteMetadataRecord;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Serialization and deserialization for {@link RemoteLogMetadataContext}. This is the root serdes for the messages
+ * that are stored in internal remote log metadata topic.
+ */
+public class RemoteLogMetadataContextSerde implements Serde<RemoteLogMetadataContext> {
+
+    public static final byte REMOTE_LOG_SEGMENT_METADATA_API_KEY = (byte) new RemoteLogSegmentMetadataRecord().apiKey();
+    public static final byte REMOTE_LOG_SEGMENT_METADATA_UPDATE_API_KEY = (byte) new RemoteLogSegmentMetadataRecordUpdate().apiKey();
+    public static final byte REMOTE_PARTITION_DELETE_API_KEY = (byte) new RemotePartitionDeleteMetadataRecord().apiKey();
+
+    private final Map<Byte, RemoteLogMetadataSerdes> keyWithSerde;
+    private final Deserializer<RemoteLogMetadataContext> rootDeserializer;
+    private final Serializer<RemoteLogMetadataContext> rootSerializer;
+
+    public RemoteLogMetadataContextSerde() {
+        keyWithSerde = createInternalSerde();
+        rootSerializer = (topic, data) -> serialize(data);
+        rootDeserializer = (topic, data) -> deserialize(data);
+    }
+
+    private Map<Byte, RemoteLogMetadataSerdes> createInternalSerde() {
+        Map<Byte, RemoteLogMetadataSerdes> serdes = new HashMap<>();
+        serdes.put(REMOTE_LOG_SEGMENT_METADATA_API_KEY, new RemoteLogSegmentMetadataSerde());
+        serdes.put(REMOTE_LOG_SEGMENT_METADATA_UPDATE_API_KEY, new RemoteLogSegmentMetadataUpdateSerde());
+        serdes.put(REMOTE_PARTITION_DELETE_API_KEY, new RemotePartitionDeleteMetadataSerde());
+        return serdes;
+    }
+
+    private byte[] serialize(RemoteLogMetadataContext remoteLogMetadataContext) {
+        RemoteLogMetadataSerdes serDe = keyWithSerde.get(remoteLogMetadataContext.apiKey());
+        if (serDe == null) {
+            throw new IllegalArgumentException("Serializer for apikey: " + remoteLogMetadataContext.apiKey() +

Review comment:
       nit: in the message s/apikey/apiKey




-- 
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:
users@infra.apache.org