jeffkbkim commented on code in PR #13675:
URL: https://github.com/apache/kafka/pull/13675#discussion_r1214829515


##########
core/src/main/scala/kafka/coordinator/group/CoordinatorPartitionWriter.scala:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.cluster.PartitionListener
+import kafka.server.ReplicaManager
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.errors.RecordTooLargeException
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.record.{CompressionType, MemoryRecords, 
TimestampType}
+import org.apache.kafka.common.record.Record.EMPTY_HEADERS
+import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
+import org.apache.kafka.common.utils.Time
+import org.apache.kafka.coordinator.group.runtime.PartitionWriter
+import org.apache.kafka.storage.internals.log.AppendOrigin
+
+import java.nio.ByteBuffer
+import java.util
+import scala.collection.Map
+
+private[group] class ListenerAdaptor(
+  val listener: PartitionWriter.Listener
+) extends PartitionListener {
+  override def onHighWatermarkUpdated(
+    tp: TopicPartition,
+    offset: Long
+  ): Unit = {
+    listener.onHighWatermarkUpdated(tp, offset)
+  }
+
+  override def equals(that: Any): Boolean = that match {
+    case other: ListenerAdaptor => listener.equals(other.listener)
+    case _ => false
+  }
+
+  override def hashCode(): Int = {
+    listener.hashCode()
+  }
+
+  override def toString: String = {
+    s"ListenerAdaptor(listener=$listener)"
+  }
+}
+
+class CoordinatorPartitionWriter[T](
+  replicaManager: ReplicaManager,
+  serializer: PartitionWriter.Serializer[T],
+  compressionType: CompressionType,
+  time: Time
+) extends PartitionWriter[T] {
+
+  /**
+   * Register a PartitionWriter.Listener.
+   *
+   * @param tp       The partition to register the listener to.
+   * @param listener The listener.
+   */
+  override def registerListener(
+    tp: TopicPartition,
+    listener: PartitionWriter.Listener
+  ): Unit = {
+    replicaManager.maybeAddListener(tp, new ListenerAdaptor(listener))
+  }
+
+  /**
+   * Deregister a PartitionWriter.Listener.
+   *
+   * @param tp       The partition to deregister the listener from.
+   * @param listener The listener.
+   */
+  override def deregisterListener(
+    tp: TopicPartition,
+    listener: PartitionWriter.Listener
+  ): Unit = {
+    replicaManager.removeListener(tp, new ListenerAdaptor(listener))
+  }
+
+  /**
+   * Write records to the partitions. Records are written in one batch so
+   * atomicity is guaranteed.
+   *
+   * @param tp      The partition to write records to.
+   * @param records The list of records. The records are written in a single 
batch.
+   * @return The log end offset right after the written records.
+   * @throws KafkaException Any KafkaException caught during the write 
operation.
+   */
+  override def append(
+    tp: TopicPartition,
+    records: util.List[T]
+  ): Long = {
+    if (records.isEmpty) throw new IllegalStateException("records must be 
non-empty.")
+
+    replicaManager.getLogConfig(tp) match {
+      case Some(logConfig) =>
+        val magic = logConfig.recordVersion.value
+        val maxBatchSize = logConfig.maxMessageSize
+        val currentTimeMs = time.milliseconds()
+
+        val recordsBuilder = MemoryRecords.builder(
+          ByteBuffer.allocate(math.min(16384, maxBatchSize)),
+          magic,
+          compressionType,
+          TimestampType.CREATE_TIME,
+          0L,
+          maxBatchSize
+        )
+
+        records.forEach { record =>
+          val keyBytes = serializer.serializeKey(record)
+          val valueBytes = serializer.serializeValue(record)
+
+          if (recordsBuilder.hasRoomFor(currentTimeMs, keyBytes, valueBytes, 
EMPTY_HEADERS)) recordsBuilder.append(
+            currentTimeMs,
+            keyBytes,
+            valueBytes,
+            EMPTY_HEADERS

Review Comment:
   do we not need anything in the header because these are only internal topics?
   
   what metadata do other record types contain in the header, client info?



##########
core/src/main/scala/kafka/coordinator/group/CoordinatorPartitionWriter.scala:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.cluster.PartitionListener
+import kafka.server.ReplicaManager
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.errors.RecordTooLargeException
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.record.{CompressionType, MemoryRecords, 
TimestampType}
+import org.apache.kafka.common.record.Record.EMPTY_HEADERS
+import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
+import org.apache.kafka.common.utils.Time
+import org.apache.kafka.coordinator.group.runtime.PartitionWriter
+import org.apache.kafka.storage.internals.log.AppendOrigin
+
+import java.nio.ByteBuffer
+import java.util
+import scala.collection.Map
+
+private[group] class ListenerAdaptor(
+  val listener: PartitionWriter.Listener
+) extends PartitionListener {
+  override def onHighWatermarkUpdated(
+    tp: TopicPartition,
+    offset: Long
+  ): Unit = {
+    listener.onHighWatermarkUpdated(tp, offset)
+  }
+
+  override def equals(that: Any): Boolean = that match {
+    case other: ListenerAdaptor => listener.equals(other.listener)
+    case _ => false
+  }
+
+  override def hashCode(): Int = {
+    listener.hashCode()
+  }
+
+  override def toString: String = {
+    s"ListenerAdaptor(listener=$listener)"
+  }
+}
+
+class CoordinatorPartitionWriter[T](
+  replicaManager: ReplicaManager,
+  serializer: PartitionWriter.Serializer[T],
+  compressionType: CompressionType,
+  time: Time
+) extends PartitionWriter[T] {
+
+  /**
+   * Register a PartitionWriter.Listener.
+   *
+   * @param tp       The partition to register the listener to.
+   * @param listener The listener.
+   */
+  override def registerListener(
+    tp: TopicPartition,
+    listener: PartitionWriter.Listener
+  ): Unit = {
+    replicaManager.maybeAddListener(tp, new ListenerAdaptor(listener))
+  }
+
+  /**
+   * Deregister a PartitionWriter.Listener.
+   *
+   * @param tp       The partition to deregister the listener from.
+   * @param listener The listener.
+   */
+  override def deregisterListener(
+    tp: TopicPartition,
+    listener: PartitionWriter.Listener
+  ): Unit = {
+    replicaManager.removeListener(tp, new ListenerAdaptor(listener))
+  }
+
+  /**
+   * Write records to the partitions. Records are written in one batch so
+   * atomicity is guaranteed.
+   *
+   * @param tp      The partition to write records to.
+   * @param records The list of records. The records are written in a single 
batch.
+   * @return The log end offset right after the written records.
+   * @throws KafkaException Any KafkaException caught during the write 
operation.
+   */
+  override def append(
+    tp: TopicPartition,
+    records: util.List[T]
+  ): Long = {
+    if (records.isEmpty) throw new IllegalStateException("records must be 
non-empty.")
+
+    replicaManager.getLogConfig(tp) match {
+      case Some(logConfig) =>
+        val magic = logConfig.recordVersion.value
+        val maxBatchSize = logConfig.maxMessageSize
+        val currentTimeMs = time.milliseconds()
+
+        val recordsBuilder = MemoryRecords.builder(
+          ByteBuffer.allocate(math.min(16384, maxBatchSize)),
+          magic,
+          compressionType,
+          TimestampType.CREATE_TIME,
+          0L,
+          maxBatchSize
+        )
+
+        records.forEach { record =>
+          val keyBytes = serializer.serializeKey(record)
+          val valueBytes = serializer.serializeValue(record)
+
+          if (recordsBuilder.hasRoomFor(currentTimeMs, keyBytes, valueBytes, 
EMPTY_HEADERS)) recordsBuilder.append(
+            currentTimeMs,
+            keyBytes,
+            valueBytes,
+            EMPTY_HEADERS
+          ) else throw new RecordTooLargeException(s"Message batch size is 
${recordsBuilder.estimatedSizeInBytes()} bytes " +
+            s"in append to partition $tp which exceeds the maximum configured 
size of $maxBatchSize.")
+        }
+
+        var appendResults: Map[TopicPartition, PartitionResponse] = Map.empty
+        replicaManager.appendRecords(
+          timeout = 0L,
+          requiredAcks = 1,
+          internalTopicsAllowed = true,
+          origin = AppendOrigin.COORDINATOR,
+          entriesPerPartition = Map(tp -> recordsBuilder.build()),
+          responseCallback = results => appendResults = results,
+          actionQueueAdd = item => item() // Immediately complete the action 
queue item.

Review Comment:
   my understanding is that the actionQueue and 
ActionQueue#tryCompleteActions() were created to resolve issues related to 
concurrency. 
   
   is it safe here because the threads from MultiThreadedEventProcessor do not 
hold locks and can we add a comment on why?



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1066,9 +1066,6 @@ class ReplicaManager(val config: KafkaConfig,
     requiredAcks == -1 || requiredAcks == 1 || requiredAcks == 0
   }
 
-  /**
-   * Append the messages to the local replica logs

Review Comment:
   what's the reason for removing this?



##########
clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java:
##########
@@ -131,6 +131,7 @@ public Map<Errors, Integer> errorCounts() {
     public static final class PartitionResponse {
         public Errors error;
         public long baseOffset;
+        public long lastOffset;

Review Comment:
   i always assumed that there was a 1-1 mapping between the json 
(ProduceResponse.json) and the internal representation. i'm guessing we don't 
have to change the actual protocol since we only use this value internally?



##########
core/src/main/scala/kafka/coordinator/group/CoordinatorPartitionWriter.scala:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.cluster.PartitionListener
+import kafka.server.ReplicaManager
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.errors.RecordTooLargeException
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.record.{CompressionType, MemoryRecords, 
TimestampType}
+import org.apache.kafka.common.record.Record.EMPTY_HEADERS
+import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
+import org.apache.kafka.common.utils.Time
+import org.apache.kafka.coordinator.group.runtime.PartitionWriter
+import org.apache.kafka.storage.internals.log.AppendOrigin
+
+import java.nio.ByteBuffer
+import java.util
+import scala.collection.Map
+
+private[group] class ListenerAdaptor(

Review Comment:
   nit: should we stick with Adapter? like we do for GroupCoordinatorAdapter
   
   also, can we have a comment on what this does or a link to PartitionListener



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -666,7 +667,8 @@ class ReplicaManager(val config: KafkaConfig,
                     recordConversionStatsCallback: Map[TopicPartition, 
RecordConversionStats] => Unit = _ => (),
                     requestLocal: RequestLocal = RequestLocal.NoCaching,
                     transactionalId: String = null,
-                    transactionStatePartition: Option[Int] = None): Unit = {
+                    transactionStatePartition: Option[Int] = None,
+                    actionQueueAdd: (() => Unit) => Unit = actionQueue.add): 
Unit = {

Review Comment:
   how's actionQueueOperation? we don't add() in CoordinatorPartitionWriter 
right



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

Reply via email to