You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2016/01/21 23:39:20 UTC

[43/50] [abbrv] calcite git commit: [CALCITE-1047] ChunkList.clear throws AssertionError

[CALCITE-1047] ChunkList.clear throws AssertionError

Re-work ChunkList with a more robust implementation of ListIterator and more thorough tests.


Project: http://git-wip-us.apache.org/repos/asf/calcite/repo
Commit: http://git-wip-us.apache.org/repos/asf/calcite/commit/3cba7055
Tree: http://git-wip-us.apache.org/repos/asf/calcite/tree/3cba7055
Diff: http://git-wip-us.apache.org/repos/asf/calcite/diff/3cba7055

Branch: refs/heads/branch-release
Commit: 3cba7055061108ce0e079d8756bcfb020b8628c3
Parents: 2712d7d
Author: Julian Hyde <jh...@apache.org>
Authored: Tue Jan 12 20:14:51 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Jan 13 08:43:29 2016 -0800

----------------------------------------------------------------------
 .../java/org/apache/calcite/util/ChunkList.java | 243 ++++++++++++-------
 .../org/apache/calcite/util/ChunkListTest.java  | 217 ++++++++++++++---
 2 files changed, 340 insertions(+), 120 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/3cba7055/core/src/main/java/org/apache/calcite/util/ChunkList.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/ChunkList.java b/core/src/main/java/org/apache/calcite/util/ChunkList.java
index 66644e9..8d7c9bf 100644
--- a/core/src/main/java/org/apache/calcite/util/ChunkList.java
+++ b/core/src/main/java/org/apache/calcite/util/ChunkList.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.util;
 
 import java.util.AbstractSequentialList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.LinkedList;
 import java.util.ListIterator;
@@ -33,10 +34,6 @@ import java.util.NoSuchElementException;
  */
 public class ChunkList<E> extends AbstractSequentialList<E> {
   private static final int HEADER_SIZE = 3;
-  private int size;
-  private Object[] first;
-  private Object[] last;
-
   private static final int CHUNK_SIZE = 64;
   private static final Integer[] INTEGERS = new Integer[CHUNK_SIZE + 3];
 
@@ -46,6 +43,10 @@ public class ChunkList<E> extends AbstractSequentialList<E> {
     }
   }
 
+  private int size;
+  private Object[] first;
+  private Object[] last;
+
   /**
    * Creates an empty ChunkList.
    */
@@ -105,6 +106,12 @@ public class ChunkList<E> extends AbstractSequentialList<E> {
     return size;
   }
 
+  @Override public void clear() {
+    // base class method works, but let's optimize
+    size = 0;
+    first = last = null;
+  }
+
   @Override public boolean add(E element) {
     Object[] chunk = last;
     int occupied;
@@ -169,18 +176,20 @@ public class ChunkList<E> extends AbstractSequentialList<E> {
   }
 
   private ChunkListIterator locate(int index) {
-    if (index == 0) {
-      return new ChunkListIterator();
+    if (index < 0 || index > size) {
+      throw new IndexOutOfBoundsException();
+    }
+    if (first == null) {
+      // Create an iterator positioned before the first element.
+      return new ChunkListIterator(null, 0, 0, -1, 0);
     }
     int n = 0;
     for (Object[] chunk = first;;) {
       final int occupied = occupied(chunk);
       final int nextN = n + occupied;
       final Object[] next = next(chunk);
-      if (nextN > index || next == null) {
-        return new ChunkListIterator(
-            chunk, n, index - n - 1 + HEADER_SIZE,
-            occupied + HEADER_SIZE);
+      if (nextN >= index || next == null) {
+        return new ChunkListIterator(chunk, n, index, -1, n + occupied);
       }
       n = nextN;
       chunk = next;
@@ -190,76 +199,87 @@ public class ChunkList<E> extends AbstractSequentialList<E> {
   /** Iterator over a {@link ChunkList}. */
   private class ChunkListIterator implements ListIterator<E> {
     private Object[] chunk;
-    private int startIndex;
-    private int offset;
+    /** Offset in the list of the first element of this chunk. */
+    private int start;
+    /** Offset within current chunk of the next element to return. */
+    private int cursor;
+    /** Offset within the current chunk of the last element returned. -1 if
+     * {@link #next} or {@link #previous()} has not been called. */
+    private int lastRet;
+    /** Offset of the first unoccupied location in the current chunk. */
     private int end;
 
-    ChunkListIterator() {
-      this(null, 0, -1, 0);
-    }
-
-    ChunkListIterator(Object[] chunk, int startIndex, int offset, int end) {
+    ChunkListIterator(Object[] chunk, int start, int cursor, int lastRet,
+        int end) {
       this.chunk = chunk;
-      this.startIndex = startIndex;
-      this.offset = offset;
+      this.start = start;
+      this.cursor = cursor;
+      this.lastRet = lastRet;
       this.end = end;
     }
 
     public boolean hasNext() {
-      return offset + 1 < end
-          || (chunk == null
-          ? first != null
-          : ChunkList.next(chunk) != null);
+      return cursor < size;
     }
 
     public E next() {
-      ++offset;
-      assert offset <= end;
-      if (offset == end) {
+      if (cursor >= size) {
+        throw new NoSuchElementException();
+      }
+      if (cursor == end) {
         if (chunk == null) {
           chunk = first;
         } else {
           chunk = ChunkList.next(chunk);
-          startIndex += end - HEADER_SIZE;
         }
+        start = end;
         if (chunk == null) {
-          throw new NoSuchElementException();
+          end = start;
+        } else {
+          end = start + occupied(chunk);
         }
-        offset = HEADER_SIZE;
-        end = occupied(chunk) + HEADER_SIZE;
       }
-      return (E) element(chunk, offset);
+      @SuppressWarnings("unchecked")
+      final E element = (E) element(chunk,
+          HEADER_SIZE + (lastRet = cursor++) - start);
+      return element;
     }
 
     public boolean hasPrevious() {
-      return offset >= HEADER_SIZE || ChunkList.prev(chunk) != null;
+      return cursor > 0;
     }
 
     public E previous() {
-      --offset;
-      if (offset == HEADER_SIZE - 1) {
+      lastRet = cursor--;
+      if (cursor < start) {
         chunk = chunk == null ? last : ChunkList.prev(chunk);
         if (chunk == null) {
           throw new NoSuchElementException();
         }
-        end = occupied(chunk);
-        startIndex -= end;
-        offset = end - 1;
+        final int o = occupied(chunk);
+        end = start;
+        start -= o;
+        assert cursor == end - 1;
       }
-      return (E) element(chunk, offset);
+      //noinspection unchecked
+      return (E) element(chunk, cursor - start);
     }
 
     public int nextIndex() {
-      return startIndex + (offset - HEADER_SIZE) + 1;
+      return cursor;
     }
 
     public int previousIndex() {
-      return startIndex + (offset - HEADER_SIZE);
+      return cursor - 1;
     }
 
     public void remove() {
+      if (lastRet < 0) {
+        throw new IllegalStateException();
+      }
       --size;
-      if (end == HEADER_SIZE + 1) {
+      --cursor;
+      if (end == start + 1) {
         // Chunk is now empty.
         final Object[] prev = prev(chunk);
         final Object[] next = ChunkList.next(chunk);
@@ -272,80 +292,125 @@ public class ChunkList<E> extends AbstractSequentialList<E> {
           }
           chunk = null;
           end = HEADER_SIZE;
-          offset = end - 1;
         } else {
           if (prev == null) {
-            first = next;
+            chunk = first = next;
             setPrev(next, null);
+            end = occupied(chunk);
           } else {
             setNext(prev, next);
             setPrev(next, prev);
+            chunk = prev;
+            end = start;
+            start -= occupied(chunk);
           }
-          chunk = next;
-          offset = HEADER_SIZE;
-          end = HEADER_SIZE + occupied(next);
         }
+        lastRet = -1;
         return;
       }
-      // Move existing contents down one.
-      System.arraycopy(
-          chunk, offset + 1, chunk, offset, end - offset - 1);
-      --end;
-      setElement(chunk, end, null); // allow gc
-      setOccupied(chunk, end - HEADER_SIZE);
-      if (offset == end) {
-        final Object[] next = ChunkList.next(chunk);
-        if (next != null) {
-          startIndex += end - HEADER_SIZE;
-          chunk = next;
-          offset = HEADER_SIZE - 1;
-          end = HEADER_SIZE + occupied(next);
+      final int r = lastRet;
+      lastRet = -1;
+      if (r < start) {
+        // Element we wish to eliminate is the last element in the previous
+        // block.
+        Object[] c = chunk;
+        if (c == null) {
+          c = last;
         }
+        int o = occupied(c);
+        if (o == 1) {
+          // Block is now empty; remove it
+          final Object[] prev = prev(c);
+          if (prev == null) {
+            if (chunk == null) {
+              first = last = null;
+            } else {
+              first = chunk;
+              setPrev(chunk, null);
+            }
+          } else {
+            setNext(prev, chunk);
+            setPrev(chunk, prev);
+          }
+        } else {
+          --o;
+          setElement(c, HEADER_SIZE + o, null); // allow gc
+          setOccupied(c, o);
+        }
+      } else {
+        // Move existing contents down one.
+        System.arraycopy(chunk, HEADER_SIZE + r - start + 1,
+            chunk, HEADER_SIZE + r - start, end - r - 1);
+        --end;
+        final int o = end - start;
+        setElement(chunk, HEADER_SIZE + o, null); // allow gc
+        setOccupied(chunk, o);
       }
     }
 
     public void set(E e) {
-      setElement(chunk, offset, e);
+      if (lastRet < 0) {
+        throw new IllegalStateException();
+      }
+      Object[] c = chunk;
+      int p = lastRet;
+      int s = start;
+      if (p < start) {
+        // The element is at the end of the previous chunk
+        c = prev(c);
+        s -= occupied(c);
+      }
+      setElement(c, HEADER_SIZE + p - s, e);
     }
 
     public void add(E e) {
-      if (chunk == null || end == CHUNK_SIZE + HEADER_SIZE) {
+      if (chunk == null) {
+        Object[] newChunk = new Object[CHUNK_SIZE + HEADER_SIZE];
+        if (first != null) {
+          setNext(newChunk, first);
+          setPrev(first, newChunk);
+        }
+        first = newChunk;
+        if (last == null) {
+          last = newChunk;
+        }
+        chunk = newChunk;
+        end = start;
+      } else if (end == start + CHUNK_SIZE) {
         // FIXME We create a new chunk, but the next chunk might be
         // less than half full. We should consider using it.
         Object[] newChunk = new Object[CHUNK_SIZE + HEADER_SIZE];
-        if (chunk == null) {
-          if (first != null) {
-            setNext(newChunk, first);
-            setPrev(first, newChunk);
-          }
-          first = newChunk;
-          if (last == null) {
-            last = newChunk;
-          }
+        final Object[] next = ChunkList.next(chunk);
+        setPrev(newChunk, chunk);
+        setNext(chunk, newChunk);
+
+        if (next == null) {
+          last = newChunk;
         } else {
-          final Object[] next = ChunkList.next(chunk);
-          setPrev(newChunk, chunk);
-          setNext(chunk, newChunk);
+          setPrev(next, newChunk);
+          setNext(newChunk, next);
+        }
 
-          if (next == null) {
-            last = newChunk;
-          } else {
-            setPrev(next, newChunk);
-            setNext(newChunk, next);
-          }
-          startIndex += CHUNK_SIZE;
+        setOccupied(chunk, CHUNK_SIZE / 2);
+        setOccupied(newChunk, CHUNK_SIZE / 2);
+        System.arraycopy(chunk, HEADER_SIZE + CHUNK_SIZE / 2,
+            newChunk, HEADER_SIZE, CHUNK_SIZE / 2);
+        Arrays.fill(chunk, HEADER_SIZE + CHUNK_SIZE / 2,
+            HEADER_SIZE + CHUNK_SIZE, null);
+
+        if (cursor - start < CHUNK_SIZE / 2) {
+          end -= CHUNK_SIZE / 2;
+        } else {
+          start += CHUNK_SIZE / 2;
+          chunk = newChunk;
         }
-        chunk = newChunk;
-        end = offset = HEADER_SIZE;
-      } else {
-        // Move existing contents up one.
-        System.arraycopy(
-            chunk, offset, chunk, offset + 1, end - offset);
       }
-      setElement(chunk, offset, e);
-//            ++offset;
+      // Move existing contents up one.
+      System.arraycopy(chunk, HEADER_SIZE + cursor - start,
+          chunk, HEADER_SIZE + cursor - start + 1, end - cursor);
       ++end;
-      setOccupied(chunk, end - HEADER_SIZE);
+      setElement(chunk, HEADER_SIZE + cursor - start, e);
+      setOccupied(chunk, end - start);
       ++size;
     }
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/3cba7055/core/src/test/java/org/apache/calcite/util/ChunkListTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/ChunkListTest.java b/core/src/test/java/org/apache/calcite/util/ChunkListTest.java
index 0e2bb2d..efb1b09 100644
--- a/core/src/test/java/org/apache/calcite/util/ChunkListTest.java
+++ b/core/src/test/java/org/apache/calcite/util/ChunkListTest.java
@@ -19,6 +19,8 @@ package org.apache.calcite.util;
 import org.apache.calcite.linq4j.function.Function0;
 import org.apache.calcite.linq4j.function.Function1;
 
+import com.google.common.collect.ImmutableList;
+
 import org.junit.Test;
 
 import java.util.ArrayList;
@@ -30,9 +32,11 @@ import java.util.List;
 import java.util.ListIterator;
 import java.util.Random;
 
+import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -44,8 +48,13 @@ public class ChunkListTest {
    * Unit test for {@link ChunkList}.
    */
   @Test public void testChunkList() {
-    final ChunkList<Integer> list = new ChunkList<Integer>();
+    final ChunkList<Integer> list = new ChunkList<>();
+    final ChunkList<Integer> list0 = new ChunkList<>(list);
+    final ChunkList<Integer> list1 = new ChunkList<>(list);
+    list1.add(123);
     assertEquals(0, list.size());
+    assertEquals(0, list0.size());
+    assertEquals(1, list1.size());
     assertTrue(list.isEmpty());
     assertEquals("[]", list.toString());
 
@@ -56,6 +65,20 @@ public class ChunkListTest {
       // ok
     }
 
+    try {
+      list.get(-1);
+      fail("expected exception");
+    } catch (IndexOutOfBoundsException e) {
+      // ok
+    }
+
+    try {
+      list.get(0);
+      fail("expected exception");
+    } catch (IndexOutOfBoundsException e) {
+      // ok
+    }
+
     list.add(7);
     assertEquals(1, list.size());
     assertEquals(7, (int) list.get(0));
@@ -76,9 +99,10 @@ public class ChunkListTest {
     assertTrue(list.contains(9));
     assertFalse(list.contains(8));
 
-    list.addAll(Collections.nCopies(40, 1));
-    assertEquals(44, list.size());
+    list.addAll(Collections.nCopies(70, 1));
+    assertEquals(74, list.size());
     assertEquals(1, (int) list.get(40));
+    assertEquals(1, (int) list.get(70));
 
     int n = 0;
     for (Integer integer : list) {
@@ -96,44 +120,142 @@ public class ChunkListTest {
     i = list.indexOf(null);
     assertEquals(-1, i);
 
+    // sort an empty list
+    Collections.sort(list0);
+    assertThat(list0.isEmpty(), is(true));
+
+    // sort a list with 1 element
+    Collections.sort(list1);
+    assertThat(list1.size(), is(1));
+
     Collections.sort(list);
+    assertEquals(74, list.size());
 
     list.remove((Integer) 7);
     Collections.sort(list);
     assertEquals(1, (int) list.get(3));
 
     // remove all instances of a value that exists
-    boolean b = list.removeAll(Arrays.asList(9));
+    boolean b = list.removeAll(Collections.singletonList(9));
     assertTrue(b);
 
     // remove all instances of a non-existent value
-    b = list.removeAll(Arrays.asList(99));
+    b = list.removeAll(Collections.singletonList(99));
     assertFalse(b);
 
     // remove all instances of a value that occurs in the last chunk
     list.add(12345);
-    b = list.removeAll(Arrays.asList(12345));
+    b = list.removeAll(Collections.singletonList(12345));
     assertTrue(b);
 
     // remove all instances of a value that occurs in the last chunk but
     // not as the last value
     list.add(12345);
     list.add(123);
-    b = list.removeAll(Arrays.asList(12345));
+    b = list.removeAll(Collections.singletonList(12345));
     assertTrue(b);
 
-    assertEquals(
-        1000, new ChunkList<Integer>(Collections.nCopies(1000, 77)).size());
+    assertThat(new ChunkList<>(Collections.nCopies(1000, 77)).size(),
+        is(1000));
 
     // add to an empty list via iterator
     //noinspection MismatchedQueryAndUpdateOfCollection
-    final ChunkList<String> list2 = new ChunkList<String>();
+    final ChunkList<String> list2 = new ChunkList<>();
     list2.listIterator(0).add("x");
     assertEquals("[x]", list2.toString());
 
     // add at start
     list2.add(0, "y");
     assertEquals("[y, x]", list2.toString());
+
+    list2.remove(0);
+    assertEquals("[x]", list2.toString());
+
+    // clear a list of length 5, one element at a time, using an iterator
+    list2.clear();
+    list2.addAll(ImmutableList.of("a", "b", "c", "d", "e"));
+    assertThat(list2.size(), is(5));
+    final ListIterator<String> listIterator = list2.listIterator(0);
+    assertThat(listIterator.next(), is("a"));
+    listIterator.remove();
+    assertThat(listIterator.next(), is("b"));
+    listIterator.remove();
+    assertThat(listIterator.next(), is("c"));
+    listIterator.remove();
+    assertThat(listIterator.next(), is("d"));
+    listIterator.remove();
+    assertThat(list2.size(), is(1));
+    assertThat(listIterator.next(), is("e"));
+    listIterator.remove();
+    assertThat(list2.size(), is(0));
+  }
+
+  /** Clears lists of various sizes. */
+  @Test public void testClear() {
+    checkListClear(0);
+    checkListClear(1);
+    checkListClear(2);
+    checkListClear(32);
+    checkListClear(64);
+    checkListClear(65);
+    checkListClear(66);
+    checkListClear(100);
+    checkListClear(127);
+    checkListClear(128);
+    checkListClear(129);
+  }
+
+  private void checkListClear(int n) {
+    for (int i = 0; i < 4; i++) {
+      ChunkList<String> list = new ChunkList<>(Collections.nCopies(n, "z"));
+      assertThat(list.size(), is(n));
+      switch (i) {
+      case 0:
+        list.clear();
+        break;
+      case 1:
+        for (int j = 0; j < n; j++) {
+          list.remove(0);
+        }
+        break;
+      case 2:
+        for (int j = 0; j < n; j++) {
+          list.remove(list.size() - 1);
+        }
+        break;
+      case 3:
+        Random random = new Random();
+        for (int j = 0; j < n; j++) {
+          list.remove(random.nextInt(list.size()));
+        }
+        break;
+      }
+      assertThat(list.isEmpty(), is(true));
+    }
+  }
+
+  /**
+   * Removing via an iterator.
+   */
+  @Test public void testIterator() {
+    final ChunkList<String> list = new ChunkList<>();
+    list.add("a");
+    list.add("b");
+    final ListIterator<String> listIterator = list.listIterator(0);
+    try {
+      listIterator.remove();
+      fail("excepted exception");
+    } catch (IllegalStateException e) {
+      // ok
+    }
+    listIterator.next();
+    listIterator.remove();
+    assertThat(list.size(), is(1));
+    assertThat(listIterator.hasNext(), is(true));
+    listIterator.next();
+    listIterator.remove();
+    assertThat(list.size(), is(0));
+    assertThat(listIterator.hasNext(), is(false));
   }
 
   /**
@@ -142,41 +264,52 @@ public class ChunkListTest {
    */
   @Test public void testRandom() {
     final int iterationCount = 10000;
-    checkRandom(new Random(1), new ChunkList<Integer>(), iterationCount);
+    checkRandom(new Random(1), new ChunkList<Integer>(),
+        new ArrayList<Integer>(), iterationCount);
     final Random random = new Random(2);
     for (int j = 0; j < 10; j++) {
-      checkRandom(random, new ChunkList<Integer>(), iterationCount);
+      checkRandom(random, new ChunkList<Integer>(), new ArrayList<Integer>(),
+          iterationCount);
     }
-    checkRandom(
-        new Random(3), new ChunkList<Integer>(Collections.nCopies(1000, 5)),
-        iterationCount);
+    final ChunkList<Integer> chunkList =
+        new ChunkList<>(Collections.nCopies(1000, 5));
+    final List<Integer> referenceList = new ArrayList<>(chunkList);
+    checkRandom(new Random(3), chunkList, referenceList, iterationCount);
   }
 
   void checkRandom(
       Random random,
       ChunkList<Integer> list,
+      List<Integer> list2,
       int iterationCount) {
     int removeCount = 0;
     int addCount = 0;
+    int size;
+    int e;
     final int initialCount = list.size();
     for (int i = 0; i < iterationCount; i++) {
       assert list.isValid(true);
-      switch (random.nextInt(8)) {
+      switch (random.nextInt(10)) {
       case 0:
         // remove last
         if (!list.isEmpty()) {
+          assertThat(list2.isEmpty(), is(false));
           list.remove(list.size() - 1);
+          list2.remove(list2.size() - 1);
           ++removeCount;
         }
         break;
       case 1:
         // add to end
-        list.add(random.nextInt(1000));
+        e = random.nextInt(1000);
+        list.add(e);
+        list2.add(e);
         ++addCount;
         break;
       case 2:
         int n = 0;
-        final int size = list.size();
+        size = list.size();
+        assertThat(list.size(), is(list2.size()));
         for (Integer integer : list) {
           Util.discard(integer);
           assertTrue(n++ < size);
@@ -184,20 +317,26 @@ public class ChunkListTest {
         break;
       case 3:
         // remove all instances of a particular value
-        int sizeBefore = list.size();
-        boolean b = list.removeAll(
-            Collections.singletonList(random.nextInt(500)));
+        size = list.size();
+        final List<Integer> zz = Collections.singletonList(random.nextInt(500));
+        boolean b = list.removeAll(zz);
+        boolean b2 = list2.removeAll(zz);
+        assertThat(b, is(b2));
         if (b) {
-          assertTrue(list.size() < sizeBefore);
+          assertTrue(list.size() < size);
+          assertTrue(list2.size() < size);
         } else {
-          assertTrue(list.size() == sizeBefore);
+          assertTrue(list.size() == size);
+          assertTrue(list2.size() == size);
         }
-        removeCount += sizeBefore - list.size();
+        removeCount += size - list.size();
         break;
       case 4:
         // remove at random position
         if (!list.isEmpty()) {
-          list.remove(random.nextInt(list.size()));
+          e = random.nextInt(list.size());
+          list.remove(e);
+          list2.remove(e);
           ++removeCount;
         }
         break;
@@ -205,19 +344,35 @@ public class ChunkListTest {
         // add at random position
         int count = random.nextInt(list.size() + 1);
         ListIterator<Integer> it = list.listIterator();
+        ListIterator<Integer> it2 = list2.listIterator();
         for (int j = 0; j < count; j++) {
           it.next();
+          it2.next();
         }
-        it.add(list.size());
+        size = list.size();
+        it.add(size);
+        it2.add(size);
         ++addCount;
         break;
+      case 6:
+        // clear
+        if (random.nextInt(200) == 0) {
+          removeCount += list.size();
+          list.clear();
+          list2.clear();
+        }
+        break;
       default:
         // add at random position
-        list.add(random.nextInt(list.size() + 1), list.size());
+        int pos = random.nextInt(list.size() + 1);
+        e = list.size();
+        list.add(pos, e);
+        list2.add(pos, e);
         ++addCount;
         break;
       }
       assertEquals(list.size(), initialCount + addCount - removeCount);
+      assertEquals(list, list2);
     }
   }
 
@@ -231,22 +386,22 @@ public class ChunkListTest {
             Arrays.asList(
                 new Function0<List<Integer>>() {
                   public List<Integer> apply() {
-                    return new ArrayList<Integer>();
+                    return new ArrayList<>();
                   }
                 },
                 new Function0<List<Integer>>() {
                   public List<Integer> apply() {
-                    return new LinkedList<Integer>();
+                    return new LinkedList<>();
                   }
                 },
                 new Function0<List<Integer>>() {
                   public List<Integer> apply() {
-                    return new ChunkList<Integer>();
+                    return new ChunkList<>();
                   }
                 }),
             Arrays.asList("ArrayList", "LinkedList", "ChunkList-64"));
     final List<Pair<Function0<List<Integer>>, String>> factories1 =
-        new ArrayList<Pair<Function0<List<Integer>>, String>>();
+        new ArrayList<>();
     for (Pair<Function0<List<Integer>>, String> pair : factories0) {
       factories1.add(pair);
     }