You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by te...@apache.org on 2022/07/26 04:44:39 UTC
[pulsar] branch master updated: Optimize concurrent collection's shrink logic (#16754)
This is an automated email from the ASF dual-hosted git repository.
technoboy pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/pulsar.git
The following commit(s) were added to refs/heads/master by this push:
new 00a0bcf7dea Optimize concurrent collection's shrink logic (#16754)
00a0bcf7dea is described below
commit 00a0bcf7deaa43e3a395eb75fffd658cb223af24
Author: wenbingshen <ol...@gmail.com>
AuthorDate: Tue Jul 26 12:44:32 2022 +0800
Optimize concurrent collection's shrink logic (#16754)
---
.../util/collections/ConcurrentLongHashMap.java | 34 +++++++++++++++++-----
.../collections/ConcurrentLongLongPairHashMap.java | 31 ++++++++++++++++----
.../util/collections/ConcurrentLongPairSet.java | 32 ++++++++++++++++----
.../util/collections/ConcurrentOpenHashMap.java | 30 +++++++++++++++----
.../util/collections/ConcurrentOpenHashSet.java | 30 +++++++++++++++----
.../collections/ConcurrentLongHashMapTest.java | 33 +++++++++++++++++++++
.../ConcurrentLongLongPairHashMapTest.java | 32 ++++++++++++++++++++
.../collections/ConcurrentLongPairSetTest.java | 34 ++++++++++++++++++++++
.../collections/ConcurrentOpenHashMapTest.java | 33 +++++++++++++++++++++
.../collections/ConcurrentOpenHashSetTest.java | 34 ++++++++++++++++++++++
10 files changed, 292 insertions(+), 31 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 90aa61a6d9b..62303015c39 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
@@ -480,7 +480,11 @@ public class ConcurrentLongHashMap<V> {
} finally {
if (autoShrink && size < resizeThresholdBelow) {
try {
- int newCapacity = alignToPowerOfTwo((int) (capacity / shrinkFactor));
+ // Shrinking must at least ensure initCapacity,
+ // so as to avoid frequent shrinking and expansion near initCapacity,
+ // frequent shrinking and expansion,
+ // additionally opened arrays will consume more memory and affect GC
+ int newCapacity = Math.max(alignToPowerOfTwo((int) (capacity / shrinkFactor)), initCapacity);
int newResizeThresholdUp = (int) (newCapacity * mapFillFactor);
if (newCapacity < capacity && newResizeThresholdUp > size) {
// shrink the hashmap
@@ -499,12 +503,13 @@ public class ConcurrentLongHashMap<V> {
long stamp = writeLock();
try {
- Arrays.fill(keys, 0);
- Arrays.fill(values, EmptyValue);
- this.size = 0;
- this.usedBuckets = 0;
- if (autoShrink) {
- rehash(initCapacity);
+ if (autoShrink && capacity > initCapacity) {
+ shrinkToInitCapacity();
+ } else {
+ Arrays.fill(keys, 0);
+ Arrays.fill(values, EmptyValue);
+ this.size = 0;
+ this.usedBuckets = 0;
}
} finally {
unlockWrite(stamp);
@@ -582,6 +587,21 @@ public class ConcurrentLongHashMap<V> {
resizeThresholdBelow = (int) (capacity * mapIdleFactor);
}
+ private void shrinkToInitCapacity() {
+ long[] newKeys = new long[initCapacity];
+ V[] newValues = (V[]) new Object[initCapacity];
+
+ keys = newKeys;
+ values = newValues;
+ size = 0;
+ usedBuckets = 0;
+ // Capacity needs to be updated after the values, so that we won't see
+ // a capacity value bigger than the actual array size
+ capacity = initCapacity;
+ resizeThresholdUp = (int) (capacity * mapFillFactor);
+ resizeThresholdBelow = (int) (capacity * mapIdleFactor);
+ }
+
private static <V> void insertKeyValueNoLock(long[] keys, V[] values, long key, V value) {
int bucket = (int) hash(key);
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 eac7268ba67..cce8be0b7fd 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
@@ -460,7 +460,11 @@ public class ConcurrentLongLongPairHashMap {
} finally {
if (autoShrink && size < resizeThresholdBelow) {
try {
- int newCapacity = alignToPowerOfTwo((int) (capacity / shrinkFactor));
+ // Shrinking must at least ensure initCapacity,
+ // so as to avoid frequent shrinking and expansion near initCapacity,
+ // frequent shrinking and expansion,
+ // additionally opened arrays will consume more memory and affect GC
+ int newCapacity = Math.max(alignToPowerOfTwo((int) (capacity / shrinkFactor)), initCapacity);
int newResizeThresholdUp = (int) (newCapacity * mapFillFactor);
if (newCapacity < capacity && newResizeThresholdUp > size) {
// shrink the hashmap
@@ -507,11 +511,12 @@ public class ConcurrentLongLongPairHashMap {
long stamp = writeLock();
try {
- Arrays.fill(table, EmptyKey);
- this.size = 0;
- this.usedBuckets = 0;
- if (autoShrink) {
- rehash(initCapacity);
+ if (autoShrink && capacity > initCapacity) {
+ shrinkToInitCapacity();
+ } else {
+ Arrays.fill(table, EmptyKey);
+ this.size = 0;
+ this.usedBuckets = 0;
}
} finally {
unlockWrite(stamp);
@@ -587,6 +592,20 @@ public class ConcurrentLongLongPairHashMap {
resizeThresholdBelow = (int) (capacity * mapIdleFactor);
}
+ private void shrinkToInitCapacity() {
+ long[] newTable = new long[4 * initCapacity];
+ Arrays.fill(newTable, EmptyKey);
+
+ table = newTable;
+ size = 0;
+ usedBuckets = 0;
+ // Capacity needs to be updated after the values, so that we won't see
+ // a capacity value bigger than the actual array size
+ capacity = initCapacity;
+ resizeThresholdUp = (int) (capacity * mapFillFactor);
+ resizeThresholdBelow = (int) (capacity * mapIdleFactor);
+ }
+
private static void insertKeyValueNoLock(long[] table, int capacity, long key1, long key2, long value1,
long value2) {
int bucket = signSafeMod(hash(key1, key2), capacity);
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 7b5e75813fa..1a892dbd966 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
@@ -479,7 +479,11 @@ public class ConcurrentLongPairSet implements LongPairSet {
private void tryShrinkThenUnlock(long stamp) {
if (autoShrink && size < resizeThresholdBelow) {
try {
- int newCapacity = alignToPowerOfTwo((int) (capacity / shrinkFactor));
+ // Shrinking must at least ensure initCapacity,
+ // so as to avoid frequent shrinking and expansion near initCapacity,
+ // frequent shrinking and expansion,
+ // additionally opened arrays will consume more memory and affect GC
+ int newCapacity = Math.max(alignToPowerOfTwo((int) (capacity / shrinkFactor)), initCapacity);
int newResizeThresholdUp = (int) (newCapacity * mapFillFactor);
if (newCapacity < capacity && newResizeThresholdUp > size) {
// shrink the hashmap
@@ -520,11 +524,12 @@ public class ConcurrentLongPairSet implements LongPairSet {
long stamp = writeLock();
try {
- Arrays.fill(table, EmptyItem);
- this.size = 0;
- this.usedBuckets = 0;
- if (autoShrink) {
- rehash(initCapacity);
+ if (autoShrink && capacity > initCapacity) {
+ shrinkToInitCapacity();
+ } else {
+ Arrays.fill(table, EmptyItem);
+ this.size = 0;
+ this.usedBuckets = 0;
}
} finally {
unlockWrite(stamp);
@@ -588,6 +593,21 @@ public class ConcurrentLongPairSet implements LongPairSet {
resizeThresholdBelow = (int) (capacity * mapIdleFactor);
}
+ private void shrinkToInitCapacity() {
+ // Expand the hashmap
+ long[] newTable = new long[2 * initCapacity];
+ Arrays.fill(newTable, EmptyItem);
+
+ table = newTable;
+ size = 0;
+ usedBuckets = 0;
+ // Capacity needs to be updated after the values, so that we won't see
+ // a capacity value bigger than the actual array size
+ capacity = initCapacity;
+ resizeThresholdUp = (int) (capacity * mapFillFactor);
+ resizeThresholdBelow = (int) (capacity * mapIdleFactor);
+ }
+
private static void insertKeyValueNoLock(long[] table, int capacity, long item1, long item2) {
int bucket = signSafeMod(hash(item1, item2), capacity);
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 e039079eeb3..618b3e539a9 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
@@ -480,7 +480,11 @@ public class ConcurrentOpenHashMap<K, V> {
} finally {
if (autoShrink && size < resizeThresholdBelow) {
try {
- int newCapacity = alignToPowerOfTwo((int) (capacity / shrinkFactor));
+ // Shrinking must at least ensure initCapacity,
+ // so as to avoid frequent shrinking and expansion near initCapacity,
+ // frequent shrinking and expansion,
+ // additionally opened arrays will consume more memory and affect GC
+ int newCapacity = Math.max(alignToPowerOfTwo((int) (capacity / shrinkFactor)), initCapacity);
int newResizeThresholdUp = (int) (newCapacity * mapFillFactor);
if (newCapacity < capacity && newResizeThresholdUp > size) {
// shrink the hashmap
@@ -499,11 +503,12 @@ public class ConcurrentOpenHashMap<K, V> {
long stamp = writeLock();
try {
- Arrays.fill(table, EmptyKey);
- this.size = 0;
- this.usedBuckets = 0;
- if (autoShrink) {
- rehash(initCapacity);
+ if (autoShrink && capacity > initCapacity) {
+ shrinkToInitCapacity();
+ } else {
+ Arrays.fill(table, EmptyKey);
+ this.size = 0;
+ this.usedBuckets = 0;
}
} finally {
unlockWrite(stamp);
@@ -566,6 +571,19 @@ public class ConcurrentOpenHashMap<K, V> {
resizeThresholdBelow = (int) (capacity * mapIdleFactor);
}
+ private void shrinkToInitCapacity() {
+ Object[] newTable = new Object[2 * initCapacity];
+
+ table = newTable;
+ size = 0;
+ usedBuckets = 0;
+ // Capacity needs to be updated after the values, so that we won't see
+ // a capacity value bigger than the actual array size
+ capacity = initCapacity;
+ resizeThresholdUp = (int) (capacity * mapFillFactor);
+ resizeThresholdBelow = (int) (capacity * mapIdleFactor);
+ }
+
private static <K, V> void insertKeyValueNoLock(Object[] table, int capacity, K key, V value) {
int bucket = signSafeMod(hash(key), capacity);
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 6dd6e6a4b63..a5333adae2f 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
@@ -425,7 +425,11 @@ public class ConcurrentOpenHashSet<V> {
} finally {
if (autoShrink && size < resizeThresholdBelow) {
try {
- int newCapacity = alignToPowerOfTwo((int) (capacity / shrinkFactor));
+ // Shrinking must at least ensure initCapacity,
+ // so as to avoid frequent shrinking and expansion near initCapacity,
+ // frequent shrinking and expansion,
+ // additionally opened arrays will consume more memory and affect GC
+ int newCapacity = Math.max(alignToPowerOfTwo((int) (capacity / shrinkFactor)), initCapacity);
int newResizeThresholdUp = (int) (newCapacity * mapFillFactor);
if (newCapacity < capacity && newResizeThresholdUp > size) {
// shrink the hashmap
@@ -444,11 +448,12 @@ public class ConcurrentOpenHashSet<V> {
long stamp = writeLock();
try {
- Arrays.fill(values, EmptyValue);
- this.size = 0;
- this.usedBuckets = 0;
- if (autoShrink) {
- rehash(initCapacity);
+ if (autoShrink && capacity > initCapacity) {
+ shrinkToInitCapacity();
+ } else {
+ Arrays.fill(values, EmptyValue);
+ this.size = 0;
+ this.usedBuckets = 0;
}
} finally {
unlockWrite(stamp);
@@ -551,6 +556,19 @@ public class ConcurrentOpenHashSet<V> {
resizeThresholdBelow = (int) (capacity * mapIdleFactor);
}
+ private void shrinkToInitCapacity() {
+ V[] newValues = (V[]) new Object[initCapacity];
+
+ values = newValues;
+ size = 0;
+ usedBuckets = 0;
+ // Capacity needs to be updated after the values, so that we won't see
+ // a capacity value bigger than the actual array size
+ capacity = initCapacity;
+ resizeThresholdUp = (int) (capacity * mapFillFactor);
+ resizeThresholdBelow = (int) (capacity * mapIdleFactor);
+ }
+
private static <V> void insertValueNoLock(V[] values, V value) {
int bucket = (int) hash(value);
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 205cf91b47d..f60fea7ab7f 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
@@ -180,6 +180,39 @@ public class ConcurrentLongHashMapTest {
assertTrue(map.capacity() == 8);
}
+ @Test
+ public void testExpandShrinkAndClear() {
+ ConcurrentLongHashMap<String> map = ConcurrentLongHashMap.<String>newBuilder()
+ .expectedItems(2)
+ .concurrencyLevel(1)
+ .autoShrink(true)
+ .mapIdleFactor(0.25f)
+ .build();
+ final long initCapacity = map.capacity();
+ assertTrue(map.capacity() == 4);
+ assertNull(map.put(1, "v1"));
+ assertNull(map.put(2, "v2"));
+ assertNull(map.put(3, "v3"));
+
+ // expand hashmap
+ assertTrue(map.capacity() == 8);
+
+ assertTrue(map.remove(1, "v1"));
+ // not shrink
+ assertTrue(map.capacity() == 8);
+ assertTrue(map.remove(2, "v2"));
+ // shrink hashmap
+ assertTrue(map.capacity() == 4);
+
+ assertTrue(map.remove(3, "v3"));
+ // Will not shrink the hashmap again because shrink capacity is less than initCapacity
+ // current capacity is equal than the initial capacity
+ assertTrue(map.capacity() == initCapacity);
+ map.clear();
+ // after clear, because current capacity is equal than the initial capacity, so not shrinkToInitCapacity
+ assertTrue(map.capacity() == initCapacity);
+ }
+
@Test
public void testRemove() {
ConcurrentLongHashMap<String> map = ConcurrentLongHashMap.<String>newBuilder()
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 1245784e1a0..eca63734915 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
@@ -173,6 +173,38 @@ public class ConcurrentLongLongPairHashMapTest {
assertEquals(map.capacity(), 8);
}
+ @Test
+ public void testExpandShrinkAndClear() {
+ ConcurrentLongLongPairHashMap map = ConcurrentLongLongPairHashMap.newBuilder()
+ .expectedItems(2)
+ .concurrencyLevel(1)
+ .autoShrink(true)
+ .mapIdleFactor(0.25f)
+ .build();
+ final long initCapacity = map.capacity();
+ assertTrue(map.put(1, 1, 11, 11));
+ assertTrue(map.put(2, 2, 22, 22));
+ assertTrue(map.put(3, 3, 33, 33));
+
+ // expand hashmap
+ assertTrue(map.capacity() == 8);
+
+ assertTrue(map.remove(1, 1, 11, 11));
+ // not shrink
+ assertTrue(map.capacity() == 8);
+ assertTrue(map.remove(2, 2, 22, 22));
+ // shrink hashmap
+ assertTrue(map.capacity() == 4);
+
+ assertTrue(map.remove(3, 3, 33, 33));
+ // Will not shrink the hashmap again because shrink capacity is less than initCapacity
+ // current capacity is equal than the initial capacity
+ assertTrue(map.capacity() == initCapacity);
+ map.clear();
+ // after clear, because current capacity is equal than the initial capacity, so not shrinkToInitCapacity
+ assertTrue(map.capacity() == initCapacity);
+ }
+
@Test
public void testNegativeUsedBucketCount() {
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 86030f21619..a7bbe9b0dc3 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
@@ -177,6 +177,40 @@ public class ConcurrentLongPairSetTest {
assertTrue(map.capacity() == 8);
}
+ @Test
+ public void testExpandShrinkAndClear() {
+ ConcurrentLongPairSet map = ConcurrentLongPairSet.newBuilder()
+ .expectedItems(2)
+ .concurrencyLevel(1)
+ .autoShrink(true)
+ .mapIdleFactor(0.25f)
+ .build();
+ final long initCapacity = map.capacity();
+ assertTrue(map.capacity() == 4);
+
+ assertTrue(map.add(1, 1));
+ assertTrue(map.add(2, 2));
+ assertTrue(map.add(3, 3));
+
+ // expand hashmap
+ assertTrue(map.capacity() == 8);
+
+ assertTrue(map.remove(1, 1));
+ // not shrink
+ assertTrue(map.capacity() == 8);
+ assertTrue(map.remove(2, 2));
+ // shrink hashmap
+ assertTrue(map.capacity() == 4);
+
+ assertTrue(map.remove(3, 3));
+ // Will not shrink the hashmap again because shrink capacity is less than initCapacity
+ // current capacity is equal than the initial capacity
+ assertTrue(map.capacity() == initCapacity);
+ map.clear();
+ // after clear, because current capacity is equal than the initial capacity, so not shrinkToInitCapacity
+ assertTrue(map.capacity() == initCapacity);
+ }
+
@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 cec52ea3ded..411dfc735fd 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
@@ -182,6 +182,39 @@ public class ConcurrentOpenHashMapTest {
assertTrue(map.capacity() == 8);
}
+ @Test
+ public void testExpandShrinkAndClear() {
+ ConcurrentOpenHashMap<String, String> map = ConcurrentOpenHashMap.<String, String>newBuilder()
+ .expectedItems(2)
+ .concurrencyLevel(1)
+ .autoShrink(true)
+ .mapIdleFactor(0.25f)
+ .build();
+ final long initCapacity = map.capacity();
+ assertTrue(map.capacity() == 4);
+ assertNull(map.put("k1", "v1"));
+ assertNull(map.put("k2", "v2"));
+ assertNull(map.put("k3", "v3"));
+
+ // expand hashmap
+ assertTrue(map.capacity() == 8);
+
+ assertTrue(map.remove("k1", "v1"));
+ // not shrink
+ assertTrue(map.capacity() == 8);
+ assertTrue(map.remove("k2", "v2"));
+ // shrink hashmap
+ assertTrue(map.capacity() == 4);
+
+ assertTrue(map.remove("k3", "v3"));
+ // Will not shrink the hashmap again because shrink capacity is less than initCapacity
+ // current capacity is equal than the initial capacity
+ assertTrue(map.capacity() == initCapacity);
+ map.clear();
+ // after clear, because current capacity is equal than the initial capacity, so not shrinkToInitCapacity
+ assertTrue(map.capacity() == initCapacity);
+ }
+
@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 12cad783f81..d6403af52cf 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
@@ -152,6 +152,40 @@ public class ConcurrentOpenHashSetTest {
assertEquals(map.capacity(), 8);
}
+ @Test
+ public void testExpandShrinkAndClear() {
+ ConcurrentOpenHashSet<String> map = ConcurrentOpenHashSet.<String>newBuilder()
+ .expectedItems(2)
+ .concurrencyLevel(1)
+ .autoShrink(true)
+ .mapIdleFactor(0.25f)
+ .build();
+ final long initCapacity = map.capacity();
+ assertTrue(map.capacity() == 4);
+
+ assertTrue(map.add("k1"));
+ assertTrue(map.add("k2"));
+ assertTrue(map.add("k3"));
+
+ // expand hashmap
+ assertTrue(map.capacity() == 8);
+
+ assertTrue(map.remove("k1"));
+ // not shrink
+ assertTrue(map.capacity() == 8);
+ assertTrue(map.remove("k2"));
+ // shrink hashmap
+ assertTrue(map.capacity() == 4);
+
+ assertTrue(map.remove("k3"));
+ // Will not shrink the hashmap again because shrink capacity is less than initCapacity
+ // current capacity is equal than the initial capacity
+ assertTrue(map.capacity() == initCapacity);
+ map.clear();
+ // after clear, because current capacity is equal than the initial capacity, so not shrinkToInitCapacity
+ assertTrue(map.capacity() == initCapacity);
+ }
+
@Test
public void testRemove() {
ConcurrentOpenHashSet<String> set =