[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13880: KAFKA-14462; [19/N] Add CoordinatorLoader implementation
jeffkbkim commented on code in PR #13880: URL: https://github.com/apache/kafka/pull/13880#discussion_r1245523659 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/RecordSerde.java: ## @@ -0,0 +1,158 @@ +/* + * 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.coordinator.group; + +import org.apache.kafka.common.protocol.ApiMessage; +import org.apache.kafka.common.protocol.ByteBufferAccessor; +import org.apache.kafka.common.protocol.MessageUtil; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMetadataValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupPartitionMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupPartitionMetadataValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMemberKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMemberValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMetadataValue; +import org.apache.kafka.coordinator.group.generated.GroupMetadataKey; +import org.apache.kafka.coordinator.group.generated.GroupMetadataValue; +import org.apache.kafka.coordinator.group.generated.OffsetCommitKey; +import org.apache.kafka.coordinator.group.generated.OffsetCommitValue; +import org.apache.kafka.coordinator.group.runtime.CoordinatorLoader; +import org.apache.kafka.coordinator.group.runtime.PartitionWriter; +import org.apache.kafka.server.common.ApiMessageAndVersion; + +import java.nio.BufferUnderflowException; +import java.nio.ByteBuffer; + +/** + * Serializer/Deserializer for {{@link Record}}. + */ +public class RecordSerde implements PartitionWriter.Serializer, CoordinatorLoader.Deserializer { +@Override +public byte[] serializeKey(Record record) { Review Comment: thanks, that would be cleaner. -- 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
[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13880: KAFKA-14462; [19/N] Add CoordinatorLoader implementation
jeffkbkim commented on code in PR #13880: URL: https://github.com/apache/kafka/pull/13880#discussion_r1245495804 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/RecordSerde.java: ## @@ -0,0 +1,158 @@ +/* + * 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.coordinator.group; + +import org.apache.kafka.common.protocol.ApiMessage; +import org.apache.kafka.common.protocol.ByteBufferAccessor; +import org.apache.kafka.common.protocol.MessageUtil; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMetadataValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupPartitionMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupPartitionMetadataValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMemberKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMemberValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMetadataValue; +import org.apache.kafka.coordinator.group.generated.GroupMetadataKey; +import org.apache.kafka.coordinator.group.generated.GroupMetadataValue; +import org.apache.kafka.coordinator.group.generated.OffsetCommitKey; +import org.apache.kafka.coordinator.group.generated.OffsetCommitValue; +import org.apache.kafka.coordinator.group.runtime.CoordinatorLoader; +import org.apache.kafka.coordinator.group.runtime.PartitionWriter; +import org.apache.kafka.server.common.ApiMessageAndVersion; + +import java.nio.BufferUnderflowException; +import java.nio.ByteBuffer; + +/** + * Serializer/Deserializer for {{@link Record}}. + */ +public class RecordSerde implements PartitionWriter.Serializer, CoordinatorLoader.Deserializer { +@Override +public byte[] serializeKey(Record record) { Review Comment: would that require a version bump or would we not have to since it's not actually used anywhere? -- 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
[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13880: KAFKA-14462; [19/N] Add CoordinatorLoader implementation
jeffkbkim commented on code in PR #13880: URL: https://github.com/apache/kafka/pull/13880#discussion_r1245489701 ## group-coordinator/src/test/java/org/apache/kafka/coordinator/group/RecordSerdeTest.java: ## @@ -0,0 +1,260 @@ +/* + * 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.coordinator.group; + +import org.apache.kafka.common.protocol.ApiMessage; +import org.apache.kafka.common.protocol.MessageUtil; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMetadataValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupPartitionMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupPartitionMetadataValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMemberKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMemberValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMetadataValue; +import org.apache.kafka.coordinator.group.generated.GroupMetadataKey; +import org.apache.kafka.coordinator.group.generated.GroupMetadataValue; +import org.apache.kafka.coordinator.group.generated.OffsetCommitKey; +import org.apache.kafka.coordinator.group.generated.OffsetCommitValue; +import org.apache.kafka.coordinator.group.runtime.CoordinatorLoader; +import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.junit.jupiter.api.Test; + +import java.nio.ByteBuffer; + +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class RecordSerdeTest { +@Test +public void testSerializeKey() { +RecordSerde serializer = new RecordSerde(); +Record record = new Record( +new ApiMessageAndVersion( +new ConsumerGroupMetadataKey().setGroupId("group"), +(short) 1 Review Comment: nvm, just saw the comment below -- 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
[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13880: KAFKA-14462; [19/N] Add CoordinatorLoader implementation
jeffkbkim commented on code in PR #13880: URL: https://github.com/apache/kafka/pull/13880#discussion_r1245486249 ## group-coordinator/src/test/java/org/apache/kafka/coordinator/group/RecordSerdeTest.java: ## @@ -0,0 +1,260 @@ +/* + * 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.coordinator.group; + +import org.apache.kafka.common.protocol.ApiMessage; +import org.apache.kafka.common.protocol.MessageUtil; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMetadataValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupPartitionMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupPartitionMetadataValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMemberKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMemberValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMetadataValue; +import org.apache.kafka.coordinator.group.generated.GroupMetadataKey; +import org.apache.kafka.coordinator.group.generated.GroupMetadataValue; +import org.apache.kafka.coordinator.group.generated.OffsetCommitKey; +import org.apache.kafka.coordinator.group.generated.OffsetCommitValue; +import org.apache.kafka.coordinator.group.runtime.CoordinatorLoader; +import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.junit.jupiter.api.Test; + +import java.nio.ByteBuffer; + +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class RecordSerdeTest { +@Test +public void testSerializeKey() { +RecordSerde serializer = new RecordSerde(); +Record record = new Record( +new ApiMessageAndVersion( +new ConsumerGroupMetadataKey().setGroupId("group"), +(short) 1 Review Comment: in general, it looks like we allow different key/value pairs, i.e. OffsetCommitKey + GroupMetadataValue. i guess we would hit an error on runtime when we replay a record with an invalid pair. i feel like we should enforce this when serializing. wdyt? -- 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
[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13880: KAFKA-14462; [19/N] Add CoordinatorLoader implementation
jeffkbkim commented on code in PR #13880: URL: https://github.com/apache/kafka/pull/13880#discussion_r1243002218 ## core/src/main/scala/kafka/coordinator/group/CoordinatorLoaderImpl.scala: ## @@ -0,0 +1,168 @@ +/* + * 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.coordinator.group + +import kafka.server.ReplicaManager +import kafka.utils.Logging +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.errors.NotLeaderOrFollowerException +import org.apache.kafka.common.record.{FileRecords, MemoryRecords} +import org.apache.kafka.coordinator.group.runtime.CoordinatorLoader.{Deserializer, UnknownRecordTypeException} +import org.apache.kafka.coordinator.group.runtime.{CoordinatorLoader, CoordinatorPlayback} +import org.apache.kafka.server.util.KafkaScheduler +import org.apache.kafka.storage.internals.log.FetchIsolation + +import java.nio.ByteBuffer +import java.util.concurrent.CompletableFuture +import java.util.concurrent.atomic.AtomicBoolean +import scala.jdk.CollectionConverters._ + +/** + * Coordinator loader which reads records from a partition and replays them + * to a group coordinator. + * + * @param replicaManager The replica manager. + * @param deserializerThe deserializer to use. + * @param loadBufferSize The load buffer size. + * @tparam T The record type. + */ +class CoordinatorLoaderImpl[T]( + replicaManager: ReplicaManager, + deserializer: Deserializer[T], + loadBufferSize: Int +) extends CoordinatorLoader[T] with Logging { + private val isRunning = new AtomicBoolean(true) + private val scheduler = new KafkaScheduler(1) + scheduler.startup() + + /** + * Loads the coordinator by reading all the records from the TopicPartition + * and applying them to the Replayable object. + * + * @param tp The TopicPartition to read from. + * @param coordinator The object to apply records to. + */ + override def load( +tp: TopicPartition, +coordinator: CoordinatorPlayback[T] +): CompletableFuture[Void] = { +val future = new CompletableFuture[Void]() +val result = scheduler.scheduleOnce(s"Load coordinator from $tp", + () => doLoad(tp, coordinator, future)) +if (result.isCancelled) { + future.completeExceptionally(new RuntimeException("Coordinator loader is closed.")) +} +future + } + + private def doLoad( +tp: TopicPartition, +coordinator: CoordinatorPlayback[T], +future: CompletableFuture[Void] + ): Unit = { +try { + replicaManager.getLog(tp) match { +case None => + future.completeExceptionally(new NotLeaderOrFollowerException( +s"Could not load records from $tp because the log does not exist.")) + +case Some(log) => + def logEndOffset: Long = replicaManager.getLogEndOffset(tp).getOrElse(-1L) + + // buffer may not be needed if records are read from memory + var buffer = ByteBuffer.allocate(0) + // loop breaks if leader changes at any time during the load, since logEndOffset is -1 + var currOffset = log.logStartOffset + // loop breaks if no records have been read, since the end of the log has been reached + var readAtLeastOneRecord = true + + while (currOffset < logEndOffset && readAtLeastOneRecord && isRunning.get) { +val fetchDataInfo = log.read( + startOffset = currOffset, + maxLength = loadBufferSize, + isolation = FetchIsolation.LOG_END, + minOneMessage = true +) + +readAtLeastOneRecord = fetchDataInfo.records.sizeInBytes > 0 + +val memoryRecords = (fetchDataInfo.records: @unchecked) match { + case records: MemoryRecords => +records + + case fileRecords: FileRecords => +val sizeInBytes = fileRecords.sizeInBytes +val bytesNeeded = Math.max(loadBufferSize, sizeInBytes) + +// minOneMessage = true in the above log.read means that the buffer may need to Review Comment: nit: was confused on log.read. maybe use `log.read()`? ## group-coordinator/src/test/java/org/apache/kafka/coordinator/group/Record
[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13880: KAFKA-14462; [19/N] Add CoordinatorLoader implementation
jeffkbkim commented on code in PR #13880: URL: https://github.com/apache/kafka/pull/13880#discussion_r1237755278 ## core/src/main/scala/kafka/coordinator/group/CoordinatorLoaderImpl.scala: ## @@ -0,0 +1,168 @@ +/* + * 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.coordinator.group + +import kafka.server.ReplicaManager +import kafka.utils.Logging +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.errors.NotLeaderOrFollowerException +import org.apache.kafka.common.record.{FileRecords, MemoryRecords} +import org.apache.kafka.coordinator.group.runtime.CoordinatorLoader.{Deserializer, UnknownRecordTypeException} +import org.apache.kafka.coordinator.group.runtime.{CoordinatorLoader, CoordinatorPlayback} +import org.apache.kafka.server.util.KafkaScheduler +import org.apache.kafka.storage.internals.log.FetchIsolation + +import java.nio.ByteBuffer +import java.util.concurrent.CompletableFuture +import java.util.concurrent.atomic.AtomicBoolean +import scala.jdk.CollectionConverters._ + +/** + * Coordinator loader which reads records from a partition and replays them + * to a group coordinator. + * + * @param replicaManager The replica manager. + * @param deserializerThe deserializer to use. + * @param loadBufferSize The load buffer size. + * @tparam T The record type. + */ +class CoordinatorLoaderImpl[T]( + replicaManager: ReplicaManager, + deserializer: Deserializer[T], + loadBufferSize: Int +) extends CoordinatorLoader[T] with Logging { + private val isRunning = new AtomicBoolean(true) + private val scheduler = new KafkaScheduler(1) + scheduler.startup() + + /** + * Loads the coordinator by reading all the records from the TopicPartition + * and applying them to the Replayable object. + * + * @param tp The TopicPartition to read from. + * @param coordinator The object to apply records to. + */ + override def load( +tp: TopicPartition, +coordinator: CoordinatorPlayback[T] +): CompletableFuture[Void] = { +val future = new CompletableFuture[Void]() +val result = scheduler.scheduleOnce(s"Load coordinator from $tp", + () => doLoad(tp, coordinator, future)) Review Comment: before this is executed by the runtime, what happens if requests for a non loaded partition come in? how do we check whether a partition is available or not? -- 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
[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13880: KAFKA-14462; [19/N] Add CoordinatorLoader implementation
jeffkbkim commented on code in PR #13880: URL: https://github.com/apache/kafka/pull/13880#discussion_r1236242369 ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/RecordSerDe.java: ## @@ -0,0 +1,161 @@ +/* Review Comment: nit: Other serdes follow "Serde" capilization ## group-coordinator/src/main/java/org/apache/kafka/coordinator/group/RecordSerDe.java: ## @@ -0,0 +1,161 @@ +/* + * 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.coordinator.group; + +import org.apache.kafka.common.protocol.ApiMessage; +import org.apache.kafka.common.protocol.ByteBufferAccessor; +import org.apache.kafka.common.protocol.MessageUtil; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMetadataValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupPartitionMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupPartitionMetadataValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMemberKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMemberValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMetadataValue; +import org.apache.kafka.coordinator.group.generated.GroupMetadataKey; +import org.apache.kafka.coordinator.group.generated.GroupMetadataValue; +import org.apache.kafka.coordinator.group.generated.OffsetCommitKey; +import org.apache.kafka.coordinator.group.generated.OffsetCommitValue; +import org.apache.kafka.coordinator.group.runtime.CoordinatorLoader; +import org.apache.kafka.coordinator.group.runtime.PartitionWriter; +import org.apache.kafka.server.common.ApiMessageAndVersion; + +import java.nio.BufferUnderflowException; +import java.nio.ByteBuffer; + +/** + * Serializer/Deserializer for {{@link Record}}. + */ +public class RecordSerDe implements PartitionWriter.Serializer, CoordinatorLoader.Deserializer { +@Override +public byte[] serializeKey(Record record) { +// Record does not accept a null key. +return MessageUtil.toVersionPrefixedBytes( +record.key().version(), +record.key().message() +); +} + +@Override +public byte[] serializeValue(Record record) { +// Tombstone is represented with a null value. +if (record.value() == null) { +return null; +} else { +return MessageUtil.toVersionPrefixedBytes( +record.value().version(), +record.value().message() +); +} +} + +@Override +public Record deserialize( +ByteBuffer keyBuffer, +ByteBuffer valueBuffer +) throws RuntimeException { +final short keyVersion = readVersion(keyBuffer, "key"); +final ApiMessage keyMessage = apiMessageKeyFor(keyVersion); +readMessage(keyMessage, keyBuffer, keyVersion, "key"); + +if (valueBuffer == null) { +return new Record( +new ApiMessageAndVersion(keyMessage, keyVersion), +null +); +} + +final ApiMessage valueMessage = apiMessageValueFor(keyVersion); +final short valueVersion = readVersion(valueBuffer, "value"); +readMessage(valueMessage, valueBuffer, valueVersion, "value"); + +return new Record( +new ApiMessageAndVersion(keyMessage, keyVersion), +new ApiMessageAndVersion(valueMessage, valueVersion) +); +} + +private short readVersion(ByteBuffer buffer, String name) throws RuntimeException { +try { +return buffer.getShort(); +}