ijuma commented on a change in pull request #9065:
URL: https://github.com/apache/kafka/pull/9065#discussion_r459616154



##########
File path: core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala
##########
@@ -116,6 +117,56 @@ class PartitionLockTest extends Logging {
     future.get(15, TimeUnit.SECONDS)
   }
 
+  /**
+   * Concurrently calling updateAssignmentAndIsr should always ensure that 
non-lock access
+   * to the inner remoteReplicaMap (accessed by getReplica) cannot see an 
intermediate state
+   * where replicas present both in the old and new assignment are missing
+   */
+  @Test
+  def testGetReplicaWithUpdateAssignmentAndIsr(): Unit = {
+    val active = new AtomicBoolean(true)
+    val replicaToCheck = 3
+    val firstReplicaSet = Seq[Integer](3, 4, 5).asJava
+    val secondReplicaSet = Seq[Integer](1, 2, 3).asJava
+    def partitionState(replicas: java.util.List[Integer]): 
LeaderAndIsrPartitionState = new LeaderAndIsrPartitionState()
+      .setControllerEpoch(1)
+      .setLeader(replicas.get(0))
+      .setLeaderEpoch(1)
+      .setIsr(replicas)
+      .setZkVersion(1)
+      .setReplicas(replicas)
+      .setIsNew(true)
+    val offsetCheckpoints: OffsetCheckpoints = mock(classOf[OffsetCheckpoints])
+    // Update replica set synchronously first to avoid race conditions
+    partition.makeLeader(partitionState(secondReplicaSet), offsetCheckpoints)
+    assertTrue(s"Expected replica $replicaToCheck to be defined", 
partition.getReplica(replicaToCheck).isDefined)
+
+    var i = 0

Review comment:
       Shouldn't this be inside the thread state?

##########
File path: core/src/main/scala/kafka/utils/Pool.scala
##########
@@ -69,6 +69,8 @@ class Pool[K,V](valueFactory: Option[K => V] = None) extends 
Iterable[(K, V)] {
 
   def remove(key: K, value: V): Boolean = pool.remove(key, value)
 
+  def removeAll(keys: Iterable[K]): Unit = 
pool.keySet().removeAll(keys.asJavaCollection)

Review comment:
       Nit: `()` is not needed.

##########
File path: core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala
##########
@@ -116,6 +117,56 @@ class PartitionLockTest extends Logging {
     future.get(15, TimeUnit.SECONDS)
   }
 
+  /**
+   * Concurrently calling updateAssignmentAndIsr should always ensure that 
non-lock access
+   * to the inner remoteReplicaMap (accessed by getReplica) cannot see an 
intermediate state
+   * where replicas present both in the old and new assignment are missing
+   */
+  @Test
+  def testGetReplicaWithUpdateAssignmentAndIsr(): Unit = {
+    val active = new AtomicBoolean(true)
+    val replicaToCheck = 3
+    val firstReplicaSet = Seq[Integer](3, 4, 5).asJava
+    val secondReplicaSet = Seq[Integer](1, 2, 3).asJava
+    def partitionState(replicas: java.util.List[Integer]): 
LeaderAndIsrPartitionState = new LeaderAndIsrPartitionState()

Review comment:
       No need to repeat `LeaderAndIsrPartitionState` twice.

##########
File path: core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala
##########
@@ -116,6 +117,56 @@ class PartitionLockTest extends Logging {
     future.get(15, TimeUnit.SECONDS)
   }
 
+  /**
+   * Concurrently calling updateAssignmentAndIsr should always ensure that 
non-lock access
+   * to the inner remoteReplicaMap (accessed by getReplica) cannot see an 
intermediate state
+   * where replicas present both in the old and new assignment are missing
+   */
+  @Test
+  def testGetReplicaWithUpdateAssignmentAndIsr(): Unit = {
+    val active = new AtomicBoolean(true)
+    val replicaToCheck = 3
+    val firstReplicaSet = Seq[Integer](3, 4, 5).asJava
+    val secondReplicaSet = Seq[Integer](1, 2, 3).asJava
+    def partitionState(replicas: java.util.List[Integer]): 
LeaderAndIsrPartitionState = new LeaderAndIsrPartitionState()
+      .setControllerEpoch(1)
+      .setLeader(replicas.get(0))
+      .setLeaderEpoch(1)
+      .setIsr(replicas)
+      .setZkVersion(1)
+      .setReplicas(replicas)
+      .setIsNew(true)
+    val offsetCheckpoints: OffsetCheckpoints = mock(classOf[OffsetCheckpoints])
+    // Update replica set synchronously first to avoid race conditions
+    partition.makeLeader(partitionState(secondReplicaSet), offsetCheckpoints)
+    assertTrue(s"Expected replica $replicaToCheck to be defined", 
partition.getReplica(replicaToCheck).isDefined)
+
+    var i = 0
+    val future = executorService.submit((() => {
+      // Flip assignment between two replica sets
+      while (active.get) {
+        val replicas = if (i % 2 == 0) {
+          firstReplicaSet
+        } else {
+          secondReplicaSet
+        }
+
+        partition.makeLeader(partitionState(replicas), offsetCheckpoints)
+
+        i += 1
+        Thread.sleep(1) // just to avoid tight loop
+      }
+    }): Runnable)
+
+    val deadline = 5.seconds.fromNow
+    while(deadline.hasTimeLeft()) {

Review comment:
       Nit: space missing after `while`.

##########
File path: core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala
##########
@@ -116,6 +117,56 @@ class PartitionLockTest extends Logging {
     future.get(15, TimeUnit.SECONDS)
   }
 
+  /**
+   * Concurrently calling updateAssignmentAndIsr should always ensure that 
non-lock access
+   * to the inner remoteReplicaMap (accessed by getReplica) cannot see an 
intermediate state
+   * where replicas present both in the old and new assignment are missing
+   */
+  @Test
+  def testGetReplicaWithUpdateAssignmentAndIsr(): Unit = {
+    val active = new AtomicBoolean(true)
+    val replicaToCheck = 3
+    val firstReplicaSet = Seq[Integer](3, 4, 5).asJava
+    val secondReplicaSet = Seq[Integer](1, 2, 3).asJava
+    def partitionState(replicas: java.util.List[Integer]): 
LeaderAndIsrPartitionState = new LeaderAndIsrPartitionState()
+      .setControllerEpoch(1)
+      .setLeader(replicas.get(0))
+      .setLeaderEpoch(1)
+      .setIsr(replicas)
+      .setZkVersion(1)
+      .setReplicas(replicas)
+      .setIsNew(true)
+    val offsetCheckpoints: OffsetCheckpoints = mock(classOf[OffsetCheckpoints])
+    // Update replica set synchronously first to avoid race conditions
+    partition.makeLeader(partitionState(secondReplicaSet), offsetCheckpoints)
+    assertTrue(s"Expected replica $replicaToCheck to be defined", 
partition.getReplica(replicaToCheck).isDefined)
+
+    var i = 0
+    val future = executorService.submit((() => {
+      // Flip assignment between two replica sets
+      while (active.get) {
+        val replicas = if (i % 2 == 0) {
+          firstReplicaSet
+        } else {
+          secondReplicaSet
+        }
+
+        partition.makeLeader(partitionState(replicas), offsetCheckpoints)
+
+        i += 1
+        Thread.sleep(1) // just to avoid tight loop
+      }
+    }): Runnable)
+
+    val deadline = 5.seconds.fromNow

Review comment:
       5 seconds is quite a bit. Can it be lower?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to