Skip to content

KAFKA-19144 Move DelayedProduce to server module #19793

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 7 commits into
base: trunk
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion core/src/main/scala/kafka/cluster/Partition.scala
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,7 @@ import org.apache.kafka.server.log.remote.TopicPartitionLog
import org.apache.kafka.server.log.remote.storage.RemoteLogManager
import org.apache.kafka.storage.internals.log.{AppendOrigin, AsyncOffsetReader, FetchDataInfo, LeaderHwChange, LogAppendInfo, LogOffsetMetadata, LogOffsetSnapshot, LogOffsetsListener, LogReadInfo, LogStartOffsetIncrementReason, OffsetResultHolder, UnifiedLog, VerificationGuard}
import org.apache.kafka.server.metrics.KafkaMetricsGroup
import org.apache.kafka.server.purgatory.{DelayedDeleteRecords, DelayedOperationPurgatory, TopicPartitionOperationKey}
import org.apache.kafka.server.purgatory.{DelayedDeleteRecords, DelayedOperationPurgatory, DelayedProduce, TopicPartitionOperationKey}
import org.apache.kafka.server.replica.Replica
import org.apache.kafka.server.share.fetch.DelayedShareFetchPartitionKey
import org.apache.kafka.server.storage.log.{FetchIsolation, FetchParams, UnexpectedAppendOffsetException}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@ import org.apache.kafka.storage.internals.log.AppendOrigin
import com.google.re2j.{Pattern, PatternSyntaxException}
import org.apache.kafka.common.errors.InvalidRegularExpression

import java.util
import scala.jdk.CollectionConverters._
import scala.collection.mutable

Expand Down Expand Up @@ -256,8 +257,8 @@ class TransactionStateManager(brokerId: Int,
expiredForPartition: Iterable[TransactionalIdCoordinatorEpochAndMetadata],
tombstoneRecords: MemoryRecords
): Unit = {
def removeFromCacheCallback(responses: collection.Map[TopicIdPartition, PartitionResponse]): Unit = {
responses.foreachEntry { (topicPartition, response) =>
def removeFromCacheCallback(responses: util.Map[TopicIdPartition, PartitionResponse]): Unit = {
responses.forEach { (topicPartition, response) =>
inReadLock(stateLock) {
transactionMetadataCache.get(topicPartition.partition).foreach { txnMetadataCacheEntry =>
expiredForPartition.foreach { idCoordinatorEpochAndMetadata =>
Expand Down Expand Up @@ -667,13 +668,13 @@ class TransactionStateManager(brokerId: Int,
val recordsPerPartition = Map(transactionStateTopicIdPartition -> records)

// set the callback function to update transaction status in cache after log append completed
def updateCacheCallback(responseStatus: collection.Map[TopicIdPartition, PartitionResponse]): Unit = {
def updateCacheCallback(responseStatus: util.Map[TopicIdPartition, PartitionResponse]): Unit = {
// the append response should only contain the topics partition
if (responseStatus.size != 1 || !responseStatus.contains(transactionStateTopicIdPartition))
if (responseStatus.size != 1 || !responseStatus.containsKey(transactionStateTopicIdPartition))
throw new IllegalStateException("Append status %s should only have one partition %s"
.format(responseStatus, transactionStateTopicPartition))

val status = responseStatus(transactionStateTopicIdPartition)
val status = responseStatus.get(transactionStateTopicIdPartition)

var responseError = if (status.error == Errors.NONE) {
Errors.NONE
Expand Down
151 changes: 0 additions & 151 deletions core/src/main/scala/kafka/server/DelayedProduce.scala

This file was deleted.

2 changes: 1 addition & 1 deletion core/src/main/scala/kafka/server/KafkaApis.scala
Original file line number Diff line number Diff line change
Expand Up @@ -1818,7 +1818,7 @@ class KafkaApis(val requestChannel: RequestChannel,
entriesPerPartition = controlRecords,
requestLocal = requestLocal,
responseCallback = errors => {
errors.foreachEntry { (topicIdPartition, partitionResponse) =>
errors.forEach { (topicIdPartition, partitionResponse) =>
addResultAndMaybeComplete(topicIdPartition.topicPartition(), partitionResponse.error)
}
}
Expand Down
59 changes: 42 additions & 17 deletions core/src/main/scala/kafka/server/ReplicaManager.scala
Original file line number Diff line number Diff line change
Expand Up @@ -50,13 +50,14 @@ import org.apache.kafka.coordinator.transaction.{AddPartitionsToTxnConfig, Trans
import org.apache.kafka.image.{LocalReplicaChanges, MetadataImage, TopicsDelta}
import org.apache.kafka.metadata.LeaderConstants.NO_LEADER
import org.apache.kafka.metadata.MetadataCache
import org.apache.kafka.server.purgatory.DelayedProduce.{ProduceMetadata, ProducePartitionStatus}
import org.apache.kafka.server.common.{DirectoryEventHandler, RequestLocal, StopPartition}
import org.apache.kafka.server.log.remote.TopicPartitionLog
import org.apache.kafka.server.config.ReplicationConfigs
import org.apache.kafka.server.log.remote.storage.RemoteLogManager
import org.apache.kafka.server.metrics.KafkaMetricsGroup
import org.apache.kafka.server.network.BrokerEndPoint
import org.apache.kafka.server.purgatory.{DelayedDeleteRecords, DelayedOperationPurgatory, DelayedRemoteListOffsets, DeleteRecordsPartitionStatus, ListOffsetsPartitionStatus, TopicPartitionOperationKey}
import org.apache.kafka.server.purgatory.{DelayedDeleteRecords, DelayedOperationPurgatory, DelayedProduce, DelayedRemoteListOffsets, DeleteRecordsPartitionStatus, ListOffsetsPartitionStatus, TopicPartitionOperationKey}
import org.apache.kafka.server.share.fetch.{DelayedShareFetchKey, DelayedShareFetchPartitionKey}
import org.apache.kafka.server.storage.log.{FetchParams, FetchPartitionData}
import org.apache.kafka.server.util.timer.{SystemTimer, TimerTask}
Expand All @@ -76,6 +77,7 @@ import java.util.{Collections, Optional, OptionalInt, OptionalLong}
import java.util.function.Consumer
import scala.collection.{Map, Seq, Set, immutable, mutable}
import scala.jdk.CollectionConverters._
import scala.jdk.FunctionConverters.enrichAsJavaConsumer
import scala.jdk.OptionConverters.{RichOption, RichOptional}

/*
Expand Down Expand Up @@ -732,7 +734,7 @@ class ReplicaManager(val config: KafkaConfig,
internalTopicsAllowed: Boolean,
origin: AppendOrigin,
entriesPerPartition: Map[TopicIdPartition, MemoryRecords],
responseCallback: Map[TopicIdPartition, PartitionResponse] => Unit,
responseCallback: util.Map[TopicIdPartition, PartitionResponse] => Unit,
recordValidationStatsCallback: Map[TopicIdPartition, RecordValidationStats] => Unit = _ => (),
requestLocal: RequestLocal = RequestLocal.noCaching,
verificationGuards: Map[TopicPartition, VerificationGuard] = Map.empty): Unit = {
Expand Down Expand Up @@ -846,8 +848,8 @@ class ReplicaManager(val config: KafkaConfig,

val preAppendPartitionResponses = buildProducePartitionStatus(errorResults).map { case (k, status) => k -> status.responseStatus }

def newResponseCallback(responses: Map[TopicIdPartition, PartitionResponse]): Unit = {
responseCallback(preAppendPartitionResponses ++ responses)
def newResponseCallback(responses: util.Map[TopicIdPartition, PartitionResponse]): Unit = {
responseCallback(preAppendPartitionResponses ++ responses.asScala)
}

appendRecords(
Expand Down Expand Up @@ -922,7 +924,7 @@ class ReplicaManager(val config: KafkaConfig,
results: Map[TopicIdPartition, LogAppendResult]
): Map[TopicIdPartition, ProducePartitionStatus] = {
results.map { case (topicIdPartition, result) =>
topicIdPartition -> ProducePartitionStatus(
topicIdPartition -> new ProducePartitionStatus(
result.info.lastOffset + 1, // required offset
new PartitionResponse(
result.error,
Expand Down Expand Up @@ -967,12 +969,33 @@ 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)

// Updates the status of a produce partition based on the current state.
// Please refer to the documentation in `DelayedProduce#tryComplete` for
// a comprehensive description of Case A, Case B and Case C.
def delegate(tp: TopicPartition, status: ProducePartitionStatus) : Unit = {
val (hasEnough, error) = getPartitionOrError(tp) match {
case Left(err) =>
// Case A
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What does this comment mean?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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 ?

(false, err)

case Right(partition) =>
partition.checkEnoughReplicasReachOffset(status.requiredOffset)
}

// Case B || C.1 || C.2
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ditto

if (error != Errors.NONE || hasEnough) {
status.setAcksPending(false)
status.responseStatus.error = error
}
}

val delayedProduce = new DelayedProduce(timeoutMs, produceMetadata, delegate, responseCallback.asJava)

// create a list of (topic, partition) pairs to use as keys for this delayed produce operation
val producerRequestKeys = entriesPerPartition.keys.map(new TopicPartitionOperationKey(_)).toList
Expand All @@ -983,23 +1006,25 @@ class ReplicaManager(val config: KafkaConfig,
delayedProducePurgatory.tryCompleteElseWatch(delayedProduce, producerRequestKeys.asJava)
} else {
// we can respond immediately
val produceResponseStatus = initialProduceStatus.map { case (k, status) => k -> status.responseStatus }
val produceResponseStatus = new util.HashMap[TopicIdPartition, PartitionResponse]
initialProduceStatus.foreach { case (k, status) => k -> produceResponseStatus.put(k, status.responseStatus) }
Comment on lines +1009 to +1010
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
val produceResponseStatus = new util.HashMap[TopicIdPartition, PartitionResponse]
initialProduceStatus.foreach { case (k, status) => k -> produceResponseStatus.put(k, status.responseStatus) }
val produceResponseStatus = initialProduceStatus.map { case (k, status) =>
k -> status.responseStatus
}.asJava

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think it's ok to use HashMap instead of asJava conversion.

responseCallback(produceResponseStatus)
}
}

private def sendInvalidRequiredAcksResponse(
entries: Map[TopicIdPartition, MemoryRecords],
responseCallback: Map[TopicIdPartition, PartitionResponse] => Unit): Unit = {
responseCallback: util.Map[TopicIdPartition, PartitionResponse] => Unit): Unit = {
// If required.acks is outside accepted range, something is wrong with the client
// Just return an error and don't handle the request at all
val responseStatus = entries.map { case (topicIdPartition, _) =>
topicIdPartition -> new PartitionResponse(
Errors.INVALID_REQUIRED_ACKS,
LogAppendInfo.UNKNOWN_LOG_APPEND_INFO.firstOffset,
RecordBatch.NO_TIMESTAMP,
LogAppendInfo.UNKNOWN_LOG_APPEND_INFO.logStartOffset
)
val responseStatus = new util.HashMap[TopicIdPartition, PartitionResponse]
entries.foreach { case(topicIdPartition, _) =>
responseStatus.put(topicIdPartition, new PartitionResponse(
Errors.INVALID_REQUIRED_ACKS,
LogAppendInfo.UNKNOWN_LOG_APPEND_INFO.firstOffset,
RecordBatch.NO_TIMESTAMP,
LogAppendInfo.UNKNOWN_LOG_APPEND_INFO.logStartOffset)
)
}
responseCallback(responseStatus)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -271,10 +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 = {
def appendCallback(responses: JMap[TopicIdPartition, PartitionResponse]): Unit = {
val response = responses.get(partition)
assertTrue(response.isDefined)
result.fire(response.get)
assertNotNull(response)
result.fire(response)
}

replicaManager.appendRecords(
Expand Down
2 changes: 1 addition & 1 deletion core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@ import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
import org.apache.kafka.coordinator.transaction.TransactionLogConfig
import org.apache.kafka.server.common.{ControllerRequestCompletionHandler, NodeToControllerChannelManager, RequestLocal}
import org.apache.kafka.server.metrics.KafkaYammerMetrics
import org.apache.kafka.server.purgatory.{DelayedDeleteRecords, DelayedOperationPurgatory, TopicPartitionOperationKey}
import org.apache.kafka.server.purgatory.{DelayedDeleteRecords, DelayedOperationPurgatory, DelayedProduce, TopicPartitionOperationKey}
import org.apache.kafka.server.share.fetch.DelayedShareFetchPartitionKey
import org.apache.kafka.server.storage.log.{FetchIsolation, FetchParams, UnexpectedAppendOffsetException}
import org.apache.kafka.server.util.{KafkaScheduler, MockTime}
Expand Down
Loading