KAFKA-19322 Remove the DelayedOperation constructor that accepts an external lock (#19798)
CI / build (push) Waiting to run Details

Remove the DelayedOperation constructor that accepts an external lock.

According to this [PR](https://github.com/apache/kafka/pull/19759).

Reviewers: Ken Huang <s7133700@gmail.com>, PoAn Yang
 <payang@apache.org>, TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
This commit is contained in:
YuChia Ma 2025-05-27 01:05:41 +08:00 committed by GitHub
parent 0600abdde3
commit 6e380fbbcc
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
7 changed files with 6 additions and 37 deletions

View File

@ -172,7 +172,7 @@ public class DelayedShareFetch extends DelayedOperation {
Uuid fetchId, Uuid fetchId,
long remoteFetchMaxWaitMs long remoteFetchMaxWaitMs
) { ) {
super(shareFetch.fetchParams().maxWaitMs, Optional.empty()); super(shareFetch.fetchParams().maxWaitMs);
this.shareFetch = shareFetch; this.shareFetch = shareFetch;
this.replicaManager = replicaManager; this.replicaManager = replicaManager;
this.partitionsAcquired = new LinkedHashMap<>(); this.partitionsAcquired = new LinkedHashMap<>();

View File

@ -18,7 +18,6 @@
package kafka.server package kafka.server
import java.util.concurrent.{ConcurrentHashMap, TimeUnit} import java.util.concurrent.{ConcurrentHashMap, TimeUnit}
import java.util.concurrent.locks.Lock
import com.typesafe.scalalogging.Logger import com.typesafe.scalalogging.Logger
import com.yammer.metrics.core.Meter import com.yammer.metrics.core.Meter
import kafka.utils.Logging import kafka.utils.Logging
@ -30,7 +29,6 @@ import org.apache.kafka.server.purgatory.DelayedOperation
import scala.collection._ import scala.collection._
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
import scala.jdk.OptionConverters.RichOption
case class ProducePartitionStatus(requiredOffset: Long, responseStatus: PartitionResponse) { case class ProducePartitionStatus(requiredOffset: Long, responseStatus: PartitionResponse) {
@volatile var acksPending = false @volatile var acksPending = false
@ -59,9 +57,8 @@ object DelayedProduce {
class DelayedProduce(delayMs: Long, class DelayedProduce(delayMs: Long,
produceMetadata: ProduceMetadata, produceMetadata: ProduceMetadata,
replicaManager: ReplicaManager, replicaManager: ReplicaManager,
responseCallback: Map[TopicIdPartition, PartitionResponse] => Unit, responseCallback: Map[TopicIdPartition, PartitionResponse] => Unit)
lockOpt: Option[Lock]) extends DelayedOperation(delayMs) with Logging {
extends DelayedOperation(delayMs, lockOpt.toJava) with Logging {
override lazy val logger: Logger = DelayedProduce.logger override lazy val logger: Logger = DelayedProduce.logger

View File

@ -71,7 +71,6 @@ import java.lang.{Long => JLong}
import java.nio.file.{Files, Paths} import java.nio.file.{Files, Paths}
import java.util import java.util
import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.atomic.AtomicBoolean
import java.util.concurrent.locks.Lock
import java.util.concurrent.{CompletableFuture, ConcurrentHashMap, Future, RejectedExecutionException, TimeUnit} import java.util.concurrent.{CompletableFuture, ConcurrentHashMap, Future, RejectedExecutionException, TimeUnit}
import java.util.{Collections, Optional, OptionalInt, OptionalLong} import java.util.{Collections, Optional, OptionalInt, OptionalLong}
import java.util.function.Consumer import java.util.function.Consumer
@ -723,7 +722,6 @@ class ReplicaManager(val config: KafkaConfig,
* If topic partition contains Uuid.ZERO_UUID as topicId the method * If topic partition contains Uuid.ZERO_UUID as topicId the method
* will fall back to the old behaviour and rely on topic name. * will fall back to the old behaviour and rely on topic name.
* @param responseCallback callback for sending the response * @param responseCallback callback for sending the response
* @param delayedProduceLock lock for the delayed actions
* @param recordValidationStatsCallback callback for updating stats on record conversions * @param recordValidationStatsCallback callback for updating stats on record conversions
* @param requestLocal container for the stateful instances scoped to this request -- this must correspond to the * @param requestLocal container for the stateful instances scoped to this request -- this must correspond to the
* thread calling this method * thread calling this method
@ -735,7 +733,6 @@ class ReplicaManager(val config: KafkaConfig,
origin: AppendOrigin, origin: AppendOrigin,
entriesPerPartition: Map[TopicIdPartition, MemoryRecords], entriesPerPartition: Map[TopicIdPartition, MemoryRecords],
responseCallback: Map[TopicIdPartition, PartitionResponse] => Unit, responseCallback: Map[TopicIdPartition, PartitionResponse] => Unit,
delayedProduceLock: Option[Lock] = None,
recordValidationStatsCallback: Map[TopicIdPartition, RecordValidationStats] => Unit = _ => (), recordValidationStatsCallback: Map[TopicIdPartition, RecordValidationStats] => Unit = _ => (),
requestLocal: RequestLocal = RequestLocal.noCaching, requestLocal: RequestLocal = RequestLocal.noCaching,
verificationGuards: Map[TopicPartition, VerificationGuard] = Map.empty): Unit = { verificationGuards: Map[TopicPartition, VerificationGuard] = Map.empty): Unit = {
@ -762,7 +759,6 @@ class ReplicaManager(val config: KafkaConfig,
maybeAddDelayedProduce( maybeAddDelayedProduce(
requiredAcks, requiredAcks,
delayedProduceLock,
timeout, timeout,
entriesPerPartition, entriesPerPartition,
localProduceResults, localProduceResults,
@ -967,7 +963,6 @@ class ReplicaManager(val config: KafkaConfig,
private def maybeAddDelayedProduce( private def maybeAddDelayedProduce(
requiredAcks: Short, requiredAcks: Short,
delayedProduceLock: Option[Lock],
timeoutMs: Long, timeoutMs: Long,
entriesPerPartition: Map[TopicIdPartition, MemoryRecords], entriesPerPartition: Map[TopicIdPartition, MemoryRecords],
initialAppendResults: Map[TopicIdPartition, LogAppendResult], initialAppendResults: Map[TopicIdPartition, LogAppendResult],
@ -977,7 +972,7 @@ class ReplicaManager(val config: KafkaConfig,
if (delayedProduceRequestRequired(requiredAcks, entriesPerPartition, initialAppendResults)) { if (delayedProduceRequestRequired(requiredAcks, entriesPerPartition, initialAppendResults)) {
// create delayed produce operation // create delayed produce operation
val produceMetadata = ProduceMetadata(requiredAcks, initialProduceStatus) val produceMetadata = ProduceMetadata(requiredAcks, initialProduceStatus)
val delayedProduce = new DelayedProduce(timeoutMs, produceMetadata, this, responseCallback, delayedProduceLock) val delayedProduce = new DelayedProduce(timeoutMs, produceMetadata, this, responseCallback)
// create a list of (topic, partition) pairs to use as keys for this delayed produce operation // create a list of (topic, partition) pairs to use as keys for this delayed produce operation
val producerRequestKeys = entriesPerPartition.keys.map(new TopicPartitionOperationKey(_)).toList val producerRequestKeys = entriesPerPartition.keys.map(new TopicPartitionOperationKey(_)).toList

View File

@ -20,7 +20,6 @@ package kafka.coordinator
import java.util.concurrent.{ConcurrentHashMap, ExecutorService, Executors} import java.util.concurrent.{ConcurrentHashMap, ExecutorService, Executors}
import java.util.{Collections, Random} import java.util.{Collections, Random}
import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicInteger
import java.util.concurrent.locks.Lock
import kafka.coordinator.AbstractCoordinatorConcurrencyTest._ import kafka.coordinator.AbstractCoordinatorConcurrencyTest._
import kafka.cluster.Partition import kafka.cluster.Partition
import kafka.log.LogManager import kafka.log.LogManager
@ -216,7 +215,6 @@ object AbstractCoordinatorConcurrencyTest {
origin: AppendOrigin, origin: AppendOrigin,
entriesPerPartition: Map[TopicIdPartition, MemoryRecords], entriesPerPartition: Map[TopicIdPartition, MemoryRecords],
responseCallback: Map[TopicIdPartition, PartitionResponse] => Unit, responseCallback: Map[TopicIdPartition, PartitionResponse] => Unit,
delayedProduceLock: Option[Lock] = None,
processingStatsCallback: Map[TopicIdPartition, RecordValidationStats] => Unit = _ => (), processingStatsCallback: Map[TopicIdPartition, RecordValidationStats] => Unit = _ => (),
requestLocal: RequestLocal = RequestLocal.noCaching, requestLocal: RequestLocal = RequestLocal.noCaching,
verificationGuards: Map[TopicPartition, VerificationGuard] = Map.empty): Unit = { verificationGuards: Map[TopicPartition, VerificationGuard] = Map.empty): Unit = {
@ -227,7 +225,7 @@ object AbstractCoordinatorConcurrencyTest {
case (tp, _) => case (tp, _) =>
(tp, ProducePartitionStatus(0L, new PartitionResponse(Errors.NONE, 0L, RecordBatch.NO_TIMESTAMP, 0L))) (tp, ProducePartitionStatus(0L, new PartitionResponse(Errors.NONE, 0L, RecordBatch.NO_TIMESTAMP, 0L)))
}) })
val delayedProduce = new DelayedProduce(5, produceMetadata, this, responseCallback, delayedProduceLock) { val delayedProduce = new DelayedProduce(5, produceMetadata, this, responseCallback) {
// Complete produce requests after a few attempts to trigger delayed produce from different threads // Complete produce requests after a few attempts to trigger delayed produce from different threads
val completeAttempts = new AtomicInteger val completeAttempts = new AtomicInteger
override def tryComplete(): Boolean = { override def tryComplete(): Boolean = {

View File

@ -19,7 +19,6 @@ package kafka.coordinator.transaction
import java.lang.management.ManagementFactory import java.lang.management.ManagementFactory
import java.nio.ByteBuffer import java.nio.ByteBuffer
import java.util.concurrent.{ConcurrentHashMap, CountDownLatch} import java.util.concurrent.{ConcurrentHashMap, CountDownLatch}
import java.util.concurrent.locks.ReentrantLock
import javax.management.ObjectName import javax.management.ObjectName
import kafka.server.ReplicaManager import kafka.server.ReplicaManager
import kafka.utils.TestUtils import kafka.utils.TestUtils
@ -758,7 +757,6 @@ class TransactionStateManagerTest {
ArgumentMatchers.eq(AppendOrigin.COORDINATOR), ArgumentMatchers.eq(AppendOrigin.COORDINATOR),
any(), any(),
any(), any(),
any[Option[ReentrantLock]],
any(), any(),
any(), any(),
any() any()
@ -803,7 +801,6 @@ class TransactionStateManagerTest {
ArgumentMatchers.eq(AppendOrigin.COORDINATOR), ArgumentMatchers.eq(AppendOrigin.COORDINATOR),
any(), any(),
any(), any(),
any[Option[ReentrantLock]],
any(), any(),
any(), any(),
any() any()
@ -847,7 +844,6 @@ class TransactionStateManagerTest {
ArgumentMatchers.eq(AppendOrigin.COORDINATOR), ArgumentMatchers.eq(AppendOrigin.COORDINATOR),
any(), any(),
any(), any(),
any[Option[ReentrantLock]],
any(), any(),
any(), any(),
any()) any())
@ -901,7 +897,6 @@ class TransactionStateManagerTest {
ArgumentMatchers.eq(AppendOrigin.COORDINATOR), ArgumentMatchers.eq(AppendOrigin.COORDINATOR),
any(), any(),
any(), any(),
any[Option[ReentrantLock]],
any(), any(),
any(), any(),
any() any()
@ -1118,7 +1113,6 @@ class TransactionStateManagerTest {
ArgumentMatchers.eq(AppendOrigin.COORDINATOR), ArgumentMatchers.eq(AppendOrigin.COORDINATOR),
recordsCapture.capture(), recordsCapture.capture(),
callbackCapture.capture(), callbackCapture.capture(),
any[Option[ReentrantLock]],
any(), any(),
any(), any(),
any() any()
@ -1271,7 +1265,6 @@ class TransactionStateManagerTest {
origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR), origin = ArgumentMatchers.eq(AppendOrigin.COORDINATOR),
any[Map[TopicIdPartition, MemoryRecords]], any[Map[TopicIdPartition, MemoryRecords]],
capturedArgument.capture(), capturedArgument.capture(),
any[Option[ReentrantLock]],
any(), any(),
any(), any(),
any() any()

View File

@ -2830,7 +2830,6 @@ class KafkaApisTest extends Logging {
any(), any(),
responseCallback.capture(), responseCallback.capture(),
any(), any(),
any(),
ArgumentMatchers.eq(requestLocal), ArgumentMatchers.eq(requestLocal),
any() 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))))
@ -2886,7 +2885,6 @@ class KafkaApisTest extends Logging {
any(), any(),
any(), any(),
any(), any(),
any(),
ArgumentMatchers.eq(requestLocal), ArgumentMatchers.eq(requestLocal),
any()) any())
} }
@ -2965,7 +2963,6 @@ class KafkaApisTest extends Logging {
entriesPerPartition.capture(), entriesPerPartition.capture(),
responseCallback.capture(), responseCallback.capture(),
any(), any(),
any(),
ArgumentMatchers.eq(RequestLocal.noCaching), ArgumentMatchers.eq(RequestLocal.noCaching),
any() any()
)).thenAnswer { _ => )).thenAnswer { _ =>

View File

@ -18,8 +18,6 @@ package org.apache.kafka.server.purgatory;
import org.apache.kafka.server.util.timer.TimerTask; import org.apache.kafka.server.util.timer.TimerTask;
import java.util.Optional;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantLock;
/** /**
@ -41,19 +39,10 @@ public abstract class DelayedOperation extends TimerTask {
private volatile boolean completed = false; private volatile boolean completed = false;
protected final Lock lock; protected final ReentrantLock lock = new ReentrantLock();
public DelayedOperation(long delayMs, Optional<Lock> lockOpt) {
this(delayMs, lockOpt.orElse(new ReentrantLock()));
}
public DelayedOperation(long delayMs) { public DelayedOperation(long delayMs) {
this(delayMs, new ReentrantLock());
}
public DelayedOperation(long delayMs, Lock lock) {
super(delayMs); super(delayMs);
this.lock = lock;
} }
/* /*