You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by yu...@apache.org on 2023/10/07 15:14:40 UTC

[pulsar] branch branch-3.0 updated: [fix][broker] fix bug caused by optimistic locking (#18390)

This is an automated email from the ASF dual-hosted git repository.

yubiao pushed a commit to branch branch-3.0
in repository https://gitbox.apache.org/repos/asf/pulsar.git


The following commit(s) were added to refs/heads/branch-3.0 by this push:
     new eb4f5906f10 [fix][broker] fix bug caused by optimistic locking (#18390)
eb4f5906f10 is described below

commit eb4f5906f105bfc5006ed0fb6951ceafca53d6e1
Author: thetumbled <52...@users.noreply.github.com>
AuthorDate: Fri Sep 15 21:12:39 2023 +0800

    [fix][broker] fix bug caused by optimistic locking (#18390)
    
    (cherry picked from commit 2aa8c3b1a284673b512db2b815514343ec3e7c19)
---
 .../util/collections/ConcurrentLongHashMap.java    | 25 ++++-----
 .../collections/ConcurrentLongLongPairHashMap.java | 39 ++++++++-----
 .../util/collections/ConcurrentLongPairSet.java    | 41 +++++++++-----
 .../util/collections/ConcurrentOpenHashMap.java    | 39 ++++++++-----
 .../util/collections/ConcurrentOpenHashSet.java    | 22 +++-----
 .../apache/pulsar/common/util/FutureUtilTest.java  |  1 -
 .../collections/ConcurrentLongHashMapTest.java     | 62 ++++++++++++++++++++-
 .../ConcurrentLongLongPairHashMapTest.java         | 65 ++++++++++++++++++++++
 .../collections/ConcurrentLongPairSetTest.java     | 64 +++++++++++++++++++++
 .../collections/ConcurrentOpenHashMapTest.java     | 64 +++++++++++++++++++++
 .../collections/ConcurrentOpenHashSetTest.java     | 64 +++++++++++++++++++++
 11 files changed, 414 insertions(+), 72 deletions(-)

diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentLongHashMap.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentLongHashMap.java
index 31b4cb7cbf1..f5b47e7f1ab 100644
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentLongHashMap.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentLongHashMap.java
@@ -306,16 +306,17 @@ public class ConcurrentLongHashMap<V> {
         }
 
         V get(long key, int keyHash) {
-            int bucket = keyHash;
-
             long stamp = tryOptimisticRead();
             boolean acquiredLock = false;
 
+            // add local variable here, so OutOfBound won't happen
+            long[] keys = this.keys;
+            V[] values = this.values;
+            // calculate table.length as capacity to avoid rehash changing capacity
+            int bucket = signSafeMod(keyHash, values.length);
+
             try {
                 while (true) {
-                    int capacity = this.capacity;
-                    bucket = signSafeMod(bucket, capacity);
-
                     // First try optimistic locking
                     long storedKey = keys[bucket];
                     V storedValue = values[bucket];
@@ -333,16 +334,15 @@ public class ConcurrentLongHashMap<V> {
                         if (!acquiredLock) {
                             stamp = readLock();
                             acquiredLock = true;
+
+                            // update local variable
+                            keys = this.keys;
+                            values = this.values;
+                            bucket = signSafeMod(keyHash, values.length);
                             storedKey = keys[bucket];
                             storedValue = values[bucket];
                         }
 
-                        if (capacity != this.capacity) {
-                            // There has been a rehashing. We need to restart the search
-                            bucket = keyHash;
-                            continue;
-                        }
-
                         if (storedKey == key) {
                             return storedValue != DeletedValue ? storedValue : null;
                         } else if (storedValue == EmptyValue) {
@@ -350,8 +350,7 @@ public class ConcurrentLongHashMap<V> {
                             return null;
                         }
                     }
-
-                    ++bucket;
+                    bucket = (bucket + 1) & (values.length - 1);
                 }
             } finally {
                 if (acquiredLock) {
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentLongLongPairHashMap.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentLongLongPairHashMap.java
index c0ccad9b73d..c3babbb8d11 100644
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentLongLongPairHashMap.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentLongLongPairHashMap.java
@@ -284,6 +284,9 @@ public class ConcurrentLongLongPairHashMap {
     // A section is a portion of the hash map that is covered by a single
     @SuppressWarnings("serial")
     private static final class Section extends StampedLock {
+        // Each item take up 4 continuous array space.
+        private static final int ITEM_SIZE = 4;
+
         // Keys and values are stored interleaved in the table array
         private volatile long[] table;
 
@@ -306,7 +309,7 @@ public class ConcurrentLongLongPairHashMap {
                 float expandFactor, float shrinkFactor) {
             this.capacity = alignToPowerOfTwo(capacity);
             this.initCapacity = this.capacity;
-            this.table = new long[4 * this.capacity];
+            this.table = new long[ITEM_SIZE * this.capacity];
             this.size = 0;
             this.usedBuckets = 0;
             this.autoShrink = autoShrink;
@@ -322,7 +325,10 @@ public class ConcurrentLongLongPairHashMap {
         LongPair get(long key1, long key2, int keyHash) {
             long stamp = tryOptimisticRead();
             boolean acquiredLock = false;
-            int bucket = signSafeMod(keyHash, capacity);
+            // add local variable here, so OutOfBound won't happen
+            long[] table = this.table;
+            // calculate table.length / 4 as capacity to avoid rehash changing capacity
+            int bucket = signSafeMod(keyHash, table.length / ITEM_SIZE);
 
             try {
                 while (true) {
@@ -345,8 +351,9 @@ public class ConcurrentLongLongPairHashMap {
                         if (!acquiredLock) {
                             stamp = readLock();
                             acquiredLock = true;
-
-                            bucket = signSafeMod(keyHash, capacity);
+                            // update local variable
+                            table = this.table;
+                            bucket = signSafeMod(keyHash, table.length / ITEM_SIZE);
                             storedKey1 = table[bucket];
                             storedKey2 = table[bucket + 1];
                             storedValue1 = table[bucket + 2];
@@ -361,7 +368,7 @@ public class ConcurrentLongLongPairHashMap {
                         }
                     }
 
-                    bucket = (bucket + 4) & (table.length - 1);
+                    bucket = (bucket + ITEM_SIZE) & (table.length - 1);
                 }
             } finally {
                 if (acquiredLock) {
@@ -413,7 +420,7 @@ public class ConcurrentLongLongPairHashMap {
                         }
                     }
 
-                    bucket = (bucket + 4) & (table.length - 1);
+                    bucket = (bucket + ITEM_SIZE) & (table.length - 1);
                 }
             } finally {
                 if (usedBuckets > resizeThresholdUp) {
@@ -454,7 +461,7 @@ public class ConcurrentLongLongPairHashMap {
                         return false;
                     }
 
-                    bucket = (bucket + 4) & (table.length - 1);
+                    bucket = (bucket + ITEM_SIZE) & (table.length - 1);
                 }
 
             } finally {
@@ -480,7 +487,7 @@ public class ConcurrentLongLongPairHashMap {
         }
 
         private void cleanBucket(int bucket) {
-            int nextInArray = (bucket + 4) & (table.length - 1);
+            int nextInArray = (bucket + ITEM_SIZE) & (table.length - 1);
             if (table[nextInArray] == EmptyKey) {
                 table[bucket] = EmptyKey;
                 table[bucket + 1] = EmptyKey;
@@ -489,7 +496,7 @@ public class ConcurrentLongLongPairHashMap {
                 --usedBuckets;
 
                 // Cleanup all the buckets that were in `DeletedKey` state, so that we can reduce unnecessary expansions
-                bucket = (bucket - 4) & (table.length - 1);
+                bucket = (bucket - ITEM_SIZE) & (table.length - 1);
                 while (table[bucket] == DeletedKey) {
                     table[bucket] = EmptyKey;
                     table[bucket + 1] = EmptyKey;
@@ -497,7 +504,7 @@ public class ConcurrentLongLongPairHashMap {
                     table[bucket + 3] = ValueNotFound;
                     --usedBuckets;
 
-                    bucket = (bucket - 4) & (table.length - 1);
+                    bucket = (bucket - ITEM_SIZE) & (table.length - 1);
                 }
             } else {
                 table[bucket] = DeletedKey;
@@ -540,7 +547,7 @@ public class ConcurrentLongLongPairHashMap {
                 }
 
                 // Go through all the buckets for this section
-                for (int bucket = 0; bucket < table.length; bucket += 4) {
+                for (int bucket = 0; bucket < table.length; bucket += ITEM_SIZE) {
                     long storedKey1 = table[bucket];
                     long storedKey2 = table[bucket + 1];
                     long storedValue1 = table[bucket + 2];
@@ -569,11 +576,11 @@ public class ConcurrentLongLongPairHashMap {
         }
 
         private void rehash(int newCapacity) {
-            long[] newTable = new long[4 * newCapacity];
+            long[] newTable = new long[ITEM_SIZE * newCapacity];
             Arrays.fill(newTable, EmptyKey);
 
             // Re-hash table
-            for (int i = 0; i < table.length; i += 4) {
+            for (int i = 0; i < table.length; i += ITEM_SIZE) {
                 long storedKey1 = table[i];
                 long storedKey2 = table[i + 1];
                 long storedValue1 = table[i + 2];
@@ -593,7 +600,7 @@ public class ConcurrentLongLongPairHashMap {
         }
 
         private void shrinkToInitCapacity() {
-            long[] newTable = new long[4 * initCapacity];
+            long[] newTable = new long[ITEM_SIZE * initCapacity];
             Arrays.fill(newTable, EmptyKey);
 
             table = newTable;
@@ -622,7 +629,7 @@ public class ConcurrentLongLongPairHashMap {
                     return;
                 }
 
-                bucket = (bucket + 4) & (table.length - 1);
+                bucket = (bucket + ITEM_SIZE) & (table.length - 1);
             }
         }
     }
@@ -641,6 +648,8 @@ public class ConcurrentLongLongPairHashMap {
     }
 
     static final int signSafeMod(long n, int max) {
+        // as the ITEM_SIZE of Section is 4, so the index is the multiple of 4
+        // that is to left shift 2 bits
         return (int) (n & (max - 1)) << 2;
     }
 
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentLongPairSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentLongPairSet.java
index 2a109050385..f0f04c4edf9 100644
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentLongPairSet.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentLongPairSet.java
@@ -294,6 +294,9 @@ public class ConcurrentLongPairSet implements LongPairSet {
     // A section is a portion of the hash map that is covered by a single
     @SuppressWarnings("serial")
     private static final class Section extends StampedLock {
+        // Each item take up 2 continuous array space.
+        private static final int ITEM_SIZE = 2;
+
         // Keys and values are stored interleaved in the table array
         private volatile long[] table;
 
@@ -315,7 +318,7 @@ public class ConcurrentLongPairSet implements LongPairSet {
                 float expandFactor, float shrinkFactor) {
             this.capacity = alignToPowerOfTwo(capacity);
             this.initCapacity = this.capacity;
-            this.table = new long[2 * this.capacity];
+            this.table = new long[ITEM_SIZE * this.capacity];
             this.size = 0;
             this.usedBuckets = 0;
             this.autoShrink = autoShrink;
@@ -331,7 +334,11 @@ public class ConcurrentLongPairSet implements LongPairSet {
         boolean contains(long item1, long item2, int hash) {
             long stamp = tryOptimisticRead();
             boolean acquiredLock = false;
-            int bucket = signSafeMod(hash, capacity);
+
+            // add local variable here, so OutOfBound won't happen
+            long[] table = this.table;
+            // calculate table.length / 2 as capacity to avoid rehash changing capacity
+            int bucket = signSafeMod(hash, table.length / ITEM_SIZE);
 
             try {
                 while (true) {
@@ -353,7 +360,9 @@ public class ConcurrentLongPairSet implements LongPairSet {
                             stamp = readLock();
                             acquiredLock = true;
 
-                            bucket = signSafeMod(hash, capacity);
+                            // update local variable
+                            table = this.table;
+                            bucket = signSafeMod(hash, table.length / ITEM_SIZE);
                             storedItem1 = table[bucket];
                             storedItem2 = table[bucket + 1];
                         }
@@ -366,7 +375,7 @@ public class ConcurrentLongPairSet implements LongPairSet {
                         }
                     }
 
-                    bucket = (bucket + 2) & (table.length - 1);
+                    bucket = (bucket + ITEM_SIZE) & (table.length - 1);
                 }
             } finally {
                 if (acquiredLock) {
@@ -410,7 +419,7 @@ public class ConcurrentLongPairSet implements LongPairSet {
                         }
                     }
 
-                    bucket = (bucket + 2) & (table.length - 1);
+                    bucket = (bucket + ITEM_SIZE) & (table.length - 1);
                 }
             } finally {
                 if (usedBuckets > resizeThresholdUp) {
@@ -445,7 +454,7 @@ public class ConcurrentLongPairSet implements LongPairSet {
                         return false;
                     }
 
-                    bucket = (bucket + 2) & (table.length - 1);
+                    bucket = (bucket + ITEM_SIZE) & (table.length - 1);
                 }
             } finally {
                 tryShrinkThenUnlock(stamp);
@@ -459,7 +468,7 @@ public class ConcurrentLongPairSet implements LongPairSet {
             // Go through all the buckets for this section
             long stamp = writeLock();
             try {
-                for (int bucket = 0; bucket < table.length; bucket += 2) {
+                for (int bucket = 0; bucket < table.length; bucket += ITEM_SIZE) {
                     long storedItem1 = table[bucket];
                     long storedItem2 = table[bucket + 1];
                     if (storedItem1 != DeletedItem && storedItem1 != EmptyItem) {
@@ -498,7 +507,7 @@ public class ConcurrentLongPairSet implements LongPairSet {
         }
 
         private void cleanBucket(int bucket) {
-            int nextInArray = (bucket + 2) & (table.length - 1);
+            int nextInArray = (bucket + ITEM_SIZE) & (table.length - 1);
             if (table[nextInArray] == EmptyItem) {
                 table[bucket] = EmptyItem;
                 table[bucket + 1] = EmptyItem;
@@ -506,13 +515,13 @@ public class ConcurrentLongPairSet implements LongPairSet {
 
                 // Cleanup all the buckets that were in `DeletedItem` state,
                 // so that we can reduce unnecessary expansions
-                int lastBucket = (bucket - 2) & (table.length - 1);
+                int lastBucket = (bucket - ITEM_SIZE) & (table.length - 1);
                 while (table[lastBucket] == DeletedItem) {
                     table[lastBucket] = EmptyItem;
                     table[lastBucket + 1] = EmptyItem;
                     --usedBuckets;
 
-                    lastBucket = (lastBucket - 2) & (table.length - 1);
+                    lastBucket = (lastBucket - ITEM_SIZE) & (table.length - 1);
                 }
             } else {
                 table[bucket] = DeletedItem;
@@ -542,7 +551,7 @@ public class ConcurrentLongPairSet implements LongPairSet {
             // Go through all the buckets for this section. We try to renew the stamp only after a validation
             // error, otherwise we keep going with the same.
             long stamp = 0;
-            for (int bucket = 0; bucket < table.length; bucket += 2) {
+            for (int bucket = 0; bucket < table.length; bucket += ITEM_SIZE) {
                 if (stamp == 0) {
                     stamp = tryOptimisticRead();
                 }
@@ -572,11 +581,11 @@ public class ConcurrentLongPairSet implements LongPairSet {
 
         private void rehash(int newCapacity) {
             // Expand the hashmap
-            long[] newTable = new long[2 * newCapacity];
+            long[] newTable = new long[ITEM_SIZE * newCapacity];
             Arrays.fill(newTable, EmptyItem);
 
             // Re-hash table
-            for (int i = 0; i < table.length; i += 2) {
+            for (int i = 0; i < table.length; i += ITEM_SIZE) {
                 long storedItem1 = table[i];
                 long storedItem2 = table[i + 1];
                 if (storedItem1 != EmptyItem && storedItem1 != DeletedItem) {
@@ -595,7 +604,7 @@ public class ConcurrentLongPairSet implements LongPairSet {
 
         private void shrinkToInitCapacity() {
             // Expand the hashmap
-            long[] newTable = new long[2 * initCapacity];
+            long[] newTable = new long[ITEM_SIZE * initCapacity];
             Arrays.fill(newTable, EmptyItem);
 
             table = newTable;
@@ -621,7 +630,7 @@ public class ConcurrentLongPairSet implements LongPairSet {
                     return;
                 }
 
-                bucket = (bucket + 2) & (table.length - 1);
+                bucket = (bucket + ITEM_SIZE) & (table.length - 1);
             }
         }
     }
@@ -640,6 +649,8 @@ public class ConcurrentLongPairSet implements LongPairSet {
     }
 
     static final int signSafeMod(long n, int max) {
+        // as the ITEM_SIZE of Section is 2, so the index is the multiple of 2
+        // that is to left shift 1 bit
         return (int) (n & (max - 1)) << 1;
     }
 
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashMap.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashMap.java
index ea2e01768ac..1aa95d3090e 100644
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashMap.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashMap.java
@@ -296,6 +296,9 @@ public class ConcurrentOpenHashMap<K, V> {
     // A section is a portion of the hash map that is covered by a single
     @SuppressWarnings("serial")
     private static final class Section<K, V> extends StampedLock {
+        // Each item take up 2 continuous array space.
+        private static final int ITEM_SIZE = 2;
+
         // Keys and values are stored interleaved in the table array
         private volatile Object[] table;
 
@@ -317,7 +320,7 @@ public class ConcurrentOpenHashMap<K, V> {
                 float expandFactor, float shrinkFactor) {
             this.capacity = alignToPowerOfTwo(capacity);
             this.initCapacity = this.capacity;
-            this.table = new Object[2 * this.capacity];
+            this.table = new Object[ITEM_SIZE * this.capacity];
             this.size = 0;
             this.usedBuckets = 0;
             this.autoShrink = autoShrink;
@@ -332,7 +335,11 @@ public class ConcurrentOpenHashMap<K, V> {
         V get(K key, int keyHash) {
             long stamp = tryOptimisticRead();
             boolean acquiredLock = false;
-            int bucket = signSafeMod(keyHash, capacity);
+
+            // add local variable here, so OutOfBound won't happen
+            Object[] table = this.table;
+            // calculate table.length / 2 as capacity to avoid rehash changing capacity
+            int bucket = signSafeMod(keyHash, table.length / ITEM_SIZE);
 
             try {
                 while (true) {
@@ -354,7 +361,9 @@ public class ConcurrentOpenHashMap<K, V> {
                             stamp = readLock();
                             acquiredLock = true;
 
-                            bucket = signSafeMod(keyHash, capacity);
+                            // update local variable
+                            table = this.table;
+                            bucket = signSafeMod(keyHash, table.length / ITEM_SIZE);
                             storedKey = (K) table[bucket];
                             storedValue = (V) table[bucket + 1];
                         }
@@ -367,7 +376,7 @@ public class ConcurrentOpenHashMap<K, V> {
                         }
                     }
 
-                    bucket = (bucket + 2) & (table.length - 1);
+                    bucket = (bucket + ITEM_SIZE) & (table.length - 1);
                 }
             } finally {
                 if (acquiredLock) {
@@ -420,7 +429,7 @@ public class ConcurrentOpenHashMap<K, V> {
                         }
                     }
 
-                    bucket = (bucket + 2) & (table.length - 1);
+                    bucket = (bucket + ITEM_SIZE) & (table.length - 1);
                 }
             } finally {
                 if (usedBuckets > resizeThresholdUp) {
@@ -449,7 +458,7 @@ public class ConcurrentOpenHashMap<K, V> {
                         if (value == null || value.equals(storedValue)) {
                             SIZE_UPDATER.decrementAndGet(this);
 
-                            int nextInArray = (bucket + 2) & (table.length - 1);
+                            int nextInArray = (bucket + ITEM_SIZE) & (table.length - 1);
                             if (table[nextInArray] == EmptyKey) {
                                 table[bucket] = EmptyKey;
                                 table[bucket + 1] = null;
@@ -457,13 +466,13 @@ public class ConcurrentOpenHashMap<K, V> {
 
                                 // Cleanup all the buckets that were in `DeletedKey` state,
                                 // so that we can reduce unnecessary expansions
-                                int lastBucket = (bucket - 2) & (table.length - 1);
+                                int lastBucket = (bucket - ITEM_SIZE) & (table.length - 1);
                                 while (table[lastBucket] == DeletedKey) {
                                     table[lastBucket] = EmptyKey;
                                     table[lastBucket + 1] = null;
                                     --usedBuckets;
 
-                                    lastBucket = (lastBucket - 2) & (table.length - 1);
+                                    lastBucket = (lastBucket - ITEM_SIZE) & (table.length - 1);
                                 }
                             } else {
                                 table[bucket] = DeletedKey;
@@ -479,7 +488,7 @@ public class ConcurrentOpenHashMap<K, V> {
                         return null;
                     }
 
-                    bucket = (bucket + 2) & (table.length - 1);
+                    bucket = (bucket + ITEM_SIZE) & (table.length - 1);
                 }
 
             } finally {
@@ -528,7 +537,7 @@ public class ConcurrentOpenHashMap<K, V> {
             // Go through all the buckets for this section. We try to renew the stamp only after a validation
             // error, otherwise we keep going with the same.
             long stamp = 0;
-            for (int bucket = 0; bucket < table.length; bucket += 2) {
+            for (int bucket = 0; bucket < table.length; bucket += ITEM_SIZE) {
                 if (stamp == 0) {
                     stamp = tryOptimisticRead();
                 }
@@ -558,10 +567,10 @@ public class ConcurrentOpenHashMap<K, V> {
 
         private void rehash(int newCapacity) {
             // Expand the hashmap
-            Object[] newTable = new Object[2 * newCapacity];
+            Object[] newTable = new Object[ITEM_SIZE * newCapacity];
 
             // Re-hash table
-            for (int i = 0; i < table.length; i += 2) {
+            for (int i = 0; i < table.length; i += ITEM_SIZE) {
                 K storedKey = (K) table[i];
                 V storedValue = (V) table[i + 1];
                 if (storedKey != EmptyKey && storedKey != DeletedKey) {
@@ -577,7 +586,7 @@ public class ConcurrentOpenHashMap<K, V> {
         }
 
         private void shrinkToInitCapacity() {
-            Object[] newTable = new Object[2 * initCapacity];
+            Object[] newTable = new Object[ITEM_SIZE * initCapacity];
 
             table = newTable;
             size = 0;
@@ -602,7 +611,7 @@ public class ConcurrentOpenHashMap<K, V> {
                     return;
                 }
 
-                bucket = (bucket + 2) & (table.length - 1);
+                bucket = (bucket + ITEM_SIZE) & (table.length - 1);
             }
         }
     }
@@ -618,6 +627,8 @@ public class ConcurrentOpenHashMap<K, V> {
     }
 
     static final int signSafeMod(long n, int max) {
+        // as the ITEM_SIZE of Section is 2, so the index is the multiple of 2
+        // that is to left shift 1 bit
         return (int) (n & (max - 1)) << 1;
     }
 
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashSet.java
index cc8bc07b430..162cbed5a5d 100644
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashSet.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashSet.java
@@ -294,16 +294,16 @@ public class ConcurrentOpenHashSet<V> {
         }
 
         boolean contains(V value, int keyHash) {
-            int bucket = keyHash;
-
             long stamp = tryOptimisticRead();
             boolean acquiredLock = false;
 
+            // add local variable here, so OutOfBound won't happen
+            V[] values = this.values;
+            // calculate table.length as capacity to avoid rehash changing capacity
+            int bucket = signSafeMod(keyHash, values.length);
+
             try {
                 while (true) {
-                    int capacity = this.capacity;
-                    bucket = signSafeMod(bucket, capacity);
-
                     // First try optimistic locking
                     V storedValue = values[bucket];
 
@@ -321,15 +321,12 @@ public class ConcurrentOpenHashSet<V> {
                             stamp = readLock();
                             acquiredLock = true;
 
+                            // update local variable
+                            values = this.values;
+                            bucket = signSafeMod(keyHash, values.length);
                             storedValue = values[bucket];
                         }
 
-                        if (capacity != this.capacity) {
-                            // There has been a rehashing. We need to restart the search
-                            bucket = keyHash;
-                            continue;
-                        }
-
                         if (value.equals(storedValue)) {
                             return true;
                         } else if (storedValue == EmptyValue) {
@@ -337,8 +334,7 @@ public class ConcurrentOpenHashSet<V> {
                             return false;
                         }
                     }
-
-                    ++bucket;
+                    bucket = (bucket + 1) & (values.length - 1);
                 }
             } finally {
                 if (acquiredLock) {
diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/FutureUtilTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/FutureUtilTest.java
index 6df4494edf8..7d44c187d73 100644
--- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/FutureUtilTest.java
+++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/FutureUtilTest.java
@@ -181,7 +181,6 @@ public class FutureUtilTest {
         }
     }
 
-    @Test
     public void testSequencer() {
         int concurrentNum = 1000;
         final ScheduledExecutorService executor = Executors.newScheduledThreadPool(concurrentNum);
diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongHashMapTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongHashMapTest.java
index a317fa63c09..e1f947ad8c4 100644
--- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongHashMapTest.java
+++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongHashMapTest.java
@@ -38,6 +38,7 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.LongFunction;
 
 import lombok.Cleanup;
@@ -213,6 +214,66 @@ public class ConcurrentLongHashMapTest {
         assertTrue(map.capacity() == initCapacity);
     }
 
+    @Test
+    public void testConcurrentExpandAndShrinkAndGet()  throws Throwable {
+        ConcurrentLongHashMap<String> map = ConcurrentLongHashMap.<String>newBuilder()
+                .expectedItems(2)
+                .concurrencyLevel(1)
+                .autoShrink(true)
+                .mapIdleFactor(0.25f)
+                .build();
+        assertEquals(map.capacity(), 4);
+
+        ExecutorService executor = Executors.newCachedThreadPool();
+        final int readThreads = 16;
+        final int writeThreads = 1;
+        final int n = 1_000;
+        CyclicBarrier barrier = new CyclicBarrier(writeThreads + readThreads);
+        Future<?> future = null;
+        AtomicReference<Exception> ex = new AtomicReference<>();
+
+        for (int i = 0; i < readThreads; i++) {
+            executor.submit(() -> {
+                try {
+                    barrier.await();
+                } catch (Exception e) {
+                    throw new RuntimeException(e);
+                }
+                try {
+                    map.get(1);
+                } catch (Exception e) {
+                    ex.set(e);
+                }
+            });
+        }
+
+        assertNull(map.put(1,"v1"));
+        future = executor.submit(() -> {
+            try {
+                barrier.await();
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+
+            for (int i = 0; i < n; i++) {
+                // expand hashmap
+                assertNull(map.put(2, "v2"));
+                assertNull(map.put(3, "v3"));
+                assertEquals(map.capacity(), 8);
+
+                // shrink hashmap
+                assertTrue(map.remove(2, "v2"));
+                assertTrue(map.remove(3, "v3"));
+                assertEquals(map.capacity(), 4);
+            }
+        });
+
+        future.get();
+        assertTrue(ex.get() == null);
+        // shut down pool
+        executor.shutdown();
+    }
+
     @Test
     public void testRemove() {
         ConcurrentLongHashMap<String> map = ConcurrentLongHashMap.<String>newBuilder()
@@ -361,7 +422,6 @@ public class ConcurrentLongHashMapTest {
         assertEquals(map.size(), N * nThreads);
     }
 
-    @Test
     public void stressConcurrentInsertionsAndReads() throws Throwable {
         ConcurrentLongHashMap<String> map = ConcurrentLongHashMap.<String>newBuilder()
                 .expectedItems(4)
diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongLongPairHashMapTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongLongPairHashMapTest.java
index 8e74d285ffb..0de3fdb5c84 100644
--- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongLongPairHashMapTest.java
+++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongLongPairHashMapTest.java
@@ -31,9 +31,12 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
+import java.util.concurrent.CyclicBarrier;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicReference;
+
 import org.apache.pulsar.common.util.collections.ConcurrentLongLongPairHashMap.LongPair;
 import org.testng.annotations.Test;
 
@@ -173,6 +176,68 @@ public class ConcurrentLongLongPairHashMapTest {
         assertEquals(map.capacity(), 8);
     }
 
+    @Test
+    public void testConcurrentExpandAndShrinkAndGet()  throws Throwable {
+        ConcurrentLongLongPairHashMap map = ConcurrentLongLongPairHashMap.newBuilder()
+                .expectedItems(2)
+                .concurrencyLevel(1)
+                .autoShrink(true)
+                .mapIdleFactor(0.25f)
+                .build();
+        assertEquals(map.capacity(), 4);
+
+        ExecutorService executor = Executors.newCachedThreadPool();
+        final int readThreads = 16;
+        final int writeThreads = 1;
+        final int n = 1_000;
+        CyclicBarrier barrier = new CyclicBarrier(writeThreads + readThreads);
+        Future<?> future = null;
+        AtomicReference<Exception> ex = new AtomicReference<>();
+
+        for (int i = 0; i < readThreads; i++) {
+            executor.submit(() -> {
+                try {
+                    barrier.await();
+                } catch (Exception e) {
+                    throw new RuntimeException(e);
+                }
+                while (true) {
+                    try {
+                        map.get(1, 1);
+                    } catch (Exception e) {
+                        ex.set(e);
+                    }
+                }
+            });
+        }
+
+        assertTrue(map.put(1, 1, 11, 11));
+        future = executor.submit(() -> {
+            try {
+                barrier.await();
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+
+            for (int i = 0; i < n; i++) {
+                // expand hashmap
+                assertTrue(map.put(2, 2, 22, 22));
+                assertTrue(map.put(3, 3, 33, 33));
+                assertEquals(map.capacity(), 8);
+
+                // shrink hashmap
+                assertTrue(map.remove(2, 2, 22, 22));
+                assertTrue(map.remove(3, 3, 33, 33));
+                assertEquals(map.capacity(), 4);
+            }
+        });
+
+        future.get();
+        assertTrue(ex.get() == null);
+        // shut down pool
+        executor.shutdown();
+    }
+
     @Test
     public void testExpandShrinkAndClear() {
         ConcurrentLongLongPairHashMap map = ConcurrentLongLongPairHashMap.newBuilder()
diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongPairSetTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongPairSetTest.java
index 7e947ae6e6a..bce2b899383 100644
--- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongPairSetTest.java
+++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongPairSetTest.java
@@ -30,9 +30,11 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Random;
 import java.util.Set;
+import java.util.concurrent.CyclicBarrier;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicReference;
 
 import lombok.Cleanup;
 import org.apache.pulsar.common.util.collections.ConcurrentLongPairSet.LongPair;
@@ -211,6 +213,68 @@ public class ConcurrentLongPairSetTest {
         assertTrue(map.capacity() == initCapacity);
     }
 
+    @Test
+    public void testConcurrentExpandAndShrinkAndGet()  throws Throwable {
+        ConcurrentLongPairSet set = ConcurrentLongPairSet.newBuilder()
+                .expectedItems(2)
+                .concurrencyLevel(1)
+                .autoShrink(true)
+                .mapIdleFactor(0.25f)
+                .build();
+        assertEquals(set.capacity(), 4);
+
+        ExecutorService executor = Executors.newCachedThreadPool();
+        final int readThreads = 16;
+        final int writeThreads = 1;
+        final int n = 1_000;
+        CyclicBarrier barrier = new CyclicBarrier(writeThreads + readThreads);
+        Future<?> future = null;
+        AtomicReference<Exception> ex = new AtomicReference<>();
+
+        for (int i = 0; i < readThreads; i++) {
+            executor.submit(() -> {
+                try {
+                    barrier.await();
+                } catch (Exception e) {
+                    throw new RuntimeException(e);
+                }
+                while (true) {
+                    try {
+                        set.contains(1, 1);
+                    } catch (Exception e) {
+                        ex.set(e);
+                    }
+                }
+            });
+        }
+
+        assertTrue(set.add(1, 1));
+        future = executor.submit(() -> {
+            try {
+                barrier.await();
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+
+            for (int i = 0; i < n; i++) {
+                // expand hashmap
+                assertTrue(set.add(2, 2));
+                assertTrue(set.add(3, 3));
+                assertEquals(set.capacity(), 8);
+
+                // shrink hashmap
+                assertTrue(set.remove(2, 2));
+                assertTrue(set.remove(3, 3));
+                assertEquals(set.capacity(), 4);
+            }
+        });
+
+        future.get();
+        assertTrue(ex.get() == null);
+        // shut down pool
+        executor.shutdown();
+    }
+
 
     @Test
     public void testRemove() {
diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashMapTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashMapTest.java
index 198a3f4c5c3..410d490b98f 100644
--- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashMapTest.java
+++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashMapTest.java
@@ -32,11 +32,13 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Random;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CyclicBarrier;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.Function;
 
 import lombok.Cleanup;
@@ -215,6 +217,68 @@ public class ConcurrentOpenHashMapTest {
         assertTrue(map.capacity() == initCapacity);
     }
 
+    @Test
+    public void testConcurrentExpandAndShrinkAndGet()  throws Throwable {
+        ConcurrentOpenHashMap<String, String> map = ConcurrentOpenHashMap.<String, String>newBuilder()
+                .expectedItems(2)
+                .concurrencyLevel(1)
+                .autoShrink(true)
+                .mapIdleFactor(0.25f)
+                .build();
+        assertEquals(map.capacity(), 4);
+
+        ExecutorService executor = Executors.newCachedThreadPool();
+        final int readThreads = 16;
+        final int writeThreads = 1;
+        final int n = 1_000;
+        CyclicBarrier barrier = new CyclicBarrier(writeThreads + readThreads);
+        Future<?> future = null;
+        AtomicReference<Exception> ex = new AtomicReference<>();
+
+        for (int i = 0; i < readThreads; i++) {
+            executor.submit(() -> {
+                try {
+                    barrier.await();
+                } catch (Exception e) {
+                    throw new RuntimeException(e);
+                }
+                while (true) {
+                    try {
+                        map.get("k2");
+                    } catch (Exception e) {
+                        ex.set(e);
+                    }
+                }
+            });
+        }
+
+        assertNull(map.put("k1","v1"));
+        future = executor.submit(() -> {
+            try {
+                barrier.await();
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+
+            for (int i = 0; i < n; i++) {
+                // expand hashmap
+                assertNull(map.put("k2", "v2"));
+                assertNull(map.put("k3", "v3"));
+                assertEquals(map.capacity(), 8);
+
+                // shrink hashmap
+                assertTrue(map.remove("k2", "v2"));
+                assertTrue(map.remove("k3", "v3"));
+                assertEquals(map.capacity(), 4);
+            }
+        });
+
+        future.get();
+        assertTrue(ex.get() == null);
+        // shut down pool
+        executor.shutdown();
+    }
+
     @Test
     public void testRemove() {
         ConcurrentOpenHashMap<String, String> map =
diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashSetTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashSetTest.java
index 27c18abb8b3..6a40095ab06 100644
--- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashSetTest.java
+++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashSetTest.java
@@ -28,9 +28,11 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Random;
+import java.util.concurrent.CyclicBarrier;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicReference;
 
 import lombok.Cleanup;
 import org.testng.annotations.Test;
@@ -186,6 +188,68 @@ public class ConcurrentOpenHashSetTest {
         assertTrue(map.capacity() == initCapacity);
     }
 
+    @Test
+    public void testConcurrentExpandAndShrinkAndGet()  throws Throwable {
+        ConcurrentOpenHashSet<String> set = ConcurrentOpenHashSet.<String>newBuilder()
+                .expectedItems(2)
+                .concurrencyLevel(1)
+                .autoShrink(true)
+                .mapIdleFactor(0.25f)
+                .build();
+        assertEquals(set.capacity(), 4);
+
+        ExecutorService executor = Executors.newCachedThreadPool();
+        final int readThreads = 16;
+        final int writeThreads = 1;
+        final int n = 1_000;
+        CyclicBarrier barrier = new CyclicBarrier(writeThreads + readThreads);
+        Future<?> future = null;
+        AtomicReference<Exception> ex = new AtomicReference<>();
+
+        for (int i = 0; i < readThreads; i++) {
+            executor.submit(() -> {
+                try {
+                    barrier.await();
+                } catch (Exception e) {
+                    throw new RuntimeException(e);
+                }
+                while (true) {
+                    try {
+                        set.contains("k2");
+                    } catch (Exception e) {
+                        ex.set(e);
+                    }
+                }
+            });
+        }
+
+        assertTrue(set.add("k1"));
+        future = executor.submit(() -> {
+            try {
+                barrier.await();
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+
+            for (int i = 0; i < n; i++) {
+                // expand hashmap
+                assertTrue(set.add("k2"));
+                assertTrue(set.add("k3"));
+                assertEquals(set.capacity(), 8);
+
+                // shrink hashmap
+                assertTrue(set.remove("k2"));
+                assertTrue(set.remove("k3"));
+                assertEquals(set.capacity(), 4);
+            }
+        });
+
+        future.get();
+        assertTrue(ex.get() == null);
+        // shut down pool
+        executor.shutdown();
+    }
+
     @Test
     public void testRemove() {
         ConcurrentOpenHashSet<String> set =