mirror of https://github.com/apache/kafka.git
				
				
				
			KAFKA-17742: Move DelayedShareFetchPurgatory declaration to ReplicaManager (#17437)
Declare the delayed share fetch purgatory inside ReplicaManager along with the existing purgatories. Check the share fetch purgatory when a replica becomes the follower or a replica is deleted from a broker through ReplicaManager. Perform a checkAndComplete for share fetch when HWM is updated. Reviewers: Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>, Jun Rao <junrao@gmail.com>
This commit is contained in:
		
							parent
							
								
									6d39031958
								
							
						
					
					
						commit
						cb3b03377d
					
				|  | @ -33,6 +33,7 @@ import kafka.server.KafkaConfig; | |||
| import kafka.server.MetadataCache; | ||||
| import kafka.server.QuotaFactory.QuotaManagers; | ||||
| import kafka.server.ReplicaManager; | ||||
| import kafka.server.share.DelayedShareFetch; | ||||
| import kafka.zk.KafkaZkClient; | ||||
| 
 | ||||
| import org.apache.kafka.common.metrics.Metrics; | ||||
|  | @ -70,6 +71,7 @@ public class ReplicaManagerBuilder { | |||
|     private Optional<DelayedOperationPurgatory<DelayedElectLeader>> delayedElectLeaderPurgatory = Optional.empty(); | ||||
|     private Optional<DelayedOperationPurgatory<DelayedRemoteFetch>> delayedRemoteFetchPurgatory = Optional.empty(); | ||||
|     private Optional<DelayedOperationPurgatory<DelayedRemoteListOffsets>> delayedRemoteListOffsetsPurgatory = Optional.empty(); | ||||
|     private Optional<DelayedOperationPurgatory<DelayedShareFetch>> delayedShareFetchPurgatory = Optional.empty(); | ||||
|     private Optional<String> threadNamePrefix = Optional.empty(); | ||||
|     private Long brokerEpoch = -1L; | ||||
|     private Optional<AddPartitionsToTxnManager> addPartitionsToTxnManager = Optional.empty(); | ||||
|  | @ -215,6 +217,7 @@ public class ReplicaManagerBuilder { | |||
|                              OptionConverters.toScala(delayedElectLeaderPurgatory), | ||||
|                              OptionConverters.toScala(delayedRemoteFetchPurgatory), | ||||
|                              OptionConverters.toScala(delayedRemoteListOffsetsPurgatory), | ||||
|                              OptionConverters.toScala(delayedShareFetchPurgatory), | ||||
|                              OptionConverters.toScala(threadNamePrefix), | ||||
|                              () -> brokerEpoch, | ||||
|                              OptionConverters.toScala(addPartitionsToTxnManager), | ||||
|  |  | |||
|  | @ -135,8 +135,12 @@ public class DelayedShareFetch extends DelayedOperation { | |||
|             // then we should check if there is a pending share fetch request for the topic-partition and complete it. | ||||
|             // We add the action to delayed actions queue to avoid an infinite call stack, which could happen if | ||||
|             // we directly call delayedShareFetchPurgatory.checkAndComplete | ||||
|             sharePartitionManager.addPurgatoryCheckAndCompleteDelayedActionToActionQueue( | ||||
|                 topicPartitionData.keySet(), shareFetchData.groupId()); | ||||
|             replicaManager.addToActionQueue(() -> { | ||||
|                 topicPartitionData.keySet().forEach(topicIdPartition -> | ||||
|                     replicaManager.completeDelayedShareFetchRequest( | ||||
|                         new DelayedShareFetchGroupKey(shareFetchData.groupId(), topicIdPartition.topicId(), topicIdPartition.partition()))); | ||||
|                 return BoxedUnit.UNIT; | ||||
|             }); | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|  |  | |||
|  | @ -29,7 +29,7 @@ public class DelayedShareFetchPartitionKey implements  DelayedShareFetchKey, Del | |||
|     private final Uuid topicId; | ||||
|     private final int partition; | ||||
| 
 | ||||
|     DelayedShareFetchPartitionKey(Uuid topicId, int partition) { | ||||
|     public DelayedShareFetchPartitionKey(Uuid topicId, int partition) { | ||||
|         this.topicId = topicId; | ||||
|         this.partition = partition; | ||||
|     } | ||||
|  |  | |||
|  | @ -16,7 +16,7 @@ | |||
|  */ | ||||
| package kafka.server.share; | ||||
| 
 | ||||
| import kafka.server.DelayedOperationPurgatory; | ||||
| import kafka.server.ReplicaManager; | ||||
| 
 | ||||
| import org.apache.kafka.common.KafkaException; | ||||
| import org.apache.kafka.common.TopicIdPartition; | ||||
|  | @ -271,9 +271,10 @@ public class SharePartition { | |||
|     private SharePartitionState partitionState; | ||||
| 
 | ||||
|     /** | ||||
|      * The delayed share fetch purgatory is used to store the share fetch requests that could not be processed immediately. | ||||
|      * The replica manager is used to check to see if any delayed share fetch request can be completed because of data | ||||
|      * availability due to acquisition lock timeout. | ||||
|      */ | ||||
|     private final DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory; | ||||
|     private final ReplicaManager replicaManager; | ||||
| 
 | ||||
|     SharePartition( | ||||
|         String groupId, | ||||
|  | @ -284,7 +285,7 @@ public class SharePartition { | |||
|         Timer timer, | ||||
|         Time time, | ||||
|         Persister persister, | ||||
|         DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory, | ||||
|         ReplicaManager replicaManager, | ||||
|         GroupConfigManager groupConfigManager | ||||
|     ) { | ||||
|         this.groupId = groupId; | ||||
|  | @ -300,7 +301,7 @@ public class SharePartition { | |||
|         this.time = time; | ||||
|         this.persister = persister; | ||||
|         this.partitionState = SharePartitionState.EMPTY; | ||||
|         this.delayedShareFetchPurgatory = delayedShareFetchPurgatory; | ||||
|         this.replicaManager = replicaManager; | ||||
|         this.groupConfigManager = groupConfigManager; | ||||
|     } | ||||
| 
 | ||||
|  | @ -1810,7 +1811,7 @@ public class SharePartition { | |||
|                     // If we have an acquisition lock timeout for a share-partition, then we should check if | ||||
|                     // there is a pending share fetch request for the share-partition and complete it. | ||||
|                     DelayedShareFetchKey delayedShareFetchKey = new DelayedShareFetchGroupKey(groupId, topicIdPartition.topicId(), topicIdPartition.partition()); | ||||
|                     delayedShareFetchPurgatory.checkAndComplete(delayedShareFetchKey); | ||||
|                     replicaManager.completeDelayedShareFetchRequest(delayedShareFetchKey); | ||||
|                 }); | ||||
|             } | ||||
|         } finally { | ||||
|  |  | |||
|  | @ -16,8 +16,6 @@ | |||
|  */ | ||||
| package kafka.server.share; | ||||
| 
 | ||||
| import kafka.server.ActionQueue; | ||||
| import kafka.server.DelayedOperationPurgatory; | ||||
| import kafka.server.ReplicaManager; | ||||
| 
 | ||||
| import org.apache.kafka.clients.consumer.AcknowledgeType; | ||||
|  | @ -75,7 +73,6 @@ import java.util.concurrent.ConcurrentLinkedQueue; | |||
| import java.util.concurrent.atomic.AtomicBoolean; | ||||
| 
 | ||||
| import scala.jdk.javaapi.CollectionConverters; | ||||
| import scala.runtime.BoxedUnit; | ||||
| 
 | ||||
| /** | ||||
|  * The SharePartitionManager is responsible for managing the SharePartitions and ShareSessions. | ||||
|  | @ -151,16 +148,6 @@ public class SharePartitionManager implements AutoCloseable { | |||
|      */ | ||||
|     private final ShareGroupMetrics shareGroupMetrics; | ||||
| 
 | ||||
|     /** | ||||
|      * The delayed share fetch purgatory is used to store the share fetch requests that could not be processed immediately. | ||||
|      */ | ||||
|     private final DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory; | ||||
| 
 | ||||
|     /** | ||||
|      * The delayed actions queue is used to complete any pending delayed share fetch actions. | ||||
|      */ | ||||
|     private final ActionQueue delayedActionsQueue; | ||||
| 
 | ||||
|     public SharePartitionManager( | ||||
|         ReplicaManager replicaManager, | ||||
|         Time time, | ||||
|  | @ -168,9 +155,7 @@ public class SharePartitionManager implements AutoCloseable { | |||
|         int defaultRecordLockDurationMs, | ||||
|         int maxDeliveryCount, | ||||
|         int maxInFlightMessages, | ||||
|         int shareFetchPurgatoryPurgeIntervalRequests, | ||||
|         Persister persister, | ||||
|         ActionQueue delayedActionsQueue, | ||||
|         GroupConfigManager groupConfigManager, | ||||
|         Metrics metrics | ||||
|     ) { | ||||
|  | @ -181,9 +166,7 @@ public class SharePartitionManager implements AutoCloseable { | |||
|             defaultRecordLockDurationMs, | ||||
|             maxDeliveryCount, | ||||
|             maxInFlightMessages, | ||||
|             shareFetchPurgatoryPurgeIntervalRequests, | ||||
|             persister, | ||||
|             delayedActionsQueue, | ||||
|             groupConfigManager, | ||||
|             metrics | ||||
|         ); | ||||
|  | @ -197,9 +180,7 @@ public class SharePartitionManager implements AutoCloseable { | |||
|         int defaultRecordLockDurationMs, | ||||
|         int maxDeliveryCount, | ||||
|         int maxInFlightMessages, | ||||
|         int shareFetchPurgatoryPurgeIntervalRequests, | ||||
|         Persister persister, | ||||
|         ActionQueue delayedActionsQueue, | ||||
|         GroupConfigManager groupConfigManager, | ||||
|         Metrics metrics | ||||
|     ) { | ||||
|  | @ -215,8 +196,6 @@ public class SharePartitionManager implements AutoCloseable { | |||
|         this.maxDeliveryCount = maxDeliveryCount; | ||||
|         this.maxInFlightMessages = maxInFlightMessages; | ||||
|         this.persister = persister; | ||||
|         this.delayedShareFetchPurgatory = new DelayedOperationPurgatory<>("ShareFetch", this.timer, this.replicaManager.localBrokerId(), shareFetchPurgatoryPurgeIntervalRequests, true, true); | ||||
|         this.delayedActionsQueue = delayedActionsQueue; | ||||
|         this.groupConfigManager = groupConfigManager; | ||||
|         this.shareGroupMetrics = new ShareGroupMetrics(Objects.requireNonNull(metrics), time); | ||||
|     } | ||||
|  | @ -234,8 +213,6 @@ public class SharePartitionManager implements AutoCloseable { | |||
|             int maxDeliveryCount, | ||||
|             int maxInFlightMessages, | ||||
|             Persister persister, | ||||
|             DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory, | ||||
|             ActionQueue delayedActionsQueue, | ||||
|             GroupConfigManager groupConfigManager, | ||||
|             Metrics metrics | ||||
|     ) { | ||||
|  | @ -250,8 +227,6 @@ public class SharePartitionManager implements AutoCloseable { | |||
|         this.maxDeliveryCount = maxDeliveryCount; | ||||
|         this.maxInFlightMessages = maxInFlightMessages; | ||||
|         this.persister = persister; | ||||
|         this.delayedShareFetchPurgatory = delayedShareFetchPurgatory; | ||||
|         this.delayedActionsQueue = delayedActionsQueue; | ||||
|         this.groupConfigManager = groupConfigManager; | ||||
|         this.shareGroupMetrics = new ShareGroupMetrics(Objects.requireNonNull(metrics), time); | ||||
|     } | ||||
|  | @ -319,7 +294,7 @@ public class SharePartitionManager implements AutoCloseable { | |||
|                 // If we have an acknowledgement completed for a topic-partition, then we should check if | ||||
|                 // there is a pending share fetch request for the topic-partition and complete it. | ||||
|                 DelayedShareFetchKey delayedShareFetchKey = new DelayedShareFetchGroupKey(groupId, topicIdPartition.topicId(), topicIdPartition.partition()); | ||||
|                 delayedShareFetchPurgatory.checkAndComplete(delayedShareFetchKey); | ||||
|                 replicaManager.completeDelayedShareFetchRequest(delayedShareFetchKey); | ||||
| 
 | ||||
|                 futures.put(topicIdPartition, future); | ||||
|             } else { | ||||
|  | @ -338,15 +313,6 @@ public class SharePartitionManager implements AutoCloseable { | |||
|         }); | ||||
|     } | ||||
| 
 | ||||
|     void addPurgatoryCheckAndCompleteDelayedActionToActionQueue(Set<TopicIdPartition> topicIdPartitions, String groupId) { | ||||
|         delayedActionsQueue.add(() -> { | ||||
|             topicIdPartitions.forEach(topicIdPartition -> | ||||
|                 delayedShareFetchPurgatory.checkAndComplete( | ||||
|                     new DelayedShareFetchGroupKey(groupId, topicIdPartition.topicId(), topicIdPartition.partition()))); | ||||
|             return BoxedUnit.UNIT; | ||||
|         }); | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|      * The release session method is used to release the session for the memberId of respective group. | ||||
|      * The method post removing session also releases acquired records for the respective member. | ||||
|  | @ -397,7 +363,7 @@ public class SharePartitionManager implements AutoCloseable { | |||
|                 // If we have a release acquired request completed for a topic-partition, then we should check if | ||||
|                 // there is a pending share fetch request for the topic-partition and complete it. | ||||
|                 DelayedShareFetchKey delayedShareFetchKey = new DelayedShareFetchGroupKey(groupId, topicIdPartition.topicId(), topicIdPartition.partition()); | ||||
|                 delayedShareFetchPurgatory.checkAndComplete(delayedShareFetchKey); | ||||
|                 replicaManager.completeDelayedShareFetchRequest(delayedShareFetchKey); | ||||
| 
 | ||||
|                 futuresMap.put(topicIdPartition, future); | ||||
|             } | ||||
|  | @ -557,13 +523,12 @@ public class SharePartitionManager implements AutoCloseable { | |||
|     // Add the share fetch request to the delayed share fetch purgatory to process the fetch request if it can be | ||||
|     // completed else watch until it can be completed/timeout. | ||||
|     private void addDelayedShareFetch(DelayedShareFetch delayedShareFetch, Set<DelayedShareFetchKey> keys) { | ||||
|         delayedShareFetchPurgatory.tryCompleteElseWatch(delayedShareFetch, | ||||
|             CollectionConverters.asScala(keys).toSeq().indices()); | ||||
|         replicaManager.addDelayedShareFetchRequest(delayedShareFetch, | ||||
|             CollectionConverters.asScala(keys).toSeq()); | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public void close() throws Exception { | ||||
|         this.delayedShareFetchPurgatory.shutdown(); | ||||
|         this.timer.close(); | ||||
|         this.persister.stop(); | ||||
|         if (!fetchQueue.isEmpty()) { | ||||
|  | @ -676,7 +641,7 @@ public class SharePartitionManager implements AutoCloseable { | |||
|                             timer, | ||||
|                             time, | ||||
|                             persister, | ||||
|                             delayedShareFetchPurgatory, | ||||
|                             replicaManager, | ||||
|                             groupConfigManager | ||||
|                     ); | ||||
|                     this.shareGroupMetrics.partitionLoadTime(start); | ||||
|  |  | |||
|  | @ -25,6 +25,7 @@ import kafka.log._ | |||
| import kafka.log.remote.RemoteLogManager | ||||
| import kafka.server._ | ||||
| import kafka.server.metadata.{KRaftMetadataCache, ZkMetadataCache} | ||||
| import kafka.server.share.{DelayedShareFetch, DelayedShareFetchPartitionKey} | ||||
| import kafka.utils.CoreUtils.{inReadLock, inWriteLock} | ||||
| import kafka.utils._ | ||||
| import kafka.zookeeper.ZooKeeperClientException | ||||
|  | @ -87,16 +88,20 @@ trait AlterPartitionListener { | |||
|   def markFailed(): Unit | ||||
| } | ||||
| 
 | ||||
| class DelayedOperations(topicPartition: TopicPartition, | ||||
| class DelayedOperations(topicId: Option[Uuid], | ||||
|                         topicPartition: TopicPartition, | ||||
|                         produce: DelayedOperationPurgatory[DelayedProduce], | ||||
|                         fetch: DelayedOperationPurgatory[DelayedFetch], | ||||
|                         deleteRecords: DelayedOperationPurgatory[DelayedDeleteRecords]) { | ||||
|                         deleteRecords: DelayedOperationPurgatory[DelayedDeleteRecords], | ||||
|                         shareFetch: DelayedOperationPurgatory[DelayedShareFetch]) { | ||||
| 
 | ||||
|   def checkAndCompleteAll(): Unit = { | ||||
|     val requestKey = TopicPartitionOperationKey(topicPartition) | ||||
|     CoreUtils.swallow(() -> fetch.checkAndComplete(requestKey), fetch, Level.ERROR) | ||||
|     CoreUtils.swallow(() -> produce.checkAndComplete(requestKey), produce, Level.ERROR) | ||||
|     CoreUtils.swallow(() -> deleteRecords.checkAndComplete(requestKey), deleteRecords, Level.ERROR) | ||||
|     if (topicId.isDefined) CoreUtils.swallow(() -> shareFetch.checkAndComplete(new DelayedShareFetchPartitionKey( | ||||
|       topicId.get, topicPartition.partition())), shareFetch, Level.ERROR) | ||||
|   } | ||||
| 
 | ||||
|   def numDelayedDelete: Int = deleteRecords.numDelayed | ||||
|  | @ -132,10 +137,12 @@ object Partition { | |||
|     } | ||||
| 
 | ||||
|     val delayedOperations = new DelayedOperations( | ||||
|       topicId, | ||||
|       topicPartition, | ||||
|       replicaManager.delayedProducePurgatory, | ||||
|       replicaManager.delayedFetchPurgatory, | ||||
|       replicaManager.delayedDeleteRecordsPurgatory) | ||||
|       replicaManager.delayedDeleteRecordsPurgatory, | ||||
|       replicaManager.delayedShareFetchPurgatory) | ||||
| 
 | ||||
|     new Partition(topicPartition, | ||||
|       _topicId = topicId, | ||||
|  |  | |||
|  | @ -433,9 +433,7 @@ class BrokerServer( | |||
|         config.shareGroupConfig.shareGroupRecordLockDurationMs, | ||||
|         config.shareGroupConfig.shareGroupDeliveryCountLimit, | ||||
|         config.shareGroupConfig.shareGroupPartitionMaxRecordLocks, | ||||
|         config.shareGroupConfig.shareFetchPurgatoryPurgeIntervalRequests, | ||||
|         persister, | ||||
|         defaultActionQueue, | ||||
|         groupConfigManager, | ||||
|         new Metrics() | ||||
|       ) | ||||
|  |  | |||
|  | @ -25,6 +25,7 @@ import kafka.server.HostedPartition.Online | |||
| import kafka.server.QuotaFactory.QuotaManagers | ||||
| import kafka.server.ReplicaManager.{AtMinIsrPartitionCountMetricName, FailedIsrUpdatesPerSecMetricName, IsrExpandsPerSecMetricName, IsrShrinksPerSecMetricName, LeaderCountMetricName, OfflineReplicaCountMetricName, PartitionCountMetricName, PartitionsWithLateTransactionsCountMetricName, ProducerIdCountMetricName, ReassigningPartitionsMetricName, UnderMinIsrPartitionCountMetricName, UnderReplicatedPartitionsMetricName, createLogReadResult, isListOffsetsTimestampUnsupported} | ||||
| import kafka.server.metadata.ZkMetadataCache | ||||
| import kafka.server.share.{DelayedShareFetch, DelayedShareFetchKey, DelayedShareFetchPartitionKey} | ||||
| import kafka.utils._ | ||||
| import kafka.zk.KafkaZkClient | ||||
| import org.apache.kafka.common.errors._ | ||||
|  | @ -55,7 +56,7 @@ import org.apache.kafka.image.{LocalReplicaChanges, MetadataImage, TopicsDelta} | |||
| import org.apache.kafka.metadata.LeaderAndIsr | ||||
| import org.apache.kafka.metadata.LeaderConstants.NO_LEADER | ||||
| import org.apache.kafka.server.common | ||||
| import org.apache.kafka.server.common.{DirectoryEventHandler, RequestLocal} | ||||
| import org.apache.kafka.server.common.{DirectoryEventHandler, RequestLocal, TopicOptionalIdPartition} | ||||
| import org.apache.kafka.server.common.MetadataVersion._ | ||||
| import org.apache.kafka.server.metrics.KafkaMetricsGroup | ||||
| import org.apache.kafka.server.storage.log.{FetchParams, FetchPartitionData} | ||||
|  | @ -286,6 +287,7 @@ class ReplicaManager(val config: KafkaConfig, | |||
|                      delayedElectLeaderPurgatoryParam: Option[DelayedOperationPurgatory[DelayedElectLeader]] = None, | ||||
|                      delayedRemoteFetchPurgatoryParam: Option[DelayedOperationPurgatory[DelayedRemoteFetch]] = None, | ||||
|                      delayedRemoteListOffsetsPurgatoryParam: Option[DelayedOperationPurgatory[DelayedRemoteListOffsets]] = None, | ||||
|                      delayedShareFetchPurgatoryParam: Option[DelayedOperationPurgatory[DelayedShareFetch]] = None, | ||||
|                      threadNamePrefix: Option[String] = None, | ||||
|                      val brokerEpochSupplier: () => Long = () => -1, | ||||
|                      addPartitionsToTxnManager: Option[AddPartitionsToTxnManager] = None, | ||||
|  | @ -315,6 +317,10 @@ class ReplicaManager(val config: KafkaConfig, | |||
|   val delayedRemoteListOffsetsPurgatory = delayedRemoteListOffsetsPurgatoryParam.getOrElse( | ||||
|     DelayedOperationPurgatory[DelayedRemoteListOffsets]( | ||||
|       purgatoryName = "RemoteListOffsets", brokerId = config.brokerId)) | ||||
|   val delayedShareFetchPurgatory = delayedShareFetchPurgatoryParam.getOrElse( | ||||
|     DelayedOperationPurgatory[DelayedShareFetch]( | ||||
|       purgatoryName = "ShareFetch", brokerId = config.brokerId, | ||||
|       purgeInterval = config.shareGroupConfig.shareFetchPurgatoryPurgeIntervalRequests)) | ||||
| 
 | ||||
|   /* epoch of the controller that last changed the leader */ | ||||
|   @volatile private[server] var controllerEpoch: Int = KafkaController.InitialControllerEpoch | ||||
|  | @ -463,12 +469,14 @@ class ReplicaManager(val config: KafkaConfig, | |||
|     }) | ||||
|   } | ||||
| 
 | ||||
|   private def completeDelayedOperationsWhenNotPartitionLeader(topicPartition: TopicPartition): Unit = { | ||||
|   private def completeDelayedOperationsWhenNotPartitionLeader(topicPartition: TopicPartition, topicId: Option[Uuid]): Unit = { | ||||
|     val topicPartitionOperationKey = TopicPartitionOperationKey(topicPartition) | ||||
|     delayedProducePurgatory.checkAndComplete(topicPartitionOperationKey) | ||||
|     delayedFetchPurgatory.checkAndComplete(topicPartitionOperationKey) | ||||
|     delayedRemoteFetchPurgatory.checkAndComplete(topicPartitionOperationKey) | ||||
|     delayedRemoteListOffsetsPurgatory.checkAndComplete(topicPartitionOperationKey) | ||||
|     if (topicId.isDefined) delayedShareFetchPurgatory.checkAndComplete( | ||||
|       new DelayedShareFetchPartitionKey(topicId.get, topicPartition.partition())) | ||||
|   } | ||||
| 
 | ||||
|   /** | ||||
|  | @ -480,6 +488,27 @@ class ReplicaManager(val config: KafkaConfig, | |||
|     topicPartitions.foreach(tp => delayedFetchPurgatory.checkAndComplete(TopicPartitionOperationKey(tp))) | ||||
|   } | ||||
| 
 | ||||
|   /** | ||||
|    * Complete any delayed share fetch requests that have been unblocked since new data is available from the leader | ||||
|    * for one of the partitions. This could happen due to acknowledgements, acquisition lock timeout of records, partition | ||||
|    * locks getting freed and release of acquired records due to share session close. | ||||
|    * @param delayedShareFetchKey The key corresponding to which the share fetch request has been stored in the purgatory | ||||
|    */ | ||||
|   private[server] def completeDelayedShareFetchRequest(delayedShareFetchKey: DelayedShareFetchKey): Unit = { | ||||
|     delayedShareFetchPurgatory.checkAndComplete(delayedShareFetchKey) | ||||
|   } | ||||
| 
 | ||||
|   /** | ||||
|    * Add and watch a share fetch request in the delayed share fetch purgatory corresponding to a set of keys in case it cannot be | ||||
|    * completed instantaneously, otherwise complete it. | ||||
|    * @param delayedShareFetch Refers to the DelayedOperation over share fetch request | ||||
|    * @param delayedShareFetchKeys The keys corresponding to which the delayed share fetch request will be stored in the purgatory | ||||
|    */ | ||||
|   private[server] def addDelayedShareFetchRequest(delayedShareFetch: DelayedShareFetch, | ||||
|                                                   delayedShareFetchKeys : Seq[DelayedShareFetchKey]): Unit = { | ||||
|     delayedShareFetchPurgatory.tryCompleteElseWatch(delayedShareFetch, delayedShareFetchKeys) | ||||
|   } | ||||
| 
 | ||||
|   /** | ||||
|    * Registers the provided listener to the partition iff the partition is online. | ||||
|    */ | ||||
|  | @ -610,6 +639,7 @@ class ReplicaManager(val config: KafkaConfig, | |||
|     val partitionsToDelete = mutable.Set.empty[TopicPartition] | ||||
|     partitionsToStop.foreach { stopPartition => | ||||
|       val topicPartition = stopPartition.topicPartition | ||||
|       var topicId: Option[Uuid] = None | ||||
|       if (stopPartition.deleteLocalLog) { | ||||
|         getPartition(topicPartition) match { | ||||
|           case hostedPartition: HostedPartition.Online => | ||||
|  | @ -618,6 +648,7 @@ class ReplicaManager(val config: KafkaConfig, | |||
|               // Logs are not deleted here. They are deleted in a single batch later on. | ||||
|               // This is done to avoid having to checkpoint for every deletions. | ||||
|               hostedPartition.partition.delete() | ||||
|               topicId = hostedPartition.partition.topicId | ||||
|             } | ||||
| 
 | ||||
|           case _ => | ||||
|  | @ -626,7 +657,7 @@ class ReplicaManager(val config: KafkaConfig, | |||
|       } | ||||
|       // If we were the leader, we may have some operations still waiting for completion. | ||||
|       // We force completion to prevent them from timing out. | ||||
|       completeDelayedOperationsWhenNotPartitionLeader(topicPartition) | ||||
|       completeDelayedOperationsWhenNotPartitionLeader(topicPartition, topicId) | ||||
|     } | ||||
| 
 | ||||
|     // Third delete the logs and checkpoint. | ||||
|  | @ -744,6 +775,8 @@ class ReplicaManager(val config: KafkaConfig, | |||
| 
 | ||||
|   def tryCompleteActions(): Unit = defaultActionQueue.tryCompleteActions() | ||||
| 
 | ||||
|   def addToActionQueue(action: () => Unit): Unit = defaultActionQueue.add(action) | ||||
| 
 | ||||
|   /** | ||||
|    * Append messages to leader replicas of the partition, and wait for them to be replicated to other replicas; | ||||
|    * the callback function will be triggered either when timeout or the required acks are satisfied; | ||||
|  | @ -783,13 +816,15 @@ class ReplicaManager(val config: KafkaConfig, | |||
|     } | ||||
| 
 | ||||
|     val sTime = time.milliseconds | ||||
|     val localProduceResults = appendToLocalLog(internalTopicsAllowed = internalTopicsAllowed, | ||||
|     val localProduceResultsWithTopicId = appendToLocalLog(internalTopicsAllowed = internalTopicsAllowed, | ||||
|       origin, entriesPerPartition, requiredAcks, requestLocal, verificationGuards.toMap) | ||||
|     debug("Produce to local log in %d ms".format(time.milliseconds - sTime)) | ||||
|     val localProduceResults : Map[TopicPartition, LogAppendResult] = localProduceResultsWithTopicId.map { | ||||
|       case(k, v) => (k.topicPartition, v)} | ||||
| 
 | ||||
|     val produceStatus = buildProducePartitionStatus(localProduceResults) | ||||
| 
 | ||||
|     addCompletePurgatoryAction(actionQueue, localProduceResults) | ||||
|     addCompletePurgatoryAction(actionQueue, localProduceResultsWithTopicId) | ||||
|     recordValidationStatsCallback(localProduceResults.map { case (k, v) => | ||||
|       k -> v.info.recordValidationStats | ||||
|     }) | ||||
|  | @ -940,17 +975,19 @@ class ReplicaManager(val config: KafkaConfig, | |||
| 
 | ||||
|   private def addCompletePurgatoryAction( | ||||
|     actionQueue: ActionQueue, | ||||
|     appendResults: Map[TopicPartition, LogAppendResult] | ||||
|     appendResults: Map[TopicOptionalIdPartition, LogAppendResult] | ||||
|   ): Unit = { | ||||
|     actionQueue.add { | ||||
|       () => appendResults.foreach { case (topicPartition, result) => | ||||
|         val requestKey = TopicPartitionOperationKey(topicPartition) | ||||
|       () => appendResults.foreach { case (topicOptionalIdPartition, result) => | ||||
|         val requestKey = TopicPartitionOperationKey(topicOptionalIdPartition.topicPartition) | ||||
|         result.info.leaderHwChange match { | ||||
|           case LeaderHwChange.INCREASED => | ||||
|             // some delayed operations may be unblocked after HW changed | ||||
|             delayedProducePurgatory.checkAndComplete(requestKey) | ||||
|             delayedFetchPurgatory.checkAndComplete(requestKey) | ||||
|             delayedDeleteRecordsPurgatory.checkAndComplete(requestKey) | ||||
|             if (topicOptionalIdPartition.topicId.isPresent) delayedShareFetchPurgatory.checkAndComplete(new DelayedShareFetchPartitionKey( | ||||
|               topicOptionalIdPartition.topicId.get, topicOptionalIdPartition.partition)) | ||||
|           case LeaderHwChange.SAME => | ||||
|             // probably unblock some follower fetch requests since log end offset has been updated | ||||
|             delayedFetchPurgatory.checkAndComplete(requestKey) | ||||
|  | @ -1392,7 +1429,8 @@ class ReplicaManager(val config: KafkaConfig, | |||
|                                entriesPerPartition: Map[TopicPartition, MemoryRecords], | ||||
|                                requiredAcks: Short, | ||||
|                                requestLocal: RequestLocal, | ||||
|                                verificationGuards: Map[TopicPartition, VerificationGuard]): Map[TopicPartition, LogAppendResult] = { | ||||
|                                verificationGuards: Map[TopicPartition, VerificationGuard]): | ||||
|   Map[TopicOptionalIdPartition, LogAppendResult] = { | ||||
|     val traceEnabled = isTraceEnabled | ||||
|     def processFailedRecord(topicPartition: TopicPartition, t: Throwable) = { | ||||
|       val logStartOffset = onlinePartition(topicPartition).map(_.logStartOffset).getOrElse(-1L) | ||||
|  | @ -1417,7 +1455,7 @@ class ReplicaManager(val config: KafkaConfig, | |||
| 
 | ||||
|       // reject appending to internal topics if it is not allowed | ||||
|       if (Topic.isInternal(topicPartition.topic) && !internalTopicsAllowed) { | ||||
|         (topicPartition, LogAppendResult( | ||||
|         (new TopicOptionalIdPartition(Optional.empty(), topicPartition), LogAppendResult( | ||||
|           LogAppendInfo.UNKNOWN_LOG_APPEND_INFO, | ||||
|           Some(new InvalidTopicException(s"Cannot append to internal topic ${topicPartition.topic}")), | ||||
|           hasCustomErrorMessage = false)) | ||||
|  | @ -1438,7 +1476,10 @@ class ReplicaManager(val config: KafkaConfig, | |||
|             trace(s"${records.sizeInBytes} written to log $topicPartition beginning at offset " + | ||||
|               s"${info.firstOffset} and ending at offset ${info.lastOffset}") | ||||
| 
 | ||||
|           (topicPartition, LogAppendResult(info, exception = None, hasCustomErrorMessage = false)) | ||||
|           var topicId: Optional[Uuid] = Optional.empty() | ||||
|           if (partition.topicId.isDefined) topicId = Optional.of(partition.topicId.get) | ||||
| 
 | ||||
|           (new TopicOptionalIdPartition(topicId, topicPartition), LogAppendResult(info, exception = None, hasCustomErrorMessage = false)) | ||||
|         } catch { | ||||
|           // NOTE: Failed produce requests metric is not incremented for known exceptions | ||||
|           // it is supposed to indicate un-expected failures of a broker in handling a produce request | ||||
|  | @ -1448,15 +1489,15 @@ class ReplicaManager(val config: KafkaConfig, | |||
|                    _: RecordBatchTooLargeException | | ||||
|                    _: CorruptRecordException | | ||||
|                    _: KafkaStorageException) => | ||||
|             (topicPartition, LogAppendResult(LogAppendInfo.UNKNOWN_LOG_APPEND_INFO, Some(e), hasCustomErrorMessage = false)) | ||||
|             (new TopicOptionalIdPartition(Optional.empty(), topicPartition), LogAppendResult(LogAppendInfo.UNKNOWN_LOG_APPEND_INFO, Some(e), hasCustomErrorMessage = false)) | ||||
|           case rve: RecordValidationException => | ||||
|             val logStartOffset = processFailedRecord(topicPartition, rve.invalidException) | ||||
|             val recordErrors = rve.recordErrors | ||||
|             (topicPartition, LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithAdditionalInfo(logStartOffset, recordErrors), | ||||
|             (new TopicOptionalIdPartition(Optional.empty(), topicPartition), LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithAdditionalInfo(logStartOffset, recordErrors), | ||||
|               Some(rve.invalidException), hasCustomErrorMessage = true)) | ||||
|           case t: Throwable => | ||||
|             val logStartOffset = processFailedRecord(topicPartition, t) | ||||
|             (topicPartition, LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithLogStartOffset(logStartOffset), | ||||
|             (new TopicOptionalIdPartition(Optional.empty(), topicPartition), LogAppendResult(LogAppendInfo.unknownLogAppendInfoWithLogStartOffset(logStartOffset), | ||||
|               Some(t), hasCustomErrorMessage = false)) | ||||
|         } | ||||
|       } | ||||
|  | @ -2452,7 +2493,7 @@ class ReplicaManager(val config: KafkaConfig, | |||
|         s"epoch $controllerEpoch with correlation id $correlationId for ${partitionsToMakeFollower.size} partitions") | ||||
| 
 | ||||
|       partitionsToMakeFollower.foreach { partition => | ||||
|         completeDelayedOperationsWhenNotPartitionLeader(partition.topicPartition) | ||||
|         completeDelayedOperationsWhenNotPartitionLeader(partition.topicPartition, partition.topicId) | ||||
|       } | ||||
| 
 | ||||
|       if (isShuttingDown.get()) { | ||||
|  | @ -2676,6 +2717,7 @@ class ReplicaManager(val config: KafkaConfig, | |||
|     delayedProducePurgatory.shutdown() | ||||
|     delayedDeleteRecordsPurgatory.shutdown() | ||||
|     delayedElectLeaderPurgatory.shutdown() | ||||
|     delayedShareFetchPurgatory.shutdown() | ||||
|     if (checkpointHW) | ||||
|       checkpointHighWatermarks() | ||||
|     replicaSelectorOpt.foreach(_.close) | ||||
|  | @ -3029,7 +3071,8 @@ class ReplicaManager(val config: KafkaConfig, | |||
|       replicaFetcherManager.addFetcherForPartitions(partitionAndOffsets) | ||||
|       stateChangeLogger.info(s"Started fetchers as part of become-follower for ${partitionsToStartFetching.size} partitions") | ||||
| 
 | ||||
|       partitionsToStartFetching.keySet.foreach(completeDelayedOperationsWhenNotPartitionLeader) | ||||
|       partitionsToStartFetching.foreach{ case (topicPartition, partition) => | ||||
|         completeDelayedOperationsWhenNotPartitionLeader(topicPartition, partition.topicId)} | ||||
| 
 | ||||
|       updateLeaderAndFollowerMetrics(followerTopicSet) | ||||
|     } | ||||
|  |  | |||
|  | @ -16,7 +16,6 @@ | |||
|  */ | ||||
| package kafka.server.share; | ||||
| 
 | ||||
| import kafka.server.DelayedActionQueue; | ||||
| import kafka.server.DelayedOperationPurgatory; | ||||
| import kafka.server.ReplicaManager; | ||||
| import kafka.server.ReplicaQuota; | ||||
|  | @ -54,6 +53,7 @@ import scala.jdk.javaapi.CollectionConverters; | |||
| import static kafka.server.share.SharePartitionManagerTest.DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL; | ||||
| import static kafka.server.share.SharePartitionManagerTest.PARTITION_MAX_BYTES; | ||||
| import static kafka.server.share.SharePartitionManagerTest.buildLogReadResult; | ||||
| import static kafka.server.share.SharePartitionManagerTest.mockReplicaManagerDelayedShareFetch; | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertFalse; | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
|  | @ -330,6 +330,7 @@ public class DelayedShareFetchTest { | |||
|         DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>( | ||||
|             "TestShareFetch", mockTimer, replicaManager.localBrokerId(), | ||||
|             DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true); | ||||
|         mockReplicaManagerDelayedShareFetch(replicaManager, delayedShareFetchPurgatory); | ||||
| 
 | ||||
|         Set<Object> delayedShareFetchWatchKeys = new HashSet<>(); | ||||
|         partitionMaxBytes1.keySet().forEach(topicIdPartition -> delayedShareFetchWatchKeys.add(new DelayedShareFetchGroupKey(groupId, topicIdPartition.topicId(), topicIdPartition.partition()))); | ||||
|  | @ -357,16 +358,12 @@ public class DelayedShareFetchTest { | |||
| 
 | ||||
|         doAnswer(invocation -> buildLogReadResult(Collections.singleton(tp1))).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); | ||||
| 
 | ||||
|         DelayedActionQueue delayedActionQueue = spy(new DelayedActionQueue()); | ||||
| 
 | ||||
|         Map<SharePartitionKey, SharePartition> partitionCacheMap = new ConcurrentHashMap<>(); | ||||
|         partitionCacheMap.put(new SharePartitionKey(groupId, tp0), sp0); | ||||
|         partitionCacheMap.put(new SharePartitionKey(groupId, tp1), sp1); | ||||
|         partitionCacheMap.put(new SharePartitionKey(groupId, tp2), sp2); | ||||
|         SharePartitionManager sharePartitionManager2 = SharePartitionManagerTest.SharePartitionManagerBuilder | ||||
|             .builder() | ||||
|             .withDelayedShareFetchPurgatory(delayedShareFetchPurgatory) | ||||
|             .withDelayedActionsQueue(delayedActionQueue) | ||||
|             .withPartitionCacheMap(partitionCacheMap) | ||||
|             .build(); | ||||
| 
 | ||||
|  | @ -390,8 +387,8 @@ public class DelayedShareFetchTest { | |||
|         Mockito.verify(replicaManager, times(1)).readFromLog( | ||||
|             any(), any(), any(ReplicaQuota.class), anyBoolean()); | ||||
|         assertFalse(delayedShareFetch1.isCompleted()); | ||||
|         Mockito.verify(delayedActionQueue, times(1)).add(any()); | ||||
|         Mockito.verify(delayedActionQueue, times(0)).tryCompleteActions(); | ||||
|         Mockito.verify(replicaManager, times(1)).addToActionQueue(any()); | ||||
|         Mockito.verify(replicaManager, times(0)).tryCompleteActions(); | ||||
|     } | ||||
| 
 | ||||
|     static class DelayedShareFetchBuilder { | ||||
|  |  | |||
|  | @ -16,7 +16,6 @@ | |||
|  */ | ||||
| package kafka.server.share; | ||||
| 
 | ||||
| import kafka.server.DelayedActionQueue; | ||||
| import kafka.server.DelayedOperationPurgatory; | ||||
| import kafka.server.LogReadResult; | ||||
| import kafka.server.ReplicaManager; | ||||
|  | @ -1034,13 +1033,13 @@ public class SharePartitionManagerTest { | |||
|         DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>( | ||||
|             "TestShareFetch", mockTimer, replicaManager.localBrokerId(), | ||||
|             DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true); | ||||
|         mockReplicaManagerDelayedShareFetch(replicaManager, delayedShareFetchPurgatory); | ||||
| 
 | ||||
|         SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder() | ||||
|             .withReplicaManager(replicaManager) | ||||
|             .withTime(time) | ||||
|             .withMetrics(metrics) | ||||
|             .withTimer(mockTimer) | ||||
|             .withDelayedShareFetchPurgatory(delayedShareFetchPurgatory) | ||||
|             .build(); | ||||
| 
 | ||||
|         doAnswer(invocation -> buildLogReadResult(partitionMaxBytes.keySet())).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); | ||||
|  | @ -1096,12 +1095,12 @@ public class SharePartitionManagerTest { | |||
|         DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>( | ||||
|             "TestShareFetch", mockTimer, replicaManager.localBrokerId(), | ||||
|             DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true); | ||||
|         mockReplicaManagerDelayedShareFetch(replicaManager, delayedShareFetchPurgatory); | ||||
| 
 | ||||
|         SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder() | ||||
|             .withTime(time) | ||||
|             .withReplicaManager(replicaManager) | ||||
|             .withTimer(mockTimer) | ||||
|             .withDelayedShareFetchPurgatory(delayedShareFetchPurgatory) | ||||
|             .build(); | ||||
| 
 | ||||
|         SharePartition sp0 = mock(SharePartition.class); | ||||
|  | @ -1192,12 +1191,13 @@ public class SharePartitionManagerTest { | |||
|         DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>( | ||||
|                 "TestShareFetch", mockTimer, replicaManager.localBrokerId(), | ||||
|                 DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true); | ||||
|         mockReplicaManagerDelayedShareFetch(replicaManager, delayedShareFetchPurgatory); | ||||
| 
 | ||||
|         SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder() | ||||
|             .withPartitionCacheMap(partitionCacheMap) | ||||
|             .withReplicaManager(replicaManager) | ||||
|             .withTimer(mockTimer) | ||||
|             .withDelayedShareFetchPurgatory(delayedShareFetchPurgatory).build(); | ||||
|             .build(); | ||||
| 
 | ||||
|         CompletableFuture<Map<TopicIdPartition, ShareFetchResponseData.PartitionData>> future = | ||||
|             sharePartitionManager.fetchMessages(groupId, memberId.toString(), fetchParams, partitionMaxBytes); | ||||
|  | @ -1223,11 +1223,11 @@ public class SharePartitionManagerTest { | |||
|         DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>( | ||||
|             "TestShareFetch", mockTimer, replicaManager.localBrokerId(), | ||||
|             DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true); | ||||
|         mockReplicaManagerDelayedShareFetch(replicaManager, delayedShareFetchPurgatory); | ||||
| 
 | ||||
|         SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder() | ||||
|             .withReplicaManager(replicaManager) | ||||
|             .withTimer(mockTimer) | ||||
|             .withDelayedShareFetchPurgatory(delayedShareFetchPurgatory) | ||||
|             .build(); | ||||
| 
 | ||||
|         doAnswer(invocation -> buildLogReadResult(partitionMaxBytes.keySet())).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); | ||||
|  | @ -1667,12 +1667,12 @@ public class SharePartitionManagerTest { | |||
|         DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>( | ||||
|                 "TestShareFetch", mockTimer, replicaManager.localBrokerId(), | ||||
|                 DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true); | ||||
|         mockReplicaManagerDelayedShareFetch(replicaManager, delayedShareFetchPurgatory); | ||||
| 
 | ||||
|         SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder() | ||||
|             .withReplicaManager(replicaManager) | ||||
|             .withTime(time) | ||||
|             .withTimer(mockTimer) | ||||
|             .withDelayedShareFetchPurgatory(delayedShareFetchPurgatory) | ||||
|             .withFetchQueue(fetchQueue).build(); | ||||
| 
 | ||||
|         doAnswer(invocation -> buildLogReadResult(partitionMaxBytes.keySet())).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); | ||||
|  | @ -1724,6 +1724,7 @@ public class SharePartitionManagerTest { | |||
|         DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>( | ||||
|                 "TestShareFetch", mockTimer, replicaManager.localBrokerId(), | ||||
|                 DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true); | ||||
|         mockReplicaManagerDelayedShareFetch(replicaManager, delayedShareFetchPurgatory); | ||||
| 
 | ||||
|         // Initially you cannot acquire records for both sp1 and sp2. | ||||
|         when(sp1.maybeAcquireFetchLock()).thenReturn(true); | ||||
|  | @ -1736,7 +1737,6 @@ public class SharePartitionManagerTest { | |||
| 
 | ||||
|         SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder() | ||||
|             .withPartitionCacheMap(partitionCacheMap) | ||||
|             .withDelayedShareFetchPurgatory(delayedShareFetchPurgatory) | ||||
|             .withReplicaManager(replicaManager) | ||||
|             .withTimer(mockTimer) | ||||
|             .build(); | ||||
|  | @ -1820,6 +1820,7 @@ public class SharePartitionManagerTest { | |||
|         DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>( | ||||
|                 "TestShareFetch", mockTimer, replicaManager.localBrokerId(), | ||||
|                 DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true); | ||||
|         mockReplicaManagerDelayedShareFetch(replicaManager, delayedShareFetchPurgatory); | ||||
| 
 | ||||
|         // Initially you cannot acquire records for both all 3 share partitions. | ||||
|         when(sp1.maybeAcquireFetchLock()).thenReturn(true); | ||||
|  | @ -1834,7 +1835,6 @@ public class SharePartitionManagerTest { | |||
| 
 | ||||
|         SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder() | ||||
|             .withPartitionCacheMap(partitionCacheMap) | ||||
|             .withDelayedShareFetchPurgatory(delayedShareFetchPurgatory) | ||||
|             .withReplicaManager(replicaManager) | ||||
|             .withTimer(mockTimer) | ||||
|             .build(); | ||||
|  | @ -1913,6 +1913,7 @@ public class SharePartitionManagerTest { | |||
|         DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>( | ||||
|                 "TestShareFetch", mockTimer, replicaManager.localBrokerId(), | ||||
|                 DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true); | ||||
|         mockReplicaManagerDelayedShareFetch(replicaManager, delayedShareFetchPurgatory); | ||||
| 
 | ||||
|         // Initially you cannot acquire records for both sp1 and sp2. | ||||
|         when(sp1.maybeAcquireFetchLock()).thenReturn(true); | ||||
|  | @ -1926,7 +1927,6 @@ public class SharePartitionManagerTest { | |||
|         SharePartitionManager sharePartitionManager = spy(SharePartitionManagerBuilder.builder() | ||||
|             .withPartitionCacheMap(partitionCacheMap) | ||||
|             .withCache(cache) | ||||
|             .withDelayedShareFetchPurgatory(delayedShareFetchPurgatory) | ||||
|             .withReplicaManager(replicaManager) | ||||
|             .withTimer(mockTimer) | ||||
|             .build()); | ||||
|  | @ -2013,6 +2013,7 @@ public class SharePartitionManagerTest { | |||
|         DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>( | ||||
|                 "TestShareFetch", mockTimer, replicaManager.localBrokerId(), | ||||
|                 DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true); | ||||
|         mockReplicaManagerDelayedShareFetch(replicaManager, delayedShareFetchPurgatory); | ||||
| 
 | ||||
|         // Initially you cannot acquire records for both all 3 share partitions. | ||||
|         when(sp1.maybeAcquireFetchLock()).thenReturn(true); | ||||
|  | @ -2028,7 +2029,6 @@ public class SharePartitionManagerTest { | |||
|         SharePartitionManager sharePartitionManager = spy(SharePartitionManagerBuilder.builder() | ||||
|             .withPartitionCacheMap(partitionCacheMap) | ||||
|             .withCache(cache) | ||||
|             .withDelayedShareFetchPurgatory(delayedShareFetchPurgatory) | ||||
|             .withReplicaManager(replicaManager) | ||||
|             .withTimer(mockTimer) | ||||
|             .build()); | ||||
|  | @ -2084,10 +2084,11 @@ public class SharePartitionManagerTest { | |||
|         DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>( | ||||
|                 "TestShareFetch", mockTimer, replicaManager.localBrokerId(), | ||||
|                 DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true); | ||||
|         mockReplicaManagerDelayedShareFetch(replicaManager, delayedShareFetchPurgatory); | ||||
| 
 | ||||
|         SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder() | ||||
|             .withPartitionCacheMap(partitionCacheMap).withReplicaManager(replicaManager).withTimer(mockTimer) | ||||
|             .withDelayedShareFetchPurgatory(delayedShareFetchPurgatory).build(); | ||||
|             .build(); | ||||
| 
 | ||||
|         CompletableFuture<Map<TopicIdPartition, ShareFetchResponseData.PartitionData>> future = | ||||
|             sharePartitionManager.fetchMessages(groupId, memberId.toString(), fetchParams, partitionMaxBytes); | ||||
|  | @ -2122,10 +2123,11 @@ public class SharePartitionManagerTest { | |||
|         DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>( | ||||
|                 "TestShareFetch", mockTimer, replicaManager.localBrokerId(), | ||||
|                 DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true); | ||||
|         mockReplicaManagerDelayedShareFetch(replicaManager, delayedShareFetchPurgatory); | ||||
| 
 | ||||
|         SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder() | ||||
|             .withPartitionCacheMap(partitionCacheMap).withReplicaManager(replicaManager).withTimer(mockTimer) | ||||
|                 .withDelayedShareFetchPurgatory(delayedShareFetchPurgatory).build(); | ||||
|             .build(); | ||||
| 
 | ||||
|         // Return LeaderNotAvailableException to simulate initialization failure. | ||||
|         when(sp0.maybeInitialize()).thenReturn(FutureUtils.failedFuture(new LeaderNotAvailableException("Leader not available"))); | ||||
|  | @ -2289,6 +2291,21 @@ public class SharePartitionManagerTest { | |||
|         return CollectionConverters.asScala(logReadResults).toSeq(); | ||||
|     } | ||||
| 
 | ||||
|     static void mockReplicaManagerDelayedShareFetch(ReplicaManager replicaManager, | ||||
|                                                     DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory) { | ||||
|         doAnswer(invocationOnMock -> { | ||||
|             Object[] args = invocationOnMock.getArguments(); | ||||
|             delayedShareFetchPurgatory.checkAndComplete(args[0]); | ||||
|             return null; | ||||
|         }).when(replicaManager).completeDelayedShareFetchRequest(any(DelayedShareFetchKey.class)); | ||||
| 
 | ||||
|         doAnswer(invocationOnMock -> { | ||||
|             Object[] args = invocationOnMock.getArguments(); | ||||
|             delayedShareFetchPurgatory.tryCompleteElseWatch((DelayedShareFetch) args[0], (Seq<Object>) args[1]); | ||||
|             return null; | ||||
|         }).when(replicaManager).addDelayedShareFetchRequest(any(), any()); | ||||
|     } | ||||
| 
 | ||||
|     static class SharePartitionManagerBuilder { | ||||
|         private ReplicaManager replicaManager = mock(ReplicaManager.class); | ||||
|         private Time time = new MockTime(); | ||||
|  | @ -2298,8 +2315,6 @@ public class SharePartitionManagerTest { | |||
|         private Timer timer = new MockTimer(); | ||||
|         private Metrics metrics = new Metrics(); | ||||
|         private ConcurrentLinkedQueue<ShareFetchData> fetchQueue = new ConcurrentLinkedQueue<>(); | ||||
|         private DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = mock(DelayedOperationPurgatory.class); | ||||
|         private DelayedActionQueue delayedActionsQueue = mock(DelayedActionQueue.class); | ||||
| 
 | ||||
|         private SharePartitionManagerBuilder withReplicaManager(ReplicaManager replicaManager) { | ||||
|             this.replicaManager = replicaManager; | ||||
|  | @ -2341,16 +2356,6 @@ public class SharePartitionManagerTest { | |||
|             return this; | ||||
|         } | ||||
| 
 | ||||
|         SharePartitionManagerBuilder withDelayedShareFetchPurgatory(DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory) { | ||||
|             this.delayedShareFetchPurgatory = delayedShareFetchPurgatory; | ||||
|             return this; | ||||
|         } | ||||
| 
 | ||||
|         SharePartitionManagerBuilder withDelayedActionsQueue(DelayedActionQueue delayedActionsQueue) { | ||||
|             this.delayedActionsQueue = delayedActionsQueue; | ||||
|             return this; | ||||
|         } | ||||
| 
 | ||||
|         public static SharePartitionManagerBuilder builder() { | ||||
|             return new SharePartitionManagerBuilder(); | ||||
|         } | ||||
|  | @ -2366,8 +2371,6 @@ public class SharePartitionManagerTest { | |||
|                     MAX_DELIVERY_COUNT, | ||||
|                     MAX_IN_FLIGHT_MESSAGES, | ||||
|                     persister, | ||||
|                     delayedShareFetchPurgatory, | ||||
|                     delayedActionsQueue, | ||||
|                     mock(GroupConfigManager.class), | ||||
|                     metrics); | ||||
|         } | ||||
|  |  | |||
|  | @ -16,7 +16,7 @@ | |||
|  */ | ||||
| package kafka.server.share; | ||||
| 
 | ||||
| import kafka.server.DelayedOperationPurgatory; | ||||
| import kafka.server.ReplicaManager; | ||||
| import kafka.server.share.SharePartition.InFlightState; | ||||
| import kafka.server.share.SharePartition.RecordState; | ||||
| import kafka.server.share.SharePartition.SharePartitionState; | ||||
|  | @ -4949,7 +4949,7 @@ public class SharePartitionTest { | |||
|         private int maxDeliveryCount = MAX_DELIVERY_COUNT; | ||||
|         private int maxInflightMessages = MAX_IN_FLIGHT_MESSAGES; | ||||
|         private Persister persister = NoOpShareStatePersister.getInstance(); | ||||
|         private final DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = Mockito.mock(DelayedOperationPurgatory.class); | ||||
|         private final ReplicaManager replicaManager = Mockito.mock(ReplicaManager.class); | ||||
|         private GroupConfigManager groupConfigManager = Mockito.mock(GroupConfigManager.class); | ||||
| 
 | ||||
|         private SharePartitionBuilder withMaxInflightMessages(int maxInflightMessages) { | ||||
|  | @ -4982,8 +4982,8 @@ public class SharePartitionTest { | |||
|         } | ||||
| 
 | ||||
|         public SharePartition build() { | ||||
|             return new SharePartition(GROUP_ID, TOPIC_ID_PARTITION, maxInflightMessages, maxDeliveryCount, defaultAcquisitionLockTimeoutMs,  | ||||
|                 mockTimer, MOCK_TIME, persister, delayedShareFetchPurgatory, groupConfigManager); | ||||
|             return new SharePartition(GROUP_ID, TOPIC_ID_PARTITION, maxInflightMessages, maxDeliveryCount, | ||||
|                     defaultAcquisitionLockTimeoutMs, mockTimer, MOCK_TIME, persister, replicaManager, groupConfigManager); | ||||
|         } | ||||
|     } | ||||
| } | ||||
|  |  | |||
|  | @ -21,7 +21,7 @@ import kafka.server.MetadataCache | |||
| import kafka.server.metadata.MockConfigRepository | ||||
| import kafka.utils.TestUtils | ||||
| import kafka.utils.TestUtils.MockAlterPartitionManager | ||||
| import org.apache.kafka.common.TopicPartition | ||||
| import org.apache.kafka.common.{TopicPartition, Uuid} | ||||
| import org.apache.kafka.common.config.TopicConfig | ||||
| import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState | ||||
| import org.apache.kafka.common.utils.Utils | ||||
|  | @ -49,6 +49,7 @@ class AbstractPartitionTest { | |||
| 
 | ||||
|   val brokerId = AbstractPartitionTest.brokerId | ||||
|   val remoteReplicaId = brokerId + 1 | ||||
|   val topicId : Option[Uuid] = Option(Uuid.randomUuid()) | ||||
|   val topicPartition = new TopicPartition("test-topic", 0) | ||||
|   val time = new MockTime() | ||||
|   var tmpDir: File = _ | ||||
|  |  | |||
|  | @ -46,6 +46,7 @@ import java.nio.ByteBuffer | |||
| import java.util.Optional | ||||
| import java.util.concurrent.{ConcurrentHashMap, CountDownLatch, Semaphore} | ||||
| import kafka.server.metadata.{KRaftMetadataCache, ZkMetadataCache} | ||||
| import kafka.server.share.{DelayedShareFetch, DelayedShareFetchPartitionKey} | ||||
| import org.apache.kafka.clients.ClientResponse | ||||
| import org.apache.kafka.common.compress.Compression | ||||
| import org.apache.kafka.common.config.TopicConfig | ||||
|  | @ -4110,7 +4111,11 @@ class PartitionTest extends AbstractPartitionTest { | |||
|     val deleteRecords = mock(classOf[DelayedOperationPurgatory[DelayedDeleteRecords]]) | ||||
|     when(deleteRecords.checkAndComplete(requestKey)).thenThrow(new RuntimeException("uh oh")) | ||||
| 
 | ||||
|     val delayedOperations = new DelayedOperations(topicPartition, produce, fetch, deleteRecords) | ||||
|     val shareFetch = mock(classOf[DelayedOperationPurgatory[DelayedShareFetch]]) | ||||
|     when(shareFetch.checkAndComplete(new DelayedShareFetchPartitionKey(topicId.get, topicPartition.partition()))) | ||||
|       .thenThrow(new RuntimeException("uh oh")) | ||||
| 
 | ||||
|     val delayedOperations = new DelayedOperations(topicId, topicPartition, produce, fetch, deleteRecords, shareFetch) | ||||
|     val spyLogManager = spy(logManager) | ||||
|     val partition = new Partition(topicPartition, | ||||
|       replicaLagTimeMaxMs = ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_DEFAULT, | ||||
|  |  | |||
|  | @ -26,6 +26,7 @@ import org.apache.kafka.server.log.remote.quota.RLMQuotaManagerConfig.INACTIVE_S | |||
| import org.apache.kafka.server.log.remote.quota.RLMQuotaMetrics | ||||
| import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota} | ||||
| import kafka.server.epoch.util.MockBlockingSender | ||||
| import kafka.server.share.DelayedShareFetch | ||||
| import kafka.utils.TestUtils.waitUntilTrue | ||||
| import kafka.utils.{Pool, TestUtils} | ||||
| import kafka.zk.KafkaZkClient | ||||
|  | @ -3004,6 +3005,8 @@ class ReplicaManagerTest { | |||
|       purgatoryName = "RemoteFetch", timer, reaperEnabled = false) | ||||
|     val mockRemoteListOffsetsPurgatory = new DelayedOperationPurgatory[DelayedRemoteListOffsets]( | ||||
|       purgatoryName = "RemoteListOffsets", timer, reaperEnabled = false) | ||||
|     val mockDelayedShareFetchPurgatory = new DelayedOperationPurgatory[DelayedShareFetch]( | ||||
|       purgatoryName = "ShareFetch", timer, reaperEnabled = false) | ||||
| 
 | ||||
|     // Mock network client to show leader offset of 5 | ||||
|     val blockingSend = new MockBlockingSender( | ||||
|  | @ -3030,6 +3033,7 @@ class ReplicaManagerTest { | |||
|       delayedElectLeaderPurgatoryParam = Some(mockElectLeaderPurgatory), | ||||
|       delayedRemoteFetchPurgatoryParam = Some(mockRemoteFetchPurgatory), | ||||
|       delayedRemoteListOffsetsPurgatoryParam = Some(mockRemoteListOffsetsPurgatory), | ||||
|       delayedShareFetchPurgatoryParam = Some(mockDelayedShareFetchPurgatory), | ||||
|       threadNamePrefix = Option(this.getClass.getName)) { | ||||
| 
 | ||||
|       override protected def createReplicaFetcherManager(metrics: Metrics, | ||||
|  | @ -3428,6 +3432,8 @@ class ReplicaManagerTest { | |||
|       purgatoryName = "DelayedRemoteFetch", timer, reaperEnabled = false) | ||||
|     val mockDelayedRemoteListOffsetsPurgatory = new DelayedOperationPurgatory[DelayedRemoteListOffsets]( | ||||
|       purgatoryName = "RemoteListOffsets", timer, reaperEnabled = false) | ||||
|     val mockDelayedShareFetchPurgatory = new DelayedOperationPurgatory[DelayedShareFetch]( | ||||
|       purgatoryName = "ShareFetch", timer, reaperEnabled = false) | ||||
| 
 | ||||
|     when(metadataCache.contains(new TopicPartition(topic, 0))).thenReturn(true) | ||||
| 
 | ||||
|  | @ -3461,6 +3467,7 @@ class ReplicaManagerTest { | |||
|       delayedElectLeaderPurgatoryParam = Some(mockDelayedElectLeaderPurgatory), | ||||
|       delayedRemoteFetchPurgatoryParam = Some(mockDelayedRemoteFetchPurgatory), | ||||
|       delayedRemoteListOffsetsPurgatoryParam = Some(mockDelayedRemoteListOffsetsPurgatory), | ||||
|       delayedShareFetchPurgatoryParam = Some(mockDelayedShareFetchPurgatory), | ||||
|       threadNamePrefix = Option(this.getClass.getName), | ||||
|       addPartitionsToTxnManager = Some(addPartitionsToTxnManager), | ||||
|       directoryEventHandler = directoryEventHandler, | ||||
|  |  | |||
|  | @ -181,7 +181,7 @@ public class ReplicaFetcherThreadBenchmark { | |||
|             Mockito.when(offsetCheckpoints.fetch(logDir.getAbsolutePath(), tp)).thenReturn(Optional.of(0L)); | ||||
|             AlterPartitionManager isrChannelManager = Mockito.mock(AlterPartitionManager.class); | ||||
|             Partition partition = new Partition(tp, 100, MetadataVersion.latestTesting(), | ||||
|                     0, () -> -1, Time.SYSTEM, alterPartitionListener, new DelayedOperationsMock(tp), | ||||
|                     0, () -> -1, Time.SYSTEM, alterPartitionListener, new DelayedOperationsMock(topicId, tp), | ||||
|                     Mockito.mock(MetadataCache.class), logManager, isrChannelManager, topicId); | ||||
| 
 | ||||
|             partition.makeFollower(partitionState, offsetCheckpoints, topicId, Option.empty()); | ||||
|  | @ -277,8 +277,8 @@ public class ReplicaFetcherThreadBenchmark { | |||
| 
 | ||||
|     // avoid mocked DelayedOperations to avoid mocked class affecting benchmark results | ||||
|     private static class DelayedOperationsMock extends DelayedOperations { | ||||
|         DelayedOperationsMock(TopicPartition topicPartition) { | ||||
|             super(topicPartition, null, null, null); | ||||
|         DelayedOperationsMock(Option<Uuid>  topicId, TopicPartition topicPartition) { | ||||
|             super(topicId, topicPartition, null, null, null, null); | ||||
|         } | ||||
| 
 | ||||
|         @Override | ||||
|  |  | |||
|  | @ -139,7 +139,7 @@ public class UpdateFollowerFetchStateBenchmark { | |||
|     // avoid mocked DelayedOperations to avoid mocked class affecting benchmark results | ||||
|     private class DelayedOperationsMock extends DelayedOperations { | ||||
|         DelayedOperationsMock() { | ||||
|             super(topicPartition, null, null, null); | ||||
|             super(topicId, topicPartition, null, null, null, null); | ||||
|         } | ||||
| 
 | ||||
|         @Override | ||||
|  |  | |||
|  | @ -0,0 +1,101 @@ | |||
| /* | ||||
|  * 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.common; | ||||
| 
 | ||||
| import org.apache.kafka.common.TopicPartition; | ||||
| import org.apache.kafka.common.Uuid; | ||||
| 
 | ||||
| import java.util.Objects; | ||||
| import java.util.Optional; | ||||
| 
 | ||||
| /** | ||||
|  * This represents universally unique identifier with topic id for a topic partition. However, for this wrapper, we can | ||||
|  * have an optional topic id with a not null topic partition to account for the functionalities that don't have topic id incorporated yet. | ||||
|  */ | ||||
| public class TopicOptionalIdPartition { | ||||
| 
 | ||||
|     private final Optional<Uuid> topicId; | ||||
|     private final TopicPartition topicPartition; | ||||
| 
 | ||||
|     /** | ||||
|      * Create an instance with the provided parameters. | ||||
|      * | ||||
|      * @param topicId the topic id | ||||
|      * @param topicPartition the topic partition | ||||
|      */ | ||||
|     public TopicOptionalIdPartition(Optional<Uuid> topicId, TopicPartition topicPartition) { | ||||
|         this.topicId = topicId; | ||||
|         this.topicPartition = Objects.requireNonNull(topicPartition, "topicPartition can not be null"); | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|      * @return Universally unique id representing this topic partition. | ||||
|      */ | ||||
|     public Optional<Uuid> topicId() { | ||||
|         return topicId; | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|      * @return the topic name. | ||||
|      */ | ||||
|     public String topic() { | ||||
|         return topicPartition.topic(); | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|      * @return the partition id. | ||||
|      */ | ||||
|     public int partition() { | ||||
|         return topicPartition.partition(); | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|      * @return Topic partition representing this instance. | ||||
|      */ | ||||
|     public TopicPartition topicPartition() { | ||||
|         return topicPartition; | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public boolean equals(Object o) { | ||||
|         if (this == o) { | ||||
|             return true; | ||||
|         } | ||||
|         if (o == null || getClass() != o.getClass()) { | ||||
|             return false; | ||||
|         } | ||||
|         TopicOptionalIdPartition that = (TopicOptionalIdPartition) o; | ||||
|         return topicId.equals(that.topicId) && | ||||
|             topicPartition.equals(that.topicPartition); | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public int hashCode() { | ||||
|         final int prime = 31; | ||||
|         int result = 0; | ||||
|         if (topicId.isPresent()) { | ||||
|             result = prime + topicId.get().hashCode(); | ||||
|         } | ||||
|         result = prime * result + topicPartition.hashCode(); | ||||
|         return result; | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public String toString() { | ||||
|         return topicId.map(uuid -> uuid + ":" + topic() + "-" + partition()).orElseGet(() -> "none" + ":" + topic() + "-" + partition()); | ||||
|     } | ||||
| } | ||||
		Loading…
	
		Reference in New Issue