You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by cm...@apache.org on 2019/08/15 23:00:45 UTC
[kafka] branch 2.3 updated: MINOR: Fix bugs in handling zero-length
ImplicitLinkedHashCollections (#7163)
This is an automated email from the ASF dual-hosted git repository.
cmccabe pushed a commit to branch 2.3
in repository https://gitbox.apache.org/repos/asf/kafka.git
The following commit(s) were added to refs/heads/2.3 by this push:
new 9cb27f5 MINOR: Fix bugs in handling zero-length ImplicitLinkedHashCollections (#7163)
9cb27f5 is described below
commit 9cb27f5f80b82aca65ccb89f3b9ba4d07222c9df
Author: Mickael Maison <mi...@users.noreply.github.com>
AuthorDate: Thu Aug 15 23:49:29 2019 +0100
MINOR: Fix bugs in handling zero-length ImplicitLinkedHashCollections (#7163)
Reviewers: Colin P. McCabe <cm...@apache.org>
---
.../kafka/common/utils/ImplicitLinkedHashCollection.java | 2 +-
.../common/utils/ImplicitLinkedHashMultiCollection.java | 4 ++--
.../kafka/common/utils/ImplicitLinkedHashCollectionTest.java | 12 +++++++++++-
.../common/utils/ImplicitLinkedHashMultiCollectionTest.java | 10 ++++++++++
4 files changed, 24 insertions(+), 4 deletions(-)
diff --git a/clients/src/main/java/org/apache/kafka/common/utils/ImplicitLinkedHashCollection.java b/clients/src/main/java/org/apache/kafka/common/utils/ImplicitLinkedHashCollection.java
index e060629..fba7d7a 100644
--- a/clients/src/main/java/org/apache/kafka/common/utils/ImplicitLinkedHashCollection.java
+++ b/clients/src/main/java/org/apache/kafka/common/utils/ImplicitLinkedHashCollection.java
@@ -304,7 +304,7 @@ public class ImplicitLinkedHashCollection<E extends ImplicitLinkedHashCollection
* @return The match index, or INVALID_INDEX if no match was found.
*/
final private int findIndexOfEqualElement(Object key) {
- if (key == null) {
+ if (key == null || size == 0) {
return INVALID_INDEX;
}
int slot = slot(elements, key);
diff --git a/clients/src/main/java/org/apache/kafka/common/utils/ImplicitLinkedHashMultiCollection.java b/clients/src/main/java/org/apache/kafka/common/utils/ImplicitLinkedHashMultiCollection.java
index 85714d6..b5ae8f9 100644
--- a/clients/src/main/java/org/apache/kafka/common/utils/ImplicitLinkedHashMultiCollection.java
+++ b/clients/src/main/java/org/apache/kafka/common/utils/ImplicitLinkedHashMultiCollection.java
@@ -91,7 +91,7 @@ public class ImplicitLinkedHashMultiCollection<E extends ImplicitLinkedHashColle
*/
@Override
int findElementToRemove(Object key) {
- if (key == null) {
+ if (key == null || size() == 0) {
return INVALID_INDEX;
}
int slot = slot(elements, key);
@@ -120,7 +120,7 @@ public class ImplicitLinkedHashMultiCollection<E extends ImplicitLinkedHashColle
* @return All of the matching elements.
*/
final public List<E> findAll(E key) {
- if (key == null) {
+ if (key == null || size() == 0) {
return Collections.<E>emptyList();
}
ArrayList<E> results = new ArrayList<>();
diff --git a/clients/src/test/java/org/apache/kafka/common/utils/ImplicitLinkedHashCollectionTest.java b/clients/src/test/java/org/apache/kafka/common/utils/ImplicitLinkedHashCollectionTest.java
index 8c102dd..389c24e 100644
--- a/clients/src/test/java/org/apache/kafka/common/utils/ImplicitLinkedHashCollectionTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/utils/ImplicitLinkedHashCollectionTest.java
@@ -31,6 +31,7 @@ import java.util.Random;
import java.util.Set;
import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertEquals;
@@ -271,7 +272,7 @@ public class ImplicitLinkedHashCollectionTest {
@Test
public void testEmptyListIterator() {
ImplicitLinkedHashCollection<TestElement> coll = new ImplicitLinkedHashCollection<>();
- ListIterator iter = coll.valuesList().listIterator();
+ ListIterator<TestElement> iter = coll.valuesList().listIterator();
assertFalse(iter.hasNext());
assertFalse(iter.hasPrevious());
assertEquals(0, iter.nextIndex());
@@ -513,6 +514,14 @@ public class ImplicitLinkedHashCollectionTest {
assertNotEquals(coll2, coll3);
}
+ @Test
+ public void testFindContainsRemoveOnEmptyCollection() {
+ ImplicitLinkedHashCollection<TestElement> coll = new ImplicitLinkedHashCollection<>();
+ assertNull(coll.find(new TestElement(2)));
+ assertFalse(coll.contains(new TestElement(2)));
+ assertFalse(coll.remove(new TestElement(2)));
+ }
+
private void addRandomElement(Random random, LinkedHashSet<Integer> existing,
ImplicitLinkedHashCollection<TestElement> set) {
int next;
@@ -523,6 +532,7 @@ public class ImplicitLinkedHashCollectionTest {
set.add(new TestElement(next));
}
+ @SuppressWarnings("unlikely-arg-type")
private void removeRandomElement(Random random, Collection<Integer> existing,
ImplicitLinkedHashCollection<TestElement> coll) {
int removeIdx = random.nextInt(existing.size());
diff --git a/clients/src/test/java/org/apache/kafka/common/utils/ImplicitLinkedHashMultiCollectionTest.java b/clients/src/test/java/org/apache/kafka/common/utils/ImplicitLinkedHashMultiCollectionTest.java
index 8d2b850..ad87b55 100644
--- a/clients/src/test/java/org/apache/kafka/common/utils/ImplicitLinkedHashMultiCollectionTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/utils/ImplicitLinkedHashMultiCollectionTest.java
@@ -28,6 +28,7 @@ import java.util.Random;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@@ -45,6 +46,15 @@ public class ImplicitLinkedHashMultiCollectionTest {
}
@Test
+ public void testFindFindAllContainsRemoveOnEmptyCollection() {
+ ImplicitLinkedHashMultiCollection<TestElement> coll = new ImplicitLinkedHashMultiCollection<>();
+ assertNull(coll.find(new TestElement(2)));
+ assertFalse(coll.contains(new TestElement(2)));
+ assertFalse(coll.remove(new TestElement(2)));
+ assertTrue(coll.findAll(new TestElement(2)).isEmpty());
+ }
+
+ @Test
public void testInsertDelete() {
ImplicitLinkedHashMultiCollection<TestElement> multiSet = new ImplicitLinkedHashMultiCollection<>(100);
TestElement e1 = new TestElement(1);