You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by yu...@apache.org on 2015/11/11 23:18:17 UTC

[07/15] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Merge branch 'cassandra-2.1' into cassandra-2.2


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

Branch: refs/heads/cassandra-2.2
Commit: e487553575d95cb5fcf28a98a8be6d4b8a26bced
Parents: 6bb6bb0 1c3ff92
Author: Yuki Morishita <yu...@apache.org>
Authored: Wed Nov 11 16:13:58 2015 -0600
Committer: Yuki Morishita <yu...@apache.org>
Committed: Wed Nov 11 16:13:58 2015 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/cassandra/db/ColumnFamilyStore.java  | 35 +++++++++++
 .../db/compaction/CompactionController.java     |  5 --
 src/java/org/apache/cassandra/dht/Bounds.java   | 62 ++++++++++++++++++++
 .../cassandra/io/sstable/SSTableRewriter.java   |  1 -
 .../cassandra/streaming/StreamReader.java       |  1 -
 .../cassandra/streaming/StreamReceiveTask.java  | 35 +++++++++++
 .../apache/cassandra/db/CounterCacheTest.java   | 45 ++++++++++++++
 .../org/apache/cassandra/db/RowCacheTest.java   | 51 ++++++++++++++++
 .../org/apache/cassandra/dht/BoundsTest.java    | 61 +++++++++++++++++++
 10 files changed, 290 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e4875535/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 0557786,92244a0..0fcf037
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,15 -1,5 +1,16 @@@
 -2.1.12
 +2.2.4
 + * (Hadoop) fix splits calculation (CASSANDRA-10640)
 + * (Hadoop) ensure that Cluster instances are always closed (CASSANDRA-10058)
 + * (cqlsh) show partial trace if incomplete after max_trace_wait (CASSANDRA-7645)
 + * Use most up-to-date version of schema for system tables (CASSANDRA-10652)
 + * Deprecate memory_allocator in cassandra.yaml (CASSANDRA-10581,10628)
 + * Expose phi values from failure detector via JMX and tweak debug
 +   and trace logging (CASSANDRA-9526)
 + * Fix RangeNamesQueryPager (CASSANDRA-10509)
 + * Deprecate Pig support (CASSANDRA-10542)
 + * Reduce contention getting instances of CompositeType (CASSANDRA-10433)
 +Merged from 2.1:
+  * Invalidate cache after stream receive task is completed (CASSANDRA-10341)
   * Reject counter writes in CQLSSTableWriter (CASSANDRA-10258)
   * Remove superfluous COUNTER_MUTATION stage mapping (CASSANDRA-10605)
   * Improve json2sstable error reporting on nonexistent columns (CASSANDRA-10401)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e4875535/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index d553f4d,54f6fff..2d58219
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@@ -2519,6 -2505,37 +2519,41 @@@ public class ColumnFamilyStore implemen
              CacheService.instance.invalidateCounterCacheForCf(metadata.ksAndCFName);
      }
  
+     public int invalidateRowCache(Collection<Bounds<Token>> boundsToInvalidate)
+     {
+         int invalidatedKeys = 0;
 -        for (RowCacheKey key : CacheService.instance.rowCache.getKeySet())
++        for (Iterator<RowCacheKey> keyIter = CacheService.instance.rowCache.keyIterator();
++             keyIter.hasNext(); )
+         {
++            RowCacheKey key = keyIter.next();
+             DecoratedKey dk = partitioner.decorateKey(ByteBuffer.wrap(key.key));
+             if (key.ksAndCFName.equals(metadata.ksAndCFName) && Bounds.isInBounds(dk.getToken(), boundsToInvalidate))
+             {
+                 invalidateCachedRow(dk);
+                 invalidatedKeys++;
+             }
+         }
+ 
+         return invalidatedKeys;
+     }
+ 
+     public int invalidateCounterCache(Collection<Bounds<Token>> boundsToInvalidate)
+     {
+         int invalidatedKeys = 0;
 -        for (CounterCacheKey key : CacheService.instance.counterCache.getKeySet())
++        for (Iterator<CounterCacheKey> keyIter = CacheService.instance.counterCache.keyIterator();
++             keyIter.hasNext(); )
+         {
++            CounterCacheKey key = keyIter.next();
+             DecoratedKey dk = partitioner.decorateKey(ByteBuffer.wrap(key.partitionKey));
+             if (key.ksAndCFName.equals(metadata.ksAndCFName) && Bounds.isInBounds(dk.getToken(), boundsToInvalidate))
+             {
+                 CacheService.instance.counterCache.remove(key);
+                 invalidatedKeys++;
+             }
+         }
+         return invalidatedKeys;
+     }
+ 
      /**
       * @return true if @param key is contained in the row cache
       */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e4875535/src/java/org/apache/cassandra/db/compaction/CompactionController.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e4875535/src/java/org/apache/cassandra/dht/Bounds.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/dht/Bounds.java
index 9060bcf,5ffde42..73414cd
--- a/src/java/org/apache/cassandra/dht/Bounds.java
+++ b/src/java/org/apache/cassandra/dht/Bounds.java
@@@ -17,10 -17,20 +17,19 @@@
   */
  package org.apache.cassandra.dht;
  
+ import java.util.ArrayList;
+ import java.util.Collection;
  import java.util.Collections;
+ import java.util.Comparator;
  import java.util.List;
+ import java.util.Set;
+ 
+ import com.google.common.collect.Iterators;
+ import com.google.common.collect.Lists;
+ import com.google.common.collect.PeekingIterator;
+ import com.google.common.collect.Sets;
  
  import org.apache.cassandra.db.RowPosition;
 -import org.apache.cassandra.service.StorageService;
  import org.apache.cassandra.utils.Pair;
  
  /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e4875535/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
index dc4fe75,af5d1d3..c62b827
--- a/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
@@@ -20,7 -20,8 +20,6 @@@ package org.apache.cassandra.io.sstable
  import java.util.*;
  
  import com.google.common.annotations.VisibleForTesting;
- import com.google.common.util.concurrent.Runnables;
 -import com.google.common.base.Throwables;
 -import com.google.common.collect.ImmutableList;
  
  import org.apache.cassandra.cache.InstrumentingCache;
  import org.apache.cassandra.cache.KeyCacheKey;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e4875535/src/java/org/apache/cassandra/streaming/StreamReader.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e4875535/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
index d4d49b3,738c93c..846524b
--- a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
@@@ -23,16 -23,23 +23,22 @@@ import java.io.IOException
  import java.util.ArrayList;
  import java.util.Collection;
  import java.util.List;
+ import java.util.Set;
  import java.util.UUID;
 -import java.util.concurrent.ThreadPoolExecutor;
 -import java.util.concurrent.TimeUnit;
 +import java.util.concurrent.ExecutorService;
 +import java.util.concurrent.Executors;
  
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ 
 -import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
 +import org.apache.cassandra.concurrent.NamedThreadFactory;
  import org.apache.cassandra.config.Schema;
  import org.apache.cassandra.db.ColumnFamilyStore;
  import org.apache.cassandra.db.Keyspace;
+ import org.apache.cassandra.dht.Bounds;
+ import org.apache.cassandra.dht.Token;
 -import org.apache.cassandra.io.sstable.SSTableReader;
 -import org.apache.cassandra.io.sstable.SSTableWriter;
 -import org.apache.cassandra.utils.FBUtilities;
 +import org.apache.cassandra.io.sstable.format.SSTableReader;
 +import org.apache.cassandra.io.sstable.format.SSTableWriter;
  import org.apache.cassandra.utils.Pair;
  
  import org.apache.cassandra.utils.concurrent.Refs;
@@@ -42,7 -49,11 +48,8 @@@
   */
  public class StreamReceiveTask extends StreamTask
  {
 -    private static final ThreadPoolExecutor executor = DebuggableThreadPoolExecutor.createWithMaximumPoolSize("StreamReceiveTask",
 -                                                                                                              FBUtilities.getAvailableProcessors(),
 -                                                                                                              60, TimeUnit.SECONDS);
 -
 +    private static final ExecutorService executor = Executors.newCachedThreadPool(new NamedThreadFactory("StreamReceiveTask"));
+     private static final Logger logger = LoggerFactory.getLogger(StreamReceiveTask.class);
  
      // number of files to receive
      private final int totalFiles;
@@@ -131,6 -143,33 +139,33 @@@
                  // add sstables and build secondary indexes
                  cfs.addSSTables(readers);
                  cfs.indexManager.maybeBuildSecondaryIndexes(readers, cfs.indexManager.allIndexesNames());
+ 
+                 //invalidate row and counter cache
+                 if (cfs.isRowCacheEnabled() || cfs.metadata.isCounter())
+                 {
+                     List<Bounds<Token>> boundsToInvalidate = new ArrayList<>(readers.size());
+                     for (SSTableReader sstable : readers)
+                         boundsToInvalidate.add(new Bounds<Token>(sstable.first.getToken(), sstable.last.getToken()));
+                     Set<Bounds<Token>> nonOverlappingBounds = Bounds.getNonOverlappingBounds(boundsToInvalidate);
+ 
+                     if (cfs.isRowCacheEnabled())
+                     {
+                         int invalidatedKeys = cfs.invalidateRowCache(nonOverlappingBounds);
+                         if (invalidatedKeys > 0)
+                             logger.debug("[Stream #{}] Invalidated {} row cache entries on table {}.{} after stream " +
+                                          "receive task completed.", task.session.planId(), invalidatedKeys,
+                                          cfs.keyspace.getName(), cfs.getColumnFamilyName());
+                     }
+ 
+                     if (cfs.metadata.isCounter())
+                     {
+                         int invalidatedKeys = cfs.invalidateCounterCache(nonOverlappingBounds);
+                         if (invalidatedKeys > 0)
+                             logger.debug("[Stream #{}] Invalidated {} counter cache entries on table {}.{} after stream " +
 -                                        "receive task completed.", task.session.planId(), invalidatedKeys,
 -                                        cfs.keyspace.getName(), cfs.getColumnFamilyName());
++                                         "receive task completed.", task.session.planId(), invalidatedKeys,
++                                         cfs.keyspace.getName(), cfs.getColumnFamilyName());
+                     }
+                 }
              }
  
              task.session.taskCompleted(task);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e4875535/test/unit/org/apache/cassandra/db/CounterCacheTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/CounterCacheTest.java
index 5b37b2c,542358d..ed7921e
--- a/test/unit/org/apache/cassandra/db/CounterCacheTest.java
+++ b/test/unit/org/apache/cassandra/db/CounterCacheTest.java
@@@ -24,12 -24,10 +25,14 @@@ import org.junit.BeforeClass
  import org.junit.Test;
  
  import org.apache.cassandra.SchemaLoader;
 -import org.apache.cassandra.config.Schema;
 +import org.apache.cassandra.config.KSMetaData;
 +import org.apache.cassandra.db.marshal.CounterColumnType;
+ import org.apache.cassandra.dht.Bounds;
+ import org.apache.cassandra.dht.Token;
 +import org.apache.cassandra.exceptions.ConfigurationException;
 +import org.apache.cassandra.config.Schema;
  import org.apache.cassandra.exceptions.WriteTimeoutException;
 +import org.apache.cassandra.locator.SimpleStrategy;
  import org.apache.cassandra.service.CacheService;
  import org.apache.cassandra.utils.FBUtilities;
  
@@@ -86,9 -74,51 +89,51 @@@ public class CounterCacheTes
      }
  
      @Test
+     public void testCounterCacheInvalidate()
+     {
 -        ColumnFamilyStore cfs = Keyspace.open(KS).getColumnFamilyStore(CF);
++        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF);
+         cfs.truncateBlocking();
+         CacheService.instance.invalidateCounterCache();
+ 
+         assertEquals(0, CacheService.instance.counterCache.size());
+         assertNull(cfs.getCachedCounter(bytes(1), cellname(1)));
+         assertNull(cfs.getCachedCounter(bytes(1), cellname(2)));
+         assertNull(cfs.getCachedCounter(bytes(2), cellname(1)));
+         assertNull(cfs.getCachedCounter(bytes(2), cellname(2)));
+         assertNull(cfs.getCachedCounter(bytes(3), cellname(1)));
+         assertNull(cfs.getCachedCounter(bytes(3), cellname(2)));
+ 
+         cfs.putCachedCounter(bytes(1), cellname(1), ClockAndCount.create(1L, 1L));
+         cfs.putCachedCounter(bytes(1), cellname(2), ClockAndCount.create(1L, 2L));
+         cfs.putCachedCounter(bytes(2), cellname(1), ClockAndCount.create(2L, 1L));
+         cfs.putCachedCounter(bytes(2), cellname(2), ClockAndCount.create(2L, 2L));
+         cfs.putCachedCounter(bytes(3), cellname(1), ClockAndCount.create(3L, 1L));
+         cfs.putCachedCounter(bytes(3), cellname(2), ClockAndCount.create(3L, 2L));
+ 
+         assertEquals(6, CacheService.instance.counterCache.size());
+         assertEquals(ClockAndCount.create(1L, 1L), cfs.getCachedCounter(bytes(1), cellname(1)));
+         assertEquals(ClockAndCount.create(1L, 2L), cfs.getCachedCounter(bytes(1), cellname(2)));
+         assertEquals(ClockAndCount.create(2L, 1L), cfs.getCachedCounter(bytes(2), cellname(1)));
+         assertEquals(ClockAndCount.create(2L, 2L), cfs.getCachedCounter(bytes(2), cellname(2)));
+         assertEquals(ClockAndCount.create(3L, 1L), cfs.getCachedCounter(bytes(3), cellname(1)));
+         assertEquals(ClockAndCount.create(3L, 2L), cfs.getCachedCounter(bytes(3), cellname(2)));
+ 
+         cfs.invalidateCounterCache(Collections.singleton(new Bounds<Token>(cfs.partitioner.decorateKey(bytes(1)).getToken(),
+                                                                            cfs.partitioner.decorateKey(bytes(2)).getToken())));
+ 
+         assertEquals(2, CacheService.instance.counterCache.size());
+         assertNull(cfs.getCachedCounter(bytes(1), cellname(1)));
+         assertNull(cfs.getCachedCounter(bytes(1), cellname(2)));
+         assertNull(cfs.getCachedCounter(bytes(2), cellname(1)));
+         assertNull(cfs.getCachedCounter(bytes(2), cellname(2)));
+         assertEquals(ClockAndCount.create(3L, 1L), cfs.getCachedCounter(bytes(3), cellname(1)));
+         assertEquals(ClockAndCount.create(3L, 2L), cfs.getCachedCounter(bytes(3), cellname(2)));
+     }
+ 
+     @Test
      public void testSaveLoad() throws ExecutionException, InterruptedException, WriteTimeoutException
      {
 -        ColumnFamilyStore cfs = Keyspace.open(KS).getColumnFamilyStore(CF);
 +        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF);
          cfs.truncateBlocking();
          CacheService.instance.invalidateCounterCache();
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e4875535/test/unit/org/apache/cassandra/db/RowCacheTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/RowCacheTest.java
index 5912d7c,719c771..9fb322b
--- a/test/unit/org/apache/cassandra/db/RowCacheTest.java
+++ b/test/unit/org/apache/cassandra/db/RowCacheTest.java
@@@ -20,10 -20,13 +20,14 @@@ package org.apache.cassandra.db
  
  import java.net.InetAddress;
  import java.nio.ByteBuffer;
+ import java.util.ArrayList;
  import java.util.Collection;
+ import java.util.Iterator;
+ import java.util.TreeSet;
  
+ import com.google.common.collect.Lists;
  import org.junit.AfterClass;
 +import org.junit.BeforeClass;
  import org.junit.Test;
  
  import org.apache.cassandra.SchemaLoader;
@@@ -35,11 -36,10 +39,13 @@@ import org.apache.cassandra.config.Sche
  import org.apache.cassandra.db.composites.*;
  import org.apache.cassandra.db.compaction.CompactionManager;
  import org.apache.cassandra.db.filter.QueryFilter;
 +import org.apache.cassandra.db.marshal.IntegerType;
+ import org.apache.cassandra.dht.Bounds;
 -import org.apache.cassandra.dht.BytesToken;
+ import org.apache.cassandra.dht.Token;
 +import org.apache.cassandra.exceptions.ConfigurationException;
 +import org.apache.cassandra.dht.ByteOrderedPartitioner.BytesToken;
  import org.apache.cassandra.locator.TokenMetadata;
 +import org.apache.cassandra.locator.SimpleStrategy;
  import org.apache.cassandra.service.CacheService;
  import org.apache.cassandra.service.StorageService;
  import org.apache.cassandra.utils.ByteBufferUtil;
@@@ -172,6 -158,51 +178,51 @@@ public class RowCacheTes
      }
  
      @Test
+     public void testInvalidateRowCache() throws Exception
+     {
+         StorageService.instance.initServer(0);
+         CacheService.instance.setRowCacheCapacityInMB(1);
+         rowCacheLoad(100, Integer.MAX_VALUE, 1000);
+ 
 -        ColumnFamilyStore store = Keyspace.open(KEYSPACE).getColumnFamilyStore(COLUMN_FAMILY);
 -        assertEquals(CacheService.instance.rowCache.getKeySet().size(), 100);
++        ColumnFamilyStore store = Keyspace.open(KEYSPACE_CACHED).getColumnFamilyStore(CF_CACHED);
++        assertEquals(CacheService.instance.rowCache.size(), 100);
+ 
+         //construct 5 ranges of 20 elements each
+         ArrayList<Bounds<Token>> subranges = getBounds(20);
+ 
+         //invalidate 3 of the 5 ranges
+         ArrayList<Bounds<Token>> boundsToInvalidate = Lists.newArrayList(subranges.get(0), subranges.get(2), subranges.get(4));
+         int invalidatedKeys = store.invalidateRowCache(boundsToInvalidate);
+         assertEquals(60, invalidatedKeys);
+ 
+         //now there should be only 40 cached entries left
 -        assertEquals(40, CacheService.instance.rowCache.getKeySet().size());
++        assertEquals(CacheService.instance.rowCache.size(), 40);
+         CacheService.instance.setRowCacheCapacityInMB(0);
+     }
+ 
+     private ArrayList<Bounds<Token>> getBounds(int nElements)
+     {
 -        ColumnFamilyStore store = Keyspace.open(KEYSPACE).getColumnFamilyStore(COLUMN_FAMILY);
++        ColumnFamilyStore store = Keyspace.open(KEYSPACE_CACHED).getColumnFamilyStore(CF_CACHED);
+         TreeSet<DecoratedKey> orderedKeys = new TreeSet<>();
+ 
 -        for (RowCacheKey key : CacheService.instance.rowCache.getKeySet())
 -            orderedKeys.add(store.partitioner.decorateKey(ByteBuffer.wrap(key.key)));
++        for(Iterator<RowCacheKey> it = CacheService.instance.rowCache.keyIterator();it.hasNext();)
++            orderedKeys.add(store.partitioner.decorateKey(ByteBuffer.wrap(it.next().key)));
+ 
+         ArrayList<Bounds<Token>> boundsToInvalidate = new ArrayList<>();
+         Iterator<DecoratedKey> iterator = orderedKeys.iterator();
+ 
+         while (iterator.hasNext())
+         {
+             Token startRange = iterator.next().getToken();
+             for (int i = 0; i < nElements-2; i++)
+                 iterator.next();
+             Token endRange = iterator.next().getToken();
+             boundsToInvalidate.add(new Bounds<>(startRange, endRange));
+         }
+         return boundsToInvalidate;
+     }
+ 
+     @Test
      public void testRowCachePartialLoad() throws Exception
      {
          CacheService.instance.setRowCacheCapacityInMB(1);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e4875535/test/unit/org/apache/cassandra/dht/BoundsTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/dht/BoundsTest.java
index 0000000,527b498..2ac06d9
mode 000000,100644..100644
--- a/test/unit/org/apache/cassandra/dht/BoundsTest.java
+++ b/test/unit/org/apache/cassandra/dht/BoundsTest.java
@@@ -1,0 -1,61 +1,61 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  *     http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ 
+ package org.apache.cassandra.dht;
+ 
+ import java.util.LinkedList;
+ import java.util.List;
+ import java.util.Set;
+ 
+ import org.junit.Test;
+ 
+ import static org.junit.Assert.assertEquals;
+ import static org.junit.Assert.assertTrue;
+ 
+ public class BoundsTest
+ {
+ 
+     private Bounds<Token> bounds(long left, long right)
+     {
 -        return new Bounds<Token>(new LongToken(left), new LongToken(right));
++        return new Bounds<Token>(new Murmur3Partitioner.LongToken(left), new Murmur3Partitioner.LongToken(right));
+     }
+ 
+     @Test
+     /**
+      * [0,1],[0,5],[1,8],[4,10] = [0, 10]
+      * [15,19][19,20] = [15,20]
+      * [21, 22] = [21,22]
+      */
+     public void testGetNonOverlappingBounds()
+     {
+         List<Bounds<Token>> bounds = new LinkedList<>();
+         bounds.add(bounds(19, 20));
+         bounds.add(bounds(0, 1));
+         bounds.add(bounds(4, 10));
+         bounds.add(bounds(15, 19));
+         bounds.add(bounds(0, 5));
+         bounds.add(bounds(21, 22));
+         bounds.add(bounds(1, 8));
+ 
+         Set<Bounds<Token>> nonOverlappingBounds = Bounds.getNonOverlappingBounds(bounds);
+         assertEquals(3, nonOverlappingBounds.size());
+         assertTrue(nonOverlappingBounds.contains(bounds(0, 10)));
+         assertTrue(nonOverlappingBounds.contains(bounds(15,20)));
+         assertTrue(nonOverlappingBounds.contains(bounds(21,22)));
+     }
+ }