cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From bened...@apache.org
Subject [1/3] cassandra git commit: Make choice of SSTableReader types explicit
Date Tue, 28 Jul 2015 09:28:55 GMT
Repository: cassandra
Updated Branches:
  refs/heads/trunk a8c50b814 -> ad8cad7c4


http://git-wip-us.apache.org/repos/asf/cassandra/blob/ad8cad7c/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java
index c587bb3..ffe7b06 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java
@@ -94,10 +94,10 @@ public class SSTableMetadataTest
 
 
         store.forceBlockingFlush();
-        assertEquals(1, store.getSSTables().size());
+        assertEquals(1, store.getLiveSSTables().size());
         int ttltimestamp = (int)(System.currentTimeMillis()/1000);
         int firstDelTime = 0;
-        for(SSTableReader sstable : store.getSSTables())
+        for(SSTableReader sstable : store.getLiveSSTables())
         {
             firstDelTime = sstable.getSSTableMetadata().maxLocalDeletionTime;
             assertEquals(ttltimestamp + 10000, firstDelTime, 10);
@@ -113,8 +113,8 @@ public class SSTableMetadataTest
 
         ttltimestamp = (int) (System.currentTimeMillis()/1000);
         store.forceBlockingFlush();
-        assertEquals(2, store.getSSTables().size());
-        List<SSTableReader> sstables = new ArrayList<>(store.getSSTables());
+        assertEquals(2, store.getLiveSSTables().size());
+        List<SSTableReader> sstables = new ArrayList<>(store.getLiveSSTables());
         if(sstables.get(0).getSSTableMetadata().maxLocalDeletionTime < sstables.get(1).getSSTableMetadata().maxLocalDeletionTime)
         {
             assertEquals(sstables.get(0).getSSTableMetadata().maxLocalDeletionTime, firstDelTime);
@@ -126,9 +126,9 @@ public class SSTableMetadataTest
             assertEquals(sstables.get(0).getSSTableMetadata().maxLocalDeletionTime, ttltimestamp
+ 20000, 10);
         }
 
-        Util.compact(store, store.getSSTables());
-        assertEquals(1, store.getSSTables().size());
-        for(SSTableReader sstable : store.getSSTables())
+        Util.compact(store, store.getLiveSSTables());
+        assertEquals(1, store.getLiveSSTables().size());
+        for(SSTableReader sstable : store.getLiveSSTables())
         {
             assertEquals(sstable.getSSTableMetadata().maxLocalDeletionTime, ttltimestamp
+ 20000, 10);
         }
@@ -167,10 +167,10 @@ public class SSTableMetadataTest
         .applyUnsafe();
 
         store.forceBlockingFlush();
-        assertEquals(1,store.getSSTables().size());
+        assertEquals(1,store.getLiveSSTables().size());
         int ttltimestamp = (int) (System.currentTimeMillis()/1000);
         int firstMaxDelTime = 0;
-        for(SSTableReader sstable : store.getSSTables())
+        for(SSTableReader sstable : store.getLiveSSTables())
         {
             firstMaxDelTime = sstable.getSSTableMetadata().maxLocalDeletionTime;
             assertEquals(ttltimestamp + 1000, firstMaxDelTime, 10);
@@ -179,9 +179,9 @@ public class SSTableMetadataTest
         RowUpdateBuilder.deleteRow(store.metadata, timestamp + 1, "deletetest", "todelete").applyUnsafe();
 
         store.forceBlockingFlush();
-        assertEquals(2,store.getSSTables().size());
+        assertEquals(2,store.getLiveSSTables().size());
         boolean foundDelete = false;
-        for(SSTableReader sstable : store.getSSTables())
+        for(SSTableReader sstable : store.getLiveSSTables())
         {
             if(sstable.getSSTableMetadata().maxLocalDeletionTime != firstMaxDelTime)
             {
@@ -190,9 +190,9 @@ public class SSTableMetadataTest
             }
         }
         assertTrue(foundDelete);
-        Util.compact(store, store.getSSTables());
-        assertEquals(1,store.getSSTables().size());
-        for(SSTableReader sstable : store.getSSTables())
+        Util.compact(store, store.getLiveSSTables());
+        assertEquals(1,store.getLiveSSTables().size());
+        for(SSTableReader sstable : store.getLiveSSTables())
         {
             assertEquals(ttltimestamp + 100, sstable.getSSTableMetadata().maxLocalDeletionTime,
10);
         }
@@ -216,8 +216,8 @@ public class SSTableMetadataTest
             }
         }
         store.forceBlockingFlush();
-        assertEquals(1, store.getSSTables().size());
-        for (SSTableReader sstable : store.getSSTables())
+        assertEquals(1, store.getLiveSSTables().size());
+        for (SSTableReader sstable : store.getLiveSSTables())
         {
             assertEquals(ByteBufferUtil.string(sstable.getSSTableMetadata().minClusteringValues.get(0)),
"0col100");
             assertEquals(ByteBufferUtil.string(sstable.getSSTableMetadata().maxClusteringValues.get(0)),
"7col149");
@@ -235,8 +235,8 @@ public class SSTableMetadataTest
 
         store.forceBlockingFlush();
         store.forceMajorCompaction();
-        assertEquals(1, store.getSSTables().size());
-        for (SSTableReader sstable : store.getSSTables())
+        assertEquals(1, store.getLiveSSTables().size());
+        for (SSTableReader sstable : store.getLiveSSTables())
         {
             assertEquals(ByteBufferUtil.string(sstable.getSSTableMetadata().minClusteringValues.get(0)),
"0col100");
             assertEquals(ByteBufferUtil.string(sstable.getSSTableMetadata().maxClusteringValues.get(0)),
"9col298");
@@ -282,8 +282,8 @@ public class SSTableMetadataTest
         }
         cfs.forceBlockingFlush();
         cfs.forceMajorCompaction();
-        assertEquals(cfs.getSSTables().size(), 1);
-        for (SSTableReader sstable : cfs.getSSTables())
+        assertEquals(cfs.getLiveSSTables().size(), 1);
+        for (SSTableReader sstable : cfs.getLiveSSTables())
         {
             assertEquals("b9", ByteBufferUtil.string(sstable.getSSTableMetadata().maxClusteringValues.get(0)));
             assertEquals(9, ByteBufferUtil.toInt(sstable.getSSTableMetadata().maxClusteringValues.get(1)));
@@ -307,7 +307,7 @@ public class SSTableMetadataTest
         cells.addColumn(new BufferCounterCell(cellname("col"), state.context, 1L, Long.MIN_VALUE));
         new Mutation(Util.dk("k").getKey(), cells).applyUnsafe();
         cfs.forceBlockingFlush();
-        assertTrue(cfs.getSSTables().iterator().next().getSSTableMetadata().hasLegacyCounterShards);
+        assertTrue(cfs.getLiveSSTables().iterator().next().getSSTableMetadata().hasLegacyCounterShards);
         cfs.truncateBlocking();
 
         // A cell with global and remote shards
@@ -318,7 +318,7 @@ public class SSTableMetadataTest
         cells.addColumn(new BufferCounterCell(cellname("col"), state.context, 1L, Long.MIN_VALUE));
         new Mutation(Util.dk("k").getKey(), cells).applyUnsafe();
         cfs.forceBlockingFlush();
-        assertTrue(cfs.getSSTables().iterator().next().getSSTableMetadata().hasLegacyCounterShards);
+        assertTrue(cfs.getLiveSSTables().iterator().next().getSSTableMetadata().hasLegacyCounterShards);
         cfs.truncateBlocking();
 
         // A cell with global and local shards
@@ -329,7 +329,7 @@ public class SSTableMetadataTest
         cells.addColumn(new BufferCounterCell(cellname("col"), state.context, 1L, Long.MIN_VALUE));
         new Mutation(Util.dk("k").getKey(), cells).applyUnsafe();
         cfs.forceBlockingFlush();
-        assertTrue(cfs.getSSTables().iterator().next().getSSTableMetadata().hasLegacyCounterShards);
+        assertTrue(cfs.getLiveSSTables().iterator().next().getSSTableMetadata().hasLegacyCounterShards);
         cfs.truncateBlocking();
 
         // A cell with global only
@@ -339,7 +339,7 @@ public class SSTableMetadataTest
         cells.addColumn(new BufferCounterCell(cellname("col"), state.context, 1L, Long.MIN_VALUE));
         new Mutation(Util.dk("k").getKey(), cells).applyUnsafe();
         cfs.forceBlockingFlush();
-        assertFalse(cfs.getSSTables().iterator().next().getSSTableMetadata().hasLegacyCounterShards);
+        assertFalse(cfs.getLiveSSTables().iterator().next().getSSTableMetadata().hasLegacyCounterShards);
         cfs.truncateBlocking();
     } */
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ad8cad7c/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
index 13a371c..651ed8d 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
@@ -129,7 +129,7 @@ public class SSTableReaderTest
         ranges.add(new Range<>(t(9), t(91)));
 
         // confirm that positions increase continuously
-        SSTableReader sstable = store.getSSTables().iterator().next();
+        SSTableReader sstable = store.getLiveSSTables().iterator().next();
         long previous = -1;
         for (Pair<Long,Long> section : sstable.getPositionsForRanges(ranges))
         {
@@ -161,7 +161,7 @@ public class SSTableReaderTest
         CompactionManager.instance.performMaximal(store, false);
 
         // check that all our keys are found correctly
-        SSTableReader sstable = store.getSSTables().iterator().next();
+        SSTableReader sstable = store.getLiveSSTables().iterator().next();
         for (int j = 0; j < 100; j += 2)
         {
             DecoratedKey dk = Util.dk(String.valueOf(j));
@@ -223,7 +223,7 @@ public class SSTableReaderTest
 
         store.forceBlockingFlush();
 
-        SSTableReader sstable = store.getSSTables().iterator().next();
+        SSTableReader sstable = store.getLiveSSTables().iterator().next();
         assertEquals(0, sstable.getReadMeter().count());
 
         DecoratedKey key = sstable.partitioner.decorateKey(ByteBufferUtil.bytes("4"));
@@ -256,7 +256,7 @@ public class SSTableReaderTest
         store.forceBlockingFlush();
         CompactionManager.instance.performMaximal(store, false);
 
-        SSTableReader sstable = store.getSSTables().iterator().next();
+        SSTableReader sstable = store.getLiveSSTables().iterator().next();
         long p2 = sstable.getPosition(k(2), SSTableReader.Operator.EQ).position;
         long p3 = sstable.getPosition(k(3), SSTableReader.Operator.EQ).position;
         long p6 = sstable.getPosition(k(6), SSTableReader.Operator.EQ).position;
@@ -308,7 +308,7 @@ public class SSTableReaderTest
         store.forceBlockingFlush();
         CompactionManager.instance.performMaximal(store, false);
 
-        SSTableReader sstable = store.getSSTables().iterator().next();
+        SSTableReader sstable = store.getLiveSSTables().iterator().next();
         sstable.getPosition(k(2), SSTableReader.Operator.EQ);
         assertEquals(0, sstable.getKeyCacheHit());
         assertEquals(1, sstable.getBloomFilterTruePositiveCount());
@@ -356,7 +356,7 @@ public class SSTableReaderTest
         }
         store.forceBlockingFlush();
 
-        SSTableReader sstable = store.getSSTables().iterator().next();
+        SSTableReader sstable = store.getLiveSSTables().iterator().next();
         Descriptor desc = sstable.descriptor;
 
         // test to see if sstable can be opened as expected
@@ -384,7 +384,7 @@ public class SSTableReaderTest
 
         ColumnFamilyStore indexCfs = store.indexManager.getIndexForColumn(store.metadata.getColumnDefinition(bytes("birthdate"))).getIndexCfs();
         assert indexCfs.partitioner instanceof LocalPartitioner;
-        SSTableReader sstable = indexCfs.getSSTables().iterator().next();
+        SSTableReader sstable = indexCfs.getLiveSSTables().iterator().next();
         assert sstable.first.getToken() instanceof LocalToken;
 
         try(SegmentedFile.Builder ibuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getIndexAccessMode(),
false);
@@ -412,7 +412,7 @@ public class SSTableReaderTest
 
         store.forceBlockingFlush();
         boolean foundScanner = false;
-        for (SSTableReader s : store.getSSTables())
+        for (SSTableReader s : store.getLiveSSTables())
         {
             try (ISSTableScanner scanner = s.getScanner(new Range<Token>(t(0), t(1)),
null))
             {
@@ -451,7 +451,7 @@ public class SSTableReaderTest
         List<Range<Token>> ranges = new ArrayList<Range<Token>>();
         ranges.add(new Range<Token>(t(98), t(99)));
 
-        SSTableReader sstable = store.getSSTables().iterator().next();
+        SSTableReader sstable = store.getLiveSSTables().iterator().next();
         List<Pair<Long,Long>> sections = sstable.getPositionsForRanges(ranges);
         assert sections.size() == 1 : "Expected to find range in sstable" ;
 
@@ -485,7 +485,7 @@ public class SSTableReaderTest
         store.forceBlockingFlush();
         CompactionManager.instance.performMaximal(store, false);
 
-        Collection<SSTableReader> sstables = store.getSSTables();
+        Collection<SSTableReader> sstables = store.getLiveSSTables();
         assert sstables.size() == 1;
         final SSTableReader sstable = sstables.iterator().next();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ad8cad7c/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
index 0e533c2..cb07d37 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
@@ -54,6 +54,8 @@ import org.apache.cassandra.db.compaction.SSTableSplitter;
 import org.apache.cassandra.db.partitions.ArrayBackedPartition;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.db.lifecycle.SSTableSet;
+import org.apache.cassandra.db.lifecycle.View;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.format.SSTableWriter;
@@ -126,7 +128,7 @@ public class SSTableRewriterTest extends SchemaLoader
                 .apply();
         }
         cfs.forceBlockingFlush();
-        Set<SSTableReader> sstables = new HashSet<>(cfs.getSSTables());
+        Set<SSTableReader> sstables = new HashSet<>(cfs.getLiveSSTables());
         assertEquals(1, sstables.size());
         assertEquals(sstables.iterator().next().bytesOnDisk(), cfs.metric.liveDiskSpaceUsed.getCount());
         int nowInSec = FBUtilities.nowInSeconds();
@@ -158,7 +160,7 @@ public class SSTableRewriterTest extends SchemaLoader
 
         SSTableReader s = writeFile(cfs, 1000);
         cfs.addSSTable(s);
-        Set<SSTableReader> sstables = new HashSet<>(cfs.getSSTables());
+        Set<SSTableReader> sstables = new HashSet<>(cfs.getLiveSSTables());
         assertEquals(1, sstables.size());
 
         int nowInSec = FBUtilities.nowInSeconds();
@@ -190,7 +192,7 @@ public class SSTableRewriterTest extends SchemaLoader
 
         SSTableReader s = writeFile(cfs, 1000);
         cfs.addSSTable(s);
-        Set<SSTableReader> sstables = new HashSet<>(cfs.getSSTables());
+        Set<SSTableReader> sstables = new HashSet<>(cfs.getLiveSSTables());
         assertEquals(1, sstables.size());
 
         int nowInSec = FBUtilities.nowInSeconds();
@@ -209,7 +211,7 @@ public class SSTableRewriterTest extends SchemaLoader
                 if (!checked && writer.currentWriter().getFilePointer() > 1500000)
                 {
                     checked = true;
-                    for (SSTableReader sstable : cfs.getSSTables())
+                    for (SSTableReader sstable : cfs.getLiveSSTables())
                     {
                         if (sstable.openReason == SSTableReader.OpenReason.EARLY)
                         {
@@ -317,7 +319,7 @@ public class SSTableRewriterTest extends SchemaLoader
                 {
                     rewriter.switchWriter(getWriter(cfs, s.descriptor.directory, txn));
                     files++;
-                    assertEquals(cfs.getSSTables().size(), files); // we have one original
file plus the ones we have switched out.
+                    assertEquals(cfs.getLiveSSTables().size(), files); // we have one original
file plus the ones we have switched out.
                     assertEquals(s.bytesOnDisk(), cfs.metric.liveDiskSpaceUsed.getCount());
                     assertEquals(s.bytesOnDisk(), cfs.metric.totalDiskSpaceUsed.getCount());
 
@@ -329,12 +331,12 @@ public class SSTableRewriterTest extends SchemaLoader
         TransactionLogs.waitForDeletions();
 
         long sum = 0;
-        for (SSTableReader x : cfs.getSSTables())
+        for (SSTableReader x : cfs.getLiveSSTables())
             sum += x.bytesOnDisk();
         assertEquals(sum, cfs.metric.liveDiskSpaceUsed.getCount());
         assertEquals(startStorageMetricsLoad - sBytesOnDisk + sum, StorageMetrics.load.getCount());
         assertEquals(files, sstables.size());
-        assertEquals(files, cfs.getSSTables().size());
+        assertEquals(files, cfs.getLiveSSTables().size());
         TransactionLogs.waitForDeletions();
 
         // tmplink and tmp files should be gone:
@@ -372,14 +374,14 @@ public class SSTableRewriterTest extends SchemaLoader
                 {
                     rewriter.switchWriter(getWriter(cfs, s.descriptor.directory, txn));
                     files++;
-                    assertEquals(cfs.getSSTables().size(), files); // we have one original
file plus the ones we have switched out.
+                    assertEquals(cfs.getLiveSSTables().size(), files); // we have one original
file plus the ones we have switched out.
                 }
             }
             sstables = rewriter.finish();
         }
 
         assertEquals(files, sstables.size());
-        assertEquals(files, cfs.getSSTables().size());
+        assertEquals(files, cfs.getLiveSSTables().size());
         TransactionLogs.waitForDeletions();
 
         assertFileCounts(s.descriptor.directory.list());
@@ -409,7 +411,7 @@ public class SSTableRewriterTest extends SchemaLoader
                         {
                         rewriter.switchWriter(getWriter(cfs, sstable.descriptor.directory,
txn));
                             files++;
-                            assertEquals(cfs.getSSTables().size(), files); // we have one
original file plus the ones we have switched out.
+                            assertEquals(cfs.getLiveSSTables().size(), files); // we have
one original file plus the ones we have switched out.
                         }
                     }
                     rewriter.abort();
@@ -440,7 +442,7 @@ public class SSTableRewriterTest extends SchemaLoader
                         {
                         rewriter.switchWriter(getWriter(cfs, sstable.descriptor.directory,
txn));
                             files++;
-                            assertEquals(cfs.getSSTables().size(), files); // we have one
original file plus the ones we have switched out.
+                            assertEquals(cfs.getLiveSSTables().size(), files); // we have
one original file plus the ones we have switched out.
                         }
                         if (files == 3)
                         {
@@ -476,7 +478,7 @@ public class SSTableRewriterTest extends SchemaLoader
                         {
                         rewriter.switchWriter(getWriter(cfs, sstable.descriptor.directory,
txn));
                             files++;
-                            assertEquals(cfs.getSSTables().size(), files); // we have one
original file plus the ones we have switched out.
+                            assertEquals(cfs.getLiveSSTables().size(), files); // we have
one original file plus the ones we have switched out.
                         }
                     }
                     rewriter.abort();
@@ -520,10 +522,10 @@ public class SSTableRewriterTest extends SchemaLoader
         TransactionLogs.waitForDeletions();
 
         assertEquals(startSize, cfs.metric.liveDiskSpaceUsed.getCount());
-        assertEquals(1, cfs.getSSTables().size());
+        assertEquals(1, cfs.getLiveSSTables().size());
         assertFileCounts(s.descriptor.directory.list());
-        assertEquals(cfs.getSSTables().iterator().next().first, origFirst);
-        assertEquals(cfs.getSSTables().iterator().next().last, origLast);
+        assertEquals(cfs.getLiveSSTables().iterator().next().first, origFirst);
+        assertEquals(cfs.getLiveSSTables().iterator().next().last, origLast);
         validateCFS(cfs);
     }
 
@@ -554,7 +556,7 @@ public class SSTableRewriterTest extends SchemaLoader
                 {
                     rewriter.switchWriter(getWriter(cfs, s.descriptor.directory, txn));
                     files++;
-                    assertEquals(cfs.getSSTables().size(), files); // we have one original
file plus the ones we have switched out.
+                    assertEquals(cfs.getLiveSSTables().size(), files); // we have one original
file plus the ones we have switched out.
                 }
                 if (files == 3)
                 {
@@ -567,7 +569,7 @@ public class SSTableRewriterTest extends SchemaLoader
 
         TransactionLogs.waitForDeletions();
 
-        assertEquals(files - 1, cfs.getSSTables().size()); // we never wrote anything to
the last file
+        assertEquals(files - 1, cfs.getLiveSSTables().size()); // we never wrote anything
to the last file
         assertFileCounts(s.descriptor.directory.list());
         validateCFS(cfs);
     }
@@ -600,7 +602,7 @@ public class SSTableRewriterTest extends SchemaLoader
                 {
                     rewriter.switchWriter(getWriter(cfs, s.descriptor.directory, txn));
                     files++;
-                    assertEquals(cfs.getSSTables().size(), files); // we have one original
file plus the ones we have switched out.
+                    assertEquals(cfs.getLiveSSTables().size(), files); // we have one original
file plus the ones we have switched out.
                 }
             }
 
@@ -638,7 +640,7 @@ public class SSTableRewriterTest extends SchemaLoader
                 rewriter.append(ci.next());
                 if (rewriter.currentWriter().getOnDiskFilePointer() > 2500000)
                 {
-                    assertEquals(files, cfs.getSSTables().size()); // all files are now opened
early
+                    assertEquals(files, cfs.getLiveSSTables().size()); // all files are now
opened early
                     rewriter.switchWriter(getWriter(cfs, s.descriptor.directory, txn));
                     files++;
                 }
@@ -647,7 +649,7 @@ public class SSTableRewriterTest extends SchemaLoader
             sstables = rewriter.finish();
         }
         assertEquals(files, sstables.size());
-        assertEquals(files, cfs.getSSTables().size());
+        assertEquals(files, cfs.getLiveSSTables().size());
         TransactionLogs.waitForDeletions();
         assertFileCounts(s.descriptor.directory.list());
 
@@ -752,13 +754,13 @@ public class SSTableRewriterTest extends SchemaLoader
         assertEquals(filecount, 1);
         if (!offline)
         {
-            assertEquals(1, cfs.getSSTables().size());
+            assertEquals(1, cfs.getLiveSSTables().size());
             validateCFS(cfs);
             truncate(cfs);
         }
         else
         {
-            assertEquals(0, cfs.getSSTables().size());
+            assertEquals(0, cfs.getLiveSSTables().size());
             cfs.truncateBlocking();
         }
         filecount = assertFileCounts(s.descriptor.directory.list());
@@ -798,8 +800,8 @@ public class SSTableRewriterTest extends SchemaLoader
         cfs.forceMajorCompaction();
         validateKeys(keyspace);
 
-        assertEquals(1, cfs.getSSTables().size());
-        SSTableReader s = cfs.getSSTables().iterator().next();
+        assertEquals(1, cfs.getLiveSSTables().size());
+        SSTableReader s = cfs.getLiveSSTables().iterator().next();
         Set<SSTableReader> compacting = new HashSet<>();
         compacting.add(s);
 
@@ -856,7 +858,7 @@ public class SSTableRewriterTest extends SchemaLoader
                 if (!checked && writer.currentWriter().getFilePointer() > 15000000)
                 {
                     checked = true;
-                    ColumnFamilyStore.ViewFragment viewFragment = cfs.select(ColumnFamilyStore.CANONICAL_SSTABLES);
+                    ColumnFamilyStore.ViewFragment viewFragment = cfs.select(View.select(SSTableSet.CANONICAL));
                     // canonical view should have only one SSTable which is not opened early.
                     assertEquals(1, viewFragment.sstables.size());
                     SSTableReader sstable = viewFragment.sstables.get(0);
@@ -965,7 +967,7 @@ public class SSTableRewriterTest extends SchemaLoader
     {
         Set<Integer> liveDescriptors = new HashSet<>();
         long spaceUsed = 0;
-        for (SSTableReader sstable : cfs.getSSTables())
+        for (SSTableReader sstable : cfs.getLiveSSTables())
         {
             assertFalse(sstable.isMarkedCompacted());
             assertEquals(1, sstable.selfRef().globalCount());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ad8cad7c/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java
index 0dd8825..272b62f 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java
@@ -213,8 +213,8 @@ public class SSTableScannerTest
             insertRowWithKey(store.metadata, i);
         store.forceBlockingFlush();
 
-        assertEquals(1, store.getSSTables().size());
-        SSTableReader sstable = store.getSSTables().iterator().next();
+        assertEquals(1, store.getLiveSSTables().size());
+        SSTableReader sstable = store.getLiveSSTables().iterator().next();
 
         // full range scan
         ISSTableScanner scanner = sstable.getScanner(null);
@@ -319,8 +319,8 @@ public class SSTableScannerTest
                 insertRowWithKey(store.metadata, i * 100 + j);
         store.forceBlockingFlush();
 
-        assertEquals(1, store.getSSTables().size());
-        SSTableReader sstable = store.getSSTables().iterator().next();
+        assertEquals(1, store.getLiveSSTables().size());
+        SSTableReader sstable = store.getLiveSSTables().iterator().next();
 
         // full range scan
         ISSTableScanner fullScanner = sstable.getScanner(null);
@@ -449,8 +449,8 @@ public class SSTableScannerTest
         insertRowWithKey(store.metadata, 205);
         store.forceBlockingFlush();
 
-        assertEquals(1, store.getSSTables().size());
-        SSTableReader sstable = store.getSSTables().iterator().next();
+        assertEquals(1, store.getLiveSSTables().size());
+        SSTableReader sstable = store.getLiveSSTables().iterator().next();
 
         // full range scan
         ISSTableScanner fullScanner = sstable.getScanner(null);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ad8cad7c/test/unit/org/apache/cassandra/schema/DefsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/schema/DefsTest.java b/test/unit/org/apache/cassandra/schema/DefsTest.java
index 2bfd6ae..df78cef 100644
--- a/test/unit/org/apache/cassandra/schema/DefsTest.java
+++ b/test/unit/org/apache/cassandra/schema/DefsTest.java
@@ -525,7 +525,7 @@ public class DefsTest
 
         cfs.forceBlockingFlush();
         ColumnFamilyStore indexedCfs = cfs.indexManager.getIndexForColumn(cfs.metadata.getColumnDefinition(ByteBufferUtil.bytes("birthdate"))).getIndexCfs();
-        Descriptor desc = indexedCfs.getSSTables().iterator().next().descriptor;
+        Descriptor desc = indexedCfs.getLiveSSTables().iterator().next().descriptor;
 
         // drop the index
         CFMetaData meta = cfs.metadata.copy();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ad8cad7c/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java b/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
index 875c306..9572552 100644
--- a/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
+++ b/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
@@ -74,7 +74,7 @@ public class StreamTransferTaskTest
 
         // create streaming task that streams those two sstables
         StreamTransferTask task = new StreamTransferTask(session, cfs.metadata.cfId);
-        for (SSTableReader sstable : cfs.getSSTables())
+        for (SSTableReader sstable : cfs.getLiveSSTables())
         {
             List<Range<Token>> ranges = new ArrayList<>();
             ranges.add(new Range<>(sstable.first.getToken(), sstable.last.getToken()));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ad8cad7c/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
index 0a3c02a..cb084a0 100644
--- a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
+++ b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
@@ -165,14 +165,14 @@ public class StreamingTransferTest
             mutator.mutate("key" + i, "col" + i, timestamp);
         cfs.forceBlockingFlush();
         Util.compactAll(cfs, Integer.MAX_VALUE).get();
-        assertEquals(1, cfs.getSSTables().size());
+        assertEquals(1, cfs.getLiveSSTables().size());
 
         // transfer the first and last key
         logger.debug("Transferring {}", cfs.name);
         int[] offs;
         if (transferSSTables)
         {
-            SSTableReader sstable = cfs.getSSTables().iterator().next();
+            SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
             cfs.clearUnsafe();
             transferSSTables(sstable);
             offs = new int[]{1, 3};
@@ -186,7 +186,7 @@ public class StreamingTransferTest
         }
 
         // confirm that a single SSTable was transferred and registered
-        assertEquals(1, cfs.getSSTables().size());
+        assertEquals(1, cfs.getLiveSSTables().size());
 
         // and that the index and filter were properly recovered
         List<ArrayBackedPartition> partitions = Util.getAllUnfiltered(Util.cmd(cfs).build());
@@ -203,7 +203,7 @@ public class StreamingTransferTest
         }
 
         // and that the max timestamp for the file was rediscovered
-        assertEquals(timestamp, cfs.getSSTables().iterator().next().getMaxTimestamp());
+        assertEquals(timestamp, cfs.getLiveSSTables().iterator().next().getMaxTimestamp());
 
         List<String> keys = new ArrayList<>();
         for (int off : offs)
@@ -322,12 +322,12 @@ public class StreamingTransferTest
 
         cfs.forceBlockingFlush();
 
-        SSTableReader sstable = cfs.getSSTables().iterator().next();
+        SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
         cfs.clearUnsafe();
         transferSSTables(sstable);
 
         // confirm that a single SSTable was transferred and registered
-        assertEquals(1, cfs.getSSTables().size());
+        assertEquals(1, cfs.getLiveSSTables().size());
 
         Row r = Util.getOnlyRow(Util.cmd(cfs).build());
         Assert.assertFalse(r.isEmpty());
@@ -389,13 +389,13 @@ public class StreamingTransferTest
             .cf(cfs.name)
             .generation(0)
             .write(cleanedEntries);
-        SSTableReader streamed = cfs.getSSTables().iterator().next();
+        SSTableReader streamed = cfs.getLiveSSTables().iterator().next();
         SSTableUtils.assertContentEquals(cleaned, streamed);
 
         // Retransfer the file, making sure it is now idempotent (see CASSANDRA-3481)
         cfs.clearUnsafe();
         transferSSTables(streamed);
-        SSTableReader restreamed = cfs.getSSTables().iterator().next();
+        SSTableReader restreamed = cfs.getLiveSSTables().iterator().next();
         SSTableUtils.assertContentEquals(streamed, restreamed);
     }
 
@@ -514,7 +514,7 @@ public class StreamingTransferTest
             mutator.mutate("key" + i, "col" + i, System.currentTimeMillis());
         cfs.forceBlockingFlush();
         Util.compactAll(cfs, Integer.MAX_VALUE).get();
-        SSTableReader sstable = cfs.getSSTables().iterator().next();
+        SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
         cfs.clearUnsafe();
 
         IPartitioner p = StorageService.getPartitioner();
@@ -523,7 +523,7 @@ public class StreamingTransferTest
         ranges.add(new Range<>(p.getToken(ByteBufferUtil.bytes("key5")), p.getToken(ByteBufferUtil.bytes("key500"))));
         ranges.add(new Range<>(p.getToken(ByteBufferUtil.bytes("key9")), p.getToken(ByteBufferUtil.bytes("key900"))));
         transfer(sstable, ranges);
-        assertEquals(1, cfs.getSSTables().size());
+        assertEquals(1, cfs.getLiveSSTables().size());
         assertEquals(7, Util.getRangeSlice(cfs).size());
     }
     */


Mime
View raw message