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



##########
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 (apiKey, version, payload)? 
   
   
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 
doc suggests).




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


Reply via email to