mirror of https://github.com/apache/kafka.git
				
				
				
			KAFKA-19322 Remove the DelayedOperation constructor that accepts an external lock (#19798)
	
		
			
	
		
	
	
		
			
				
	
				CI / build (push) Waiting to run
				
					Details
				
			
		
	
				
					
				
			
				
	
				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:
		
							parent
							
								
									0600abdde3
								
							
						
					
					
						commit
						6e380fbbcc
					
				|  | @ -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<>(); | ||||||
|  |  | ||||||
|  | @ -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 | ||||||
| 
 | 
 | ||||||
|  |  | ||||||
|  | @ -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 | ||||||
|  |  | ||||||
|  | @ -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 = { | ||||||
|  |  | ||||||
|  | @ -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() | ||||||
|  |  | ||||||
|  | @ -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 { _ => | ||||||
|  |  | ||||||
|  | @ -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; |  | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|     /* |     /* | ||||||
|  |  | ||||||
		Loading…
	
		Reference in New Issue