frankvicky commented on code in PR #19793: URL: https://github.com/apache/kafka/pull/19793#discussion_r2108058643
########## core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala: ########## @@ -271,9 +271,10 @@ class LocalLeaderEndPointTest extends Logging { origin: AppendOrigin = AppendOrigin.CLIENT, requiredAcks: Short = -1): CallbackResult[PartitionResponse] = { val result = new CallbackResult[PartitionResponse]() - def appendCallback(responses: scala.collection.Map[TopicIdPartition, PartitionResponse]): Unit = { - val response = responses.get(partition) - assertTrue(response.isDefined) + def appendCallback(responses: java.util.Map[TopicIdPartition, PartitionResponse]): Unit = { Review Comment: Given that we already have an import alias for `java.util.Map`, we could reuse it. ```suggestion def appendCallback(responses: JMap[TopicIdPartition, PartitionResponse]): Unit = { ``` ########## core/src/main/scala/kafka/server/ReplicaManager.scala: ########## @@ -967,12 +969,30 @@ class ReplicaManager(val config: KafkaConfig, entriesPerPartition: Map[TopicIdPartition, MemoryRecords], initialAppendResults: Map[TopicIdPartition, LogAppendResult], initialProduceStatus: Map[TopicIdPartition, ProducePartitionStatus], - responseCallback: Map[TopicIdPartition, PartitionResponse] => Unit, + responseCallback: util.Map[TopicIdPartition, PartitionResponse] => Unit, ): Unit = { if (delayedProduceRequestRequired(requiredAcks, entriesPerPartition, initialAppendResults)) { // create delayed produce operation - val produceMetadata = ProduceMetadata(requiredAcks, initialProduceStatus) - val delayedProduce = new DelayedProduce(timeoutMs, produceMetadata, this, responseCallback) + val produceMetadata = new ProduceMetadata(requiredAcks, initialProduceStatus.asJava) + + def delegate(tp: TopicPartition, status: ProducePartitionStatus) : Unit = { + val (hasEnough, error) = getPartitionOrError(tp) match { + case Left(err) => + // Case A Review Comment: What does this comment mean? ########## core/src/main/scala/kafka/server/ReplicaManager.scala: ########## @@ -967,12 +969,30 @@ class ReplicaManager(val config: KafkaConfig, entriesPerPartition: Map[TopicIdPartition, MemoryRecords], initialAppendResults: Map[TopicIdPartition, LogAppendResult], initialProduceStatus: Map[TopicIdPartition, ProducePartitionStatus], - responseCallback: Map[TopicIdPartition, PartitionResponse] => Unit, + responseCallback: util.Map[TopicIdPartition, PartitionResponse] => Unit, ): Unit = { if (delayedProduceRequestRequired(requiredAcks, entriesPerPartition, initialAppendResults)) { // create delayed produce operation - val produceMetadata = ProduceMetadata(requiredAcks, initialProduceStatus) - val delayedProduce = new DelayedProduce(timeoutMs, produceMetadata, this, responseCallback) + val produceMetadata = new ProduceMetadata(requiredAcks, initialProduceStatus.asJava) + + def delegate(tp: TopicPartition, status: ProducePartitionStatus) : Unit = { + val (hasEnough, error) = getPartitionOrError(tp) match { + case Left(err) => + // Case A Review Comment: I see this is relevant to the comment of `DelayedProduce,` but it's confusing that these comments are standalone here. Could you write the whole meaning of these cases, or link these comments to `tryComplete` ? ########## core/src/test/scala/unit/kafka/server/KafkaApisTest.scala: ########## @@ -2832,7 +2832,7 @@ class KafkaApisTest extends Logging { any(), ArgumentMatchers.eq(requestLocal), any() - )).thenAnswer(_ => responseCallback.getValue.apply(Map(new TopicIdPartition(topicId,tp2) -> new PartitionResponse(Errors.NONE)))) + )).thenAnswer(_ => responseCallback.getValue.apply(Map(new TopicIdPartition(topicId,tp2) -> new PartitionResponse(Errors.NONE)).asJava)) Review Comment: ```suggestion )).thenAnswer(_ => responseCallback.getValue.apply(util.Map.of(new TopicIdPartition(topicId,tp2), new PartitionResponse(Errors.NONE)))) ``` ########## core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala: ########## @@ -2990,9 +2990,9 @@ class ReplicaManagerTest { requiredAcks: Short = -1): CallbackResult[PartitionResponse] = { val result = new CallbackResult[PartitionResponse]() val topicIdPartition = new TopicIdPartition(topicId, partition) - def appendCallback(responses: Map[TopicIdPartition, PartitionResponse]): Unit = { - val response = responses.get(topicIdPartition) - assertTrue(response.isDefined) + def appendCallback(responses: util.Map[TopicIdPartition, PartitionResponse]): Unit = { + val response = java.util.Optional.ofNullable(responses.get(topicIdPartition)) Review Comment: We already imported `java.util.Optional`, so we don't need a full-qualified name. ########## core/src/main/scala/kafka/server/ReplicaManager.scala: ########## @@ -967,12 +969,30 @@ class ReplicaManager(val config: KafkaConfig, entriesPerPartition: Map[TopicIdPartition, MemoryRecords], initialAppendResults: Map[TopicIdPartition, LogAppendResult], initialProduceStatus: Map[TopicIdPartition, ProducePartitionStatus], - responseCallback: Map[TopicIdPartition, PartitionResponse] => Unit, + responseCallback: util.Map[TopicIdPartition, PartitionResponse] => Unit, ): Unit = { if (delayedProduceRequestRequired(requiredAcks, entriesPerPartition, initialAppendResults)) { // create delayed produce operation - val produceMetadata = ProduceMetadata(requiredAcks, initialProduceStatus) - val delayedProduce = new DelayedProduce(timeoutMs, produceMetadata, this, responseCallback) + val produceMetadata = new ProduceMetadata(requiredAcks, initialProduceStatus.asJava) + + def delegate(tp: TopicPartition, status: ProducePartitionStatus) : Unit = { + val (hasEnough, error) = getPartitionOrError(tp) match { + case Left(err) => + // Case A + (false, err) + + case Right(partition) => + partition.checkEnoughReplicasReachOffset(status.requiredOffset) + } + + // Case B || C.1 || C.2 Review Comment: ditto ########## core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala: ########## @@ -1104,7 +1104,7 @@ class TransactionStateManagerTest { capturedAppends: mutable.Map[TopicIdPartition, mutable.Buffer[MemoryRecords]] ): Unit = { val recordsCapture: ArgumentCaptor[Map[TopicIdPartition, MemoryRecords]] = ArgumentCaptor.forClass(classOf[Map[TopicIdPartition, MemoryRecords]]) - val callbackCapture: ArgumentCaptor[Map[TopicIdPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[Map[TopicIdPartition, PartitionResponse] => Unit]) + val callbackCapture: ArgumentCaptor[java.util.Map[TopicIdPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[java.util.Map[TopicIdPartition, PartitionResponse] => Unit]) Review Comment: please import `java.util` ########## server/src/main/java/org/apache/kafka/server/DelayedProduce.java: ########## @@ -0,0 +1,235 @@ +/* + * 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; + +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse; +import org.apache.kafka.server.metrics.KafkaMetricsGroup; +import org.apache.kafka.server.purgatory.DelayedOperation; + +import com.yammer.metrics.core.Meter; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; +import java.util.function.BiConsumer; +import java.util.function.Consumer; +import java.util.stream.Collectors; + +/** + * A delayed produce operation that can be created by the replica manager and watched + * in the produce operation purgatory + */ +public class DelayedProduce extends DelayedOperation { + private static final Logger LOGGER = LoggerFactory.getLogger(DelayedProduce.class); + + public static final class ProducePartitionStatus { + private final long requiredOffset; + private final PartitionResponse responseStatus; + + private volatile boolean acksPending; + + public ProducePartitionStatus(long requiredOffset, PartitionResponse responseStatus) { + this.requiredOffset = requiredOffset; + this.responseStatus = responseStatus; + } + + public long requiredOffset() { + return requiredOffset; + } + + public PartitionResponse responseStatus() { + return responseStatus; + } + + public boolean acksPending() { + return acksPending; + } + + public void setAcksPending(boolean acksPending) { + this.acksPending = acksPending; + } + + @Override + public String toString() { + return String.format( + "[acksPending: %s, error: %s, startOffset: %s, requiredOffset: %d]", + acksPending, + responseStatus.error.code(), + responseStatus.baseOffset, + requiredOffset + ); + } + + @Override + public boolean equals(Object o) { + if (o == null || getClass() != o.getClass()) { + return false; + } + final ProducePartitionStatus that = (ProducePartitionStatus) o; + return requiredOffset == that.requiredOffset && acksPending == that.acksPending && Objects.equals(responseStatus, that.responseStatus); + } + + @Override + public int hashCode() { + return Objects.hash(requiredOffset, responseStatus, acksPending); + } + } + + /** + * The produce metadata maintained by the delayed produce operation + */ + public static final class ProduceMetadata { + private final short produceRequiredAcks; + private final Map<TopicIdPartition, ProducePartitionStatus> produceStatus; + + public ProduceMetadata(short produceRequiredAcks, + Map<TopicIdPartition, ProducePartitionStatus> produceStatus) { + this.produceRequiredAcks = produceRequiredAcks; + this.produceStatus = produceStatus; + } + + @Override + public String toString() { + return String.format( + "[requiredAcks: %d, partitionStatus: %s]", + produceRequiredAcks, + produceStatus + ); + } + + @Override + public boolean equals(Object o) { + if (o == null || getClass() != o.getClass()) { + return false; + } + final ProduceMetadata that = (ProduceMetadata) o; + return produceRequiredAcks == that.produceRequiredAcks && Objects.equals(produceStatus, that.produceStatus); + } + + @Override + public int hashCode() { + return Objects.hash(produceRequiredAcks, produceStatus); + } + } + + private static final class DelayedProduceMetrics { + private static final KafkaMetricsGroup METRICS_GROUP = new KafkaMetricsGroup(DelayedProduceMetrics.class); + private static final Meter AGGREGATE_EXPIRATION_METER = METRICS_GROUP.newMeter("ExpiresPerSec", "requests", TimeUnit.SECONDS); + private static final ConcurrentHashMap<TopicPartition, Meter> PARTITION_EXPIRATION_METERS = new ConcurrentHashMap<>(); + + private static void recordExpiration(TopicPartition partition) { + AGGREGATE_EXPIRATION_METER.mark(); + PARTITION_EXPIRATION_METERS.computeIfAbsent(partition, + key -> METRICS_GROUP.newMeter("ExpiresPerSec", "requests", TimeUnit.SECONDS, + Map.of("topic", key.topic(), "partition", String.valueOf(key.partition())))) + .mark(); + } + } + + private final ProduceMetadata produceMetadata; + private final BiConsumer<TopicPartition, ProducePartitionStatus> updateProducePartitionStatusCallback; + private final Consumer<Map<TopicIdPartition, PartitionResponse>> responseCallback; + + public DelayedProduce(long delayMs, + ProduceMetadata produceMetadata, + BiConsumer<TopicPartition, ProducePartitionStatus> updateProducePartitionStatusCallback, + Consumer<Map<TopicIdPartition, PartitionResponse>> responseCallback) { + super(delayMs); + + this.produceMetadata = produceMetadata; + this.updateProducePartitionStatusCallback = updateProducePartitionStatusCallback; + this.responseCallback = responseCallback; + + // first update the acks pending variable according to the error code + produceMetadata.produceStatus.forEach((topicPartition, status) -> { + if (status.responseStatus.error == Errors.NONE) { + // Timeout error state will be cleared when required acks are received + status.acksPending = true; + status.responseStatus.error = Errors.REQUEST_TIMED_OUT; + } else { + status.acksPending = false; + } + + LOGGER.trace("Initial partition status for {} is {}", topicPartition, status); + }); + } + + /** + * The delayed produce operation can be completed if every partition + * it produces to is satisfied by one of the following: + * + * Case A: Replica not assigned to partition + * Case B: Replica is no longer the leader of this partition + * Case C: This broker is the leader: + * C.1 - If there was a local error thrown while checking if at least requiredAcks + * replicas have caught up to this operation: set an error in response + * C.2 - Otherwise, set the response with no error. + */ + @Override + public boolean tryComplete() { + // check for each partition if it still has pending acks + produceMetadata.produceStatus.forEach((topicIdPartition, status) -> { + LOGGER.trace("Checking produce satisfaction for {}, current status {}", topicIdPartition, status); + // skip those partitions that have already been satisfied + if (status.acksPending) { + updateProducePartitionStatusCallback.accept(topicIdPartition.topicPartition(), status); + } + }); + + // check if every partition has satisfied at least one of case A, B or C + boolean anyPending = produceMetadata.produceStatus + .values() + .stream() + .anyMatch(ProducePartitionStatus::acksPending); + if (!anyPending) { + return forceComplete(); + } + + return false; Review Comment: nit: ```suggestion return produceMetadata.produceStatus.values() .stream() .findAny() .map(__ -> false) .orElseGet(this::forceComplete); ``` ########## server/src/main/java/org/apache/kafka/server/DelayedProduce.java: ########## @@ -0,0 +1,235 @@ +/* + * 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; + +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse; +import org.apache.kafka.server.metrics.KafkaMetricsGroup; +import org.apache.kafka.server.purgatory.DelayedOperation; + +import com.yammer.metrics.core.Meter; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; +import java.util.function.BiConsumer; +import java.util.function.Consumer; +import java.util.stream.Collectors; + +/** + * A delayed produce operation that can be created by the replica manager and watched + * in the produce operation purgatory + */ +public class DelayedProduce extends DelayedOperation { + private static final Logger LOGGER = LoggerFactory.getLogger(DelayedProduce.class); + + public static final class ProducePartitionStatus { + private final long requiredOffset; + private final PartitionResponse responseStatus; + + private volatile boolean acksPending; + + public ProducePartitionStatus(long requiredOffset, PartitionResponse responseStatus) { + this.requiredOffset = requiredOffset; + this.responseStatus = responseStatus; + } + + public long requiredOffset() { + return requiredOffset; + } + + public PartitionResponse responseStatus() { + return responseStatus; + } + + public boolean acksPending() { + return acksPending; + } + + public void setAcksPending(boolean acksPending) { + this.acksPending = acksPending; + } + + @Override + public String toString() { + return String.format( + "[acksPending: %s, error: %s, startOffset: %s, requiredOffset: %d]", + acksPending, + responseStatus.error.code(), + responseStatus.baseOffset, + requiredOffset + ); + } + + @Override + public boolean equals(Object o) { + if (o == null || getClass() != o.getClass()) { + return false; + } + final ProducePartitionStatus that = (ProducePartitionStatus) o; + return requiredOffset == that.requiredOffset && acksPending == that.acksPending && Objects.equals(responseStatus, that.responseStatus); + } + + @Override + public int hashCode() { + return Objects.hash(requiredOffset, responseStatus, acksPending); + } + } + + /** + * The produce metadata maintained by the delayed produce operation + */ + public static final class ProduceMetadata { + private final short produceRequiredAcks; + private final Map<TopicIdPartition, ProducePartitionStatus> produceStatus; + + public ProduceMetadata(short produceRequiredAcks, + Map<TopicIdPartition, ProducePartitionStatus> produceStatus) { + this.produceRequiredAcks = produceRequiredAcks; + this.produceStatus = produceStatus; + } + + @Override + public String toString() { + return String.format( + "[requiredAcks: %d, partitionStatus: %s]", + produceRequiredAcks, + produceStatus + ); + } + + @Override + public boolean equals(Object o) { + if (o == null || getClass() != o.getClass()) { + return false; + } + final ProduceMetadata that = (ProduceMetadata) o; + return produceRequiredAcks == that.produceRequiredAcks && Objects.equals(produceStatus, that.produceStatus); + } + + @Override + public int hashCode() { + return Objects.hash(produceRequiredAcks, produceStatus); + } + } + + private static final class DelayedProduceMetrics { + private static final KafkaMetricsGroup METRICS_GROUP = new KafkaMetricsGroup(DelayedProduceMetrics.class); + private static final Meter AGGREGATE_EXPIRATION_METER = METRICS_GROUP.newMeter("ExpiresPerSec", "requests", TimeUnit.SECONDS); + private static final ConcurrentHashMap<TopicPartition, Meter> PARTITION_EXPIRATION_METERS = new ConcurrentHashMap<>(); + + private static void recordExpiration(TopicPartition partition) { + AGGREGATE_EXPIRATION_METER.mark(); + PARTITION_EXPIRATION_METERS.computeIfAbsent(partition, + key -> METRICS_GROUP.newMeter("ExpiresPerSec", "requests", TimeUnit.SECONDS, + Map.of("topic", key.topic(), "partition", String.valueOf(key.partition())))) + .mark(); Review Comment: style nit: ```suggestion PARTITION_EXPIRATION_METERS.computeIfAbsent(partition, key -> METRICS_GROUP.newMeter("ExpiresPerSec", "requests", TimeUnit.SECONDS, Map.of("topic", key.topic(), "partition", String.valueOf(key.partition()))) ).mark(); ``` -- 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