You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by be...@apache.org on 2015/05/22 10:46:55 UTC
[1/6] cassandra git commit: Extend Transactional API to sstable
lifecycle management
Repository: cassandra
Updated Branches:
refs/heads/cassandra-2.2 33d71b825 -> e5a76bdb5
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/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 5dca589..fa91d00 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
@@ -18,14 +18,16 @@
package org.apache.cassandra.io.sstable;
import java.io.File;
-import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.*;
import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import com.google.common.collect.Iterables;
import com.google.common.collect.Sets;
import org.junit.After;
import org.junit.BeforeClass;
+import com.google.common.util.concurrent.Uninterruptibles;
import org.junit.Test;
import org.apache.cassandra.SchemaLoader;
@@ -43,6 +45,7 @@ import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.sstable.format.SSTableWriter;
import org.apache.cassandra.locator.SimpleStrategy;
import org.apache.cassandra.db.compaction.SSTableSplitter;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.dht.Range;
import org.apache.cassandra.dht.Token;
import org.apache.cassandra.metrics.StorageMetrics;
@@ -52,7 +55,6 @@ import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.Pair;
import static org.junit.Assert.*;
-import static org.apache.cassandra.utils.Throwables.maybeFail;
public class SSTableRewriterTest extends SchemaLoader
{
@@ -83,7 +85,9 @@ public class SSTableRewriterTest extends SchemaLoader
{
Keyspace keyspace = Keyspace.open(KEYSPACE);
ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
- cfs.truncateBlocking();
+ truncate(cfs);
+ assertEquals(0, cfs.metric.liveDiskSpaceUsed.getCount());
+
for (int j = 0; j < 100; j ++)
{
ByteBuffer key = ByteBufferUtil.bytes(String.valueOf(j));
@@ -94,8 +98,10 @@ public class SSTableRewriterTest extends SchemaLoader
cfs.forceBlockingFlush();
Set<SSTableReader> sstables = new HashSet<>(cfs.getSSTables());
assertEquals(1, sstables.size());
- try (SSTableRewriter writer = new SSTableRewriter(cfs, sstables, 1000, false);
- AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables);)
+ assertEquals(sstables.iterator().next().bytesOnDisk(), cfs.metric.liveDiskSpaceUsed.getCount());
+ try (AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables);
+ LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN);
+ SSTableRewriter writer = new SSTableRewriter(cfs, txn, 1000, false);)
{
ISSTableScanner scanner = scanners.scanners.get(0);
CompactionController controller = new CompactionController(cfs, sstables, cfs.gcBefore(System.currentTimeMillis()));
@@ -105,30 +111,29 @@ public class SSTableRewriterTest extends SchemaLoader
AbstractCompactedRow row = new LazilyCompactedRow(controller, Arrays.asList(scanner.next()));
writer.append(row);
}
- Collection<SSTableReader> newsstables = writer.finish();
- cfs.getDataTracker().markCompactedSSTablesReplaced(sstables, newsstables , OperationType.COMPACTION);
+ writer.finish();
}
SSTableDeletingTask.waitForDeletions();
-
validateCFS(cfs);
int filecounts = assertFileCounts(sstables.iterator().next().descriptor.directory.list(), 0, 0);
assertEquals(1, filecounts);
-
+ truncate(cfs);
}
@Test
public void basicTest2() throws InterruptedException
{
Keyspace keyspace = Keyspace.open(KEYSPACE);
ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
- cfs.truncateBlocking();
+ truncate(cfs);
SSTableReader s = writeFile(cfs, 1000);
cfs.addSSTable(s);
Set<SSTableReader> sstables = new HashSet<>(cfs.getSSTables());
assertEquals(1, sstables.size());
SSTableRewriter.overrideOpenInterval(10000000);
- try (SSTableRewriter writer = new SSTableRewriter(cfs, sstables, 1000, false);
- AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables);)
+ try (AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables);
+ LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN);
+ SSTableRewriter writer = new SSTableRewriter(cfs, txn, 1000, false);)
{
ISSTableScanner scanner = scanners.scanners.get(0);
CompactionController controller = new CompactionController(cfs, sstables, cfs.gcBefore(System.currentTimeMillis()));
@@ -138,11 +143,9 @@ public class SSTableRewriterTest extends SchemaLoader
AbstractCompactedRow row = new LazilyCompactedRow(controller, Arrays.asList(scanner.next()));
writer.append(row);
}
- Collection<SSTableReader> newsstables = writer.finish();
- cfs.getDataTracker().markCompactedSSTablesReplaced(sstables, newsstables, OperationType.COMPACTION);
+ writer.finish();
}
SSTableDeletingTask.waitForDeletions();
-
validateCFS(cfs);
int filecounts = assertFileCounts(sstables.iterator().next().descriptor.directory.list(), 0, 0);
assertEquals(1, filecounts);
@@ -153,7 +156,9 @@ public class SSTableRewriterTest extends SchemaLoader
{
Keyspace keyspace = Keyspace.open(KEYSPACE);
ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
- cfs.truncateBlocking();
+ truncate(cfs);
+ assertEquals(0, cfs.metric.liveDiskSpaceUsed.getCount());
+ assertEquals(0, cfs.metric.totalDiskSpaceUsed.getCount());
SSTableReader s = writeFile(cfs, 1000);
cfs.addSSTable(s);
@@ -161,8 +166,9 @@ public class SSTableRewriterTest extends SchemaLoader
assertEquals(1, sstables.size());
SSTableRewriter.overrideOpenInterval(10000000);
boolean checked = false;
- try (SSTableRewriter writer = new SSTableRewriter(cfs, sstables, 1000, false);
- AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables);)
+ try (AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables);
+ LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN);
+ SSTableRewriter writer = new SSTableRewriter(cfs, txn, 1000, false);)
{
ISSTableScanner scanner = scanners.scanners.get(0);
CompactionController controller = new CompactionController(cfs, sstables, cfs.gcBefore(System.currentTimeMillis()));
@@ -178,7 +184,7 @@ public class SSTableRewriterTest extends SchemaLoader
{
if (sstable.openReason == SSTableReader.OpenReason.EARLY)
{
- SSTableReader c = sstables.iterator().next();
+ SSTableReader c = txn.current(sstables.iterator().next());
Collection<Range<Token>> r = Arrays.asList(new Range<>(cfs.partitioner.getMinimumToken(), cfs.partitioner.getMinimumToken()));
List<Pair<Long, Long>> tmplinkPositions = sstable.getPositionsForRanges(r);
List<Pair<Long, Long>> compactingPositions = c.getPositionsForRanges(r);
@@ -193,17 +199,14 @@ public class SSTableRewriterTest extends SchemaLoader
}
}
assertTrue(checked);
- Collection<SSTableReader> newsstables = writer.finish();
- cfs.getDataTracker().markCompactedSSTablesReplaced(sstables, newsstables, OperationType.COMPACTION);
+ writer.finish();
}
SSTableDeletingTask.waitForDeletions();
-
validateCFS(cfs);
int filecounts = assertFileCounts(sstables.iterator().next().descriptor.directory.list(), 0, 0);
assertEquals(1, filecounts);
- cfs.truncateBlocking();
+ truncate(cfs);
SSTableDeletingTask.waitForDeletions();
-
validateCFS(cfs);
}
@@ -212,7 +215,8 @@ public class SSTableRewriterTest extends SchemaLoader
{
Keyspace keyspace = Keyspace.open(KEYSPACE);
ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
- cfs.truncateBlocking();
+ truncate(cfs);
+ assertEquals(0, cfs.metric.liveDiskSpaceUsed.getCount());
ArrayBackedSortedColumns cf = ArrayBackedSortedColumns.factory.create(cfs.metadata);
for (int i = 0; i < 100; i++)
cf.addColumn(Util.cellname(i), ByteBuffer.allocate(1000), 1);
@@ -253,7 +257,7 @@ public class SSTableRewriterTest extends SchemaLoader
{
Keyspace keyspace = Keyspace.open(KEYSPACE);
ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
- cfs.truncateBlocking();
+ truncate(cfs);
SSTableReader s = writeFile(cfs, 1000);
cfs.addSSTable(s);
@@ -263,9 +267,10 @@ public class SSTableRewriterTest extends SchemaLoader
List<SSTableReader> sstables;
int files = 1;
- try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
- ISSTableScanner scanner = s.getScanner();
- CompactionController controller = new CompactionController(cfs, compacting, 0))
+ try (ISSTableScanner scanner = s.getScanner();
+ CompactionController controller = new CompactionController(cfs, compacting, 0);
+ LifecycleTransaction txn = cfs.getTracker().tryModify(compacting, OperationType.UNKNOWN);
+ SSTableRewriter rewriter = new SSTableRewriter(cfs, txn, 1000, false);)
{
rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
while(scanner.hasNext())
@@ -282,8 +287,6 @@ public class SSTableRewriterTest extends SchemaLoader
}
}
sstables = rewriter.finish();
- cfs.getDataTracker().markCompactedSSTablesReplaced(compacting, sstables, OperationType.COMPACTION);
- assertEquals(files, sstables.size());
}
long sum = 0;
for (SSTableReader x : cfs.getSSTables())
@@ -305,7 +308,7 @@ public class SSTableRewriterTest extends SchemaLoader
{
Keyspace keyspace = Keyspace.open(KEYSPACE);
ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
- cfs.truncateBlocking();
+ truncate(cfs);
SSTableReader s = writeFile(cfs, 1000);
cfs.addSSTable(s);
@@ -315,9 +318,10 @@ public class SSTableRewriterTest extends SchemaLoader
List<SSTableReader> sstables;
int files = 1;
- try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
- ISSTableScanner scanner = s.getScanner();
- CompactionController controller = new CompactionController(cfs, compacting, 0))
+ try (ISSTableScanner scanner = s.getScanner();
+ CompactionController controller = new CompactionController(cfs, compacting, 0);
+ LifecycleTransaction txn = cfs.getTracker().tryModify(compacting, OperationType.UNKNOWN);
+ SSTableRewriter rewriter = new SSTableRewriter(cfs, txn, 1000, false);)
{
rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
while(scanner.hasNext())
@@ -335,10 +339,6 @@ public class SSTableRewriterTest extends SchemaLoader
assertEquals(files, sstables.size());
assertEquals(files, cfs.getSSTables().size());
- assertEquals(1, cfs.getDataTracker().getView().shadowed.size());
- cfs.getDataTracker().markCompactedSSTablesReplaced(compacting, sstables, OperationType.COMPACTION);
- assertEquals(files, cfs.getSSTables().size());
- assertEquals(0, cfs.getDataTracker().getView().shadowed.size());
SSTableDeletingTask.waitForDeletions();
assertFileCounts(s.descriptor.directory.list(), 0, 0);
@@ -429,7 +429,7 @@ public class SSTableRewriterTest extends SchemaLoader
{
Keyspace keyspace = Keyspace.open(KEYSPACE);
ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
- cfs.truncateBlocking();
+ truncate(cfs);
SSTableReader s = writeFile(cfs, 1000);
cfs.addSSTable(s);
@@ -440,9 +440,10 @@ public class SSTableRewriterTest extends SchemaLoader
Set<SSTableReader> compacting = Sets.newHashSet(s);
SSTableRewriter.overrideOpenInterval(10000000);
- try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
- ISSTableScanner scanner = s.getScanner();
- CompactionController controller = new CompactionController(cfs, compacting, 0))
+ try (ISSTableScanner scanner = s.getScanner();
+ CompactionController controller = new CompactionController(cfs, compacting, 0);
+ LifecycleTransaction txn = cfs.getTracker().tryModify(compacting, OperationType.UNKNOWN);
+ SSTableRewriter rewriter = new SSTableRewriter(cfs, txn, 1000, false);)
{
rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
test.run(scanner, controller, s, cfs, rewriter);
@@ -463,7 +464,7 @@ public class SSTableRewriterTest extends SchemaLoader
{
Keyspace keyspace = Keyspace.open(KEYSPACE);
ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
- cfs.truncateBlocking();
+ truncate(cfs);
SSTableReader s = writeFile(cfs, 1000);
cfs.addSSTable(s);
@@ -472,9 +473,10 @@ public class SSTableRewriterTest extends SchemaLoader
SSTableRewriter.overrideOpenInterval(10000000);
int files = 1;
- try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
- ISSTableScanner scanner = s.getScanner();
- CompactionController controller = new CompactionController(cfs, compacting, 0))
+ try (ISSTableScanner scanner = s.getScanner();
+ CompactionController controller = new CompactionController(cfs, compacting, 0);
+ LifecycleTransaction txn = cfs.getTracker().tryModify(compacting, OperationType.UNKNOWN);
+ SSTableRewriter rewriter = new SSTableRewriter(cfs, txn, 1000, false);)
{
rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
while(scanner.hasNext())
@@ -489,8 +491,7 @@ public class SSTableRewriterTest extends SchemaLoader
if (files == 3)
{
//testing to finish when we have nothing written in the new file
- List<SSTableReader> sstables = rewriter.finish();
- cfs.getDataTracker().markCompactedSSTablesReplaced(compacting, sstables, OperationType.COMPACTION);
+ rewriter.finish();
break;
}
}
@@ -508,7 +509,7 @@ public class SSTableRewriterTest extends SchemaLoader
{
Keyspace keyspace = Keyspace.open(KEYSPACE);
ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
- cfs.truncateBlocking();
+ truncate(cfs);
cfs.disableAutoCompaction();
SSTableReader s = writeFile(cfs, 1000);
@@ -518,9 +519,10 @@ public class SSTableRewriterTest extends SchemaLoader
List<SSTableReader> sstables;
int files = 1;
- try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
- ISSTableScanner scanner = s.getScanner();
- CompactionController controller = new CompactionController(cfs, compacting, 0))
+ try (ISSTableScanner scanner = s.getScanner();
+ CompactionController controller = new CompactionController(cfs, compacting, 0);
+ LifecycleTransaction txn = cfs.getTracker().tryModify(compacting, OperationType.UNKNOWN);
+ SSTableRewriter rewriter = new SSTableRewriter(cfs, txn, 1000, false);)
{
rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
while(scanner.hasNext())
@@ -535,7 +537,6 @@ public class SSTableRewriterTest extends SchemaLoader
}
sstables = rewriter.finish();
- cfs.getDataTracker().markCompactedSSTablesReplaced(compacting, sstables, OperationType.COMPACTION);
}
SSTableDeletingTask.waitForDeletions();
@@ -550,7 +551,7 @@ public class SSTableRewriterTest extends SchemaLoader
{
Keyspace keyspace = Keyspace.open(KEYSPACE);
ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
- cfs.truncateBlocking();
+ truncate(cfs);
cfs.disableAutoCompaction();
SSTableReader s = writeFile(cfs, 400);
@@ -560,9 +561,10 @@ public class SSTableRewriterTest extends SchemaLoader
List<SSTableReader> sstables;
int files = 1;
- try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
- ISSTableScanner scanner = s.getScanner();
- CompactionController controller = new CompactionController(cfs, compacting, 0))
+ try (ISSTableScanner scanner = s.getScanner();
+ CompactionController controller = new CompactionController(cfs, compacting, 0);
+ LifecycleTransaction txn = cfs.getTracker().tryModify(compacting, OperationType.UNKNOWN);
+ SSTableRewriter rewriter = new SSTableRewriter(cfs, txn, 1000, false);)
{
rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
while(scanner.hasNext())
@@ -577,7 +579,6 @@ public class SSTableRewriterTest extends SchemaLoader
}
sstables = rewriter.finish();
- cfs.getDataTracker().markCompactedSSTablesReplaced(compacting, sstables, OperationType.COMPACTION);
}
assertEquals(files, sstables.size());
@@ -593,22 +594,24 @@ public class SSTableRewriterTest extends SchemaLoader
{
Keyspace keyspace = Keyspace.open(KEYSPACE);
ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
- cfs.truncateBlocking();
+ truncate(cfs);
cfs.disableAutoCompaction();
SSTableReader s = writeFile(cfs, 1000);
- cfs.getDataTracker().markCompacting(Arrays.asList(s), true, false);
- SSTableSplitter splitter = new SSTableSplitter(cfs, s, 10);
- splitter.split();
+ try (LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.UNKNOWN, s))
+ {
+ SSTableSplitter splitter = new SSTableSplitter(cfs, txn, 10);
+ splitter.split();
- assertFileCounts(s.descriptor.directory.list(), 0, 0);
+ assertFileCounts(s.descriptor.directory.list(), 0, 0);
- s.selfRef().release();
- SSTableDeletingTask.waitForDeletions();
+ s.selfRef().release();
+ SSTableDeletingTask.waitForDeletions();
- for (File f : s.descriptor.directory.listFiles())
- {
- // we need to clear out the data dir, otherwise tests running after this breaks
- FileUtils.deleteRecursive(f);
+ for (File f : s.descriptor.directory.listFiles())
+ {
+ // we need to clear out the data dir, otherwise tests running after this breaks
+ FileUtils.deleteRecursive(f);
+ }
}
}
@@ -639,17 +642,18 @@ public class SSTableRewriterTest extends SchemaLoader
{
Keyspace keyspace = Keyspace.open(KEYSPACE);
ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
- cfs.truncateBlocking();
+ truncate(cfs);
SSTableReader s = writeFile(cfs, 1000);
if (!offline)
cfs.addSSTable(s);
Set<SSTableReader> compacting = Sets.newHashSet(s);
- cfs.getDataTracker().markCompacting(compacting);
SSTableRewriter.overrideOpenInterval(10000000);
-
- try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, offline);
- ISSTableScanner scanner = compacting.iterator().next().getScanner();
- CompactionController controller = new CompactionController(cfs, compacting, 0))
+ try (ISSTableScanner scanner = compacting.iterator().next().getScanner();
+ CompactionController controller = new CompactionController(cfs, compacting, 0);
+ LifecycleTransaction txn = offline ? LifecycleTransaction.offline(OperationType.UNKNOWN, compacting)
+ : cfs.getTracker().tryModify(compacting, OperationType.UNKNOWN);
+ SSTableRewriter rewriter = new SSTableRewriter(cfs, txn, 1000, offline);
+ )
{
rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
while (scanner.hasNext())
@@ -672,7 +676,6 @@ public class SSTableRewriterTest extends SchemaLoader
}
finally
{
- cfs.getDataTracker().unmarkCompacting(compacting);
if (offline)
s.selfRef().release();
}
@@ -686,9 +689,8 @@ public class SSTableRewriterTest extends SchemaLoader
assertEquals(1, cfs.getSSTables().size());
validateCFS(cfs);
}
- cfs.truncateBlocking();
+ truncate(cfs);
SSTableDeletingTask.waitForDeletions();
-
filecount = assertFileCounts(s.descriptor.directory.list(), 0, 0);
if (offline)
{
@@ -709,7 +711,7 @@ public class SSTableRewriterTest extends SchemaLoader
{
Keyspace keyspace = Keyspace.open(KEYSPACE);
ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
- cfs.truncateBlocking();
+ truncate(cfs);
for (int i = 0; i < 100; i++)
{
DecoratedKey key = Util.dk(Integer.toString(i));
@@ -726,14 +728,14 @@ public class SSTableRewriterTest extends SchemaLoader
SSTableReader s = cfs.getSSTables().iterator().next();
Set<SSTableReader> compacting = new HashSet<>();
compacting.add(s);
- cfs.getDataTracker().markCompacting(compacting);
-
SSTableRewriter.overrideOpenInterval(1);
int keyCount = 0;
- try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
- ISSTableScanner scanner = compacting.iterator().next().getScanner();
- CompactionController controller = new CompactionController(cfs, compacting, 0))
+ try (ISSTableScanner scanner = compacting.iterator().next().getScanner();
+ CompactionController controller = new CompactionController(cfs, compacting, 0);
+ LifecycleTransaction txn = cfs.getTracker().tryModify(compacting, OperationType.UNKNOWN);
+ SSTableRewriter rewriter = new SSTableRewriter(cfs, txn, 1000, false);
+ )
{
rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
while (scanner.hasNext())
@@ -746,15 +748,7 @@ public class SSTableRewriterTest extends SchemaLoader
keyCount++;
validateKeys(keyspace);
}
- try
- {
- cfs.getDataTracker().markCompactedSSTablesReplaced(compacting, rewriter.finish(), OperationType.COMPACTION);
- cfs.getDataTracker().unmarkCompacting(compacting);
- }
- catch (Throwable t)
- {
- rewriter.abort();
- }
+ rewriter.finish();
}
validateKeys(keyspace);
SSTableDeletingTask.waitForDeletions();
@@ -762,7 +756,7 @@ public class SSTableRewriterTest extends SchemaLoader
}
@Test
- public void testCanonicalView() throws IOException
+ public void testCanonicalView() throws Exception
{
Keyspace keyspace = Keyspace.open(KEYSPACE);
ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
@@ -770,15 +764,16 @@ public class SSTableRewriterTest extends SchemaLoader
SSTableReader s = writeFile(cfs, 1000);
cfs.addSSTable(s);
- Set<SSTableReader> sstables = Sets.newHashSet(cfs.markAllCompacting());
+ Set<SSTableReader> sstables = Sets.newHashSet(s);
assertEquals(1, sstables.size());
SSTableRewriter.overrideOpenInterval(10000000);
- SSTableRewriter writer = new SSTableRewriter(cfs, sstables, 1000, false);
boolean checked = false;
- try (AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables))
+ try (ISSTableScanner scanner = sstables.iterator().next().getScanner();
+ CompactionController controller = new CompactionController(cfs, sstables, 0);
+ LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN);
+ SSTableRewriter writer = new SSTableRewriter(cfs, txn, 1000, false);
+ )
{
- ISSTableScanner scanner = scanners.scanners.get(0);
- CompactionController controller = new CompactionController(cfs, sstables, cfs.gcBefore(System.currentTimeMillis()));
writer.switchWriter(getWriter(cfs, sstables.iterator().next().descriptor.directory));
while (scanner.hasNext())
{
@@ -796,10 +791,7 @@ public class SSTableRewriterTest extends SchemaLoader
}
}
}
- writer.abort();
- cfs.getDataTracker().unmarkCompacting(sstables);
- cfs.truncateBlocking();
- SSTableDeletingTask.waitForDeletions();
+ truncateCF();
validateCFS(cfs);
}
@@ -813,30 +805,52 @@ public class SSTableRewriterTest extends SchemaLoader
}
}
- private SSTableReader writeFile(ColumnFamilyStore cfs, int count)
+ public static void truncate(ColumnFamilyStore cfs)
{
- ArrayBackedSortedColumns cf = ArrayBackedSortedColumns.factory.create(cfs.metadata);
- for (int i = 0; i < count / 100; i++)
- cf.addColumn(Util.cellname(i), random(0, 1000), 1);
- File dir = cfs.directories.getDirectoryForNewSSTables();
- String filename = cfs.getTempSSTablePath(dir);
+ cfs.truncateBlocking();
+ Uninterruptibles.sleepUninterruptibly(10L,TimeUnit.MILLISECONDS);
+ assertEquals(0, cfs.metric.liveDiskSpaceUsed.getCount());
+ assertEquals(0, cfs.metric.totalDiskSpaceUsed.getCount());
+ }
- try (SSTableWriter writer = SSTableWriter.create(filename, 0, 0);)
+ public static SSTableReader writeFile(ColumnFamilyStore cfs, int count)
+ {
+ return Iterables.getFirst(writeFiles(cfs, 1, count * 5, count / 100, 1000), null);
+ }
+
+ public static Set<SSTableReader> writeFiles(ColumnFamilyStore cfs, int fileCount, int partitionCount, int cellCount, int cellSize)
+ {
+ int i = 0;
+ Set<SSTableReader> result = new LinkedHashSet<>();
+ for (int f = 0 ; f < fileCount ; f++)
{
- for (int i = 0; i < count * 5; i++)
+ File dir = cfs.directories.getDirectoryForNewSSTables();
+ String filename = cfs.getTempSSTablePath(dir);
+
+ SSTableWriter writer = SSTableWriter.create(filename, 0, 0);
+ int end = f == fileCount - 1 ? partitionCount : ((f + 1) * partitionCount) / fileCount;
+ for ( ; i < end ; i++)
+ {
+ ArrayBackedSortedColumns cf = ArrayBackedSortedColumns.factory.create(cfs.metadata);
+ for (int j = 0; j < cellCount ; j++)
+ cf.addColumn(Util.cellname(j), random(0, cellSize), 1);
writer.append(StorageService.getPartitioner().decorateKey(ByteBufferUtil.bytes(i)), cf);
- return writer.finish(true);
+ }
+ result.add(writer.finish(true));
}
+ return result;
}
- private void validateCFS(ColumnFamilyStore cfs)
+ public static void validateCFS(ColumnFamilyStore cfs)
{
Set<Integer> liveDescriptors = new HashSet<>();
+ long spaceUsed = 0;
for (SSTableReader sstable : cfs.getSSTables())
{
assertFalse(sstable.isMarkedCompacted());
assertEquals(1, sstable.selfRef().globalCount());
liveDescriptors.add(sstable.descriptor.generation);
+ spaceUsed += sstable.bytesOnDisk();
}
for (File dir : cfs.directories.getCFDirectories())
{
@@ -849,11 +863,13 @@ public class SSTableRewriterTest extends SchemaLoader
}
}
}
- assertTrue(cfs.getDataTracker().getCompacting().isEmpty());
+ assertEquals(spaceUsed, cfs.metric.liveDiskSpaceUsed.getCount());
+ assertEquals(spaceUsed, cfs.metric.totalDiskSpaceUsed.getCount());
+ assertTrue(cfs.getTracker().getCompacting().isEmpty());
}
- private int assertFileCounts(String [] files, int expectedtmplinkCount, int expectedtmpCount)
+ public static int assertFileCounts(String [] files, int expectedtmplinkCount, int expectedtmpCount)
{
int tmplinkcount = 0;
int tmpcount = 0;
@@ -874,7 +890,7 @@ public class SSTableRewriterTest extends SchemaLoader
return datacount;
}
- private SSTableWriter getWriter(ColumnFamilyStore cfs, File directory)
+ public static SSTableWriter getWriter(ColumnFamilyStore cfs, File directory)
{
String filename = cfs.getTempSSTablePath(directory);
return SSTableWriter.create(filename, 0, 0);
[4/6] cassandra git commit: Extend Transactional API to sstable
lifecycle management
Posted by be...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java b/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
new file mode 100644
index 0000000..acc9747
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
@@ -0,0 +1,511 @@
+/*
+ * 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.db.lifecycle;
+
+import java.util.*;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Function;
+import com.google.common.base.Predicate;
+import com.google.common.collect.*;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader.UniqueIdentifier;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import static com.google.common.base.Functions.compose;
+import static com.google.common.base.Predicates.*;
+import static com.google.common.collect.ImmutableSet.copyOf;
+import static com.google.common.collect.Iterables.*;
+import static java.util.Collections.singleton;
+import static org.apache.cassandra.db.lifecycle.Helpers.*;
+import static org.apache.cassandra.db.lifecycle.View.updateCompacting;
+import static org.apache.cassandra.db.lifecycle.View.updateLiveSet;
+import static org.apache.cassandra.utils.Throwables.maybeFail;
+import static org.apache.cassandra.utils.concurrent.Refs.release;
+import static org.apache.cassandra.utils.concurrent.Refs.selfRefs;
+
+public class LifecycleTransaction extends Transactional.AbstractTransactional
+{
+ private static final Logger logger = LoggerFactory.getLogger(LifecycleTransaction.class);
+
+ /**
+ * a class that represents accumulated modifications to the Tracker.
+ * has two instances, one containing modifications that are "staged" (i.e. invisible)
+ * and one containing those "logged" that have been made visible through a call to checkpoint()
+ */
+ private static class State
+ {
+ // readers that are either brand new, update a previous new reader, or update one of the original readers
+ final Set<SSTableReader> update = new HashSet<>();
+ // disjoint from update, represents a subset of originals that is no longer needed
+ final Set<SSTableReader> obsolete = new HashSet<>();
+
+ void log(State staged)
+ {
+ update.removeAll(staged.obsolete);
+ update.removeAll(staged.update);
+ update.addAll(staged.update);
+ obsolete.addAll(staged.obsolete);
+ }
+
+ boolean contains(SSTableReader reader)
+ {
+ return update.contains(reader) || obsolete.contains(reader);
+ }
+
+ boolean isEmpty()
+ {
+ return update.isEmpty() && obsolete.isEmpty();
+ }
+
+ void clear()
+ {
+ update.clear();
+ obsolete.clear();
+ }
+ }
+
+ public final Tracker tracker;
+ private final OperationType operationType;
+ // the original readers this transaction was opened over, and that it guards
+ // (no other transactions may operate over these readers concurrently)
+ private final Set<SSTableReader> originals = new HashSet<>();
+ // the set of readers we've marked as compacting (only updated on creation and in checkpoint())
+ private final Set<SSTableReader> marked = new HashSet<>();
+ // the identity set of readers we've ever encountered; used to ensure we don't accidentally revisit the
+ // same version of a reader. potentially a dangerous property if there are reference counting bugs
+ // as they won't be caught until the transaction's lifespan is over.
+ private final Set<UniqueIdentifier> identities = Collections.newSetFromMap(new IdentityHashMap<UniqueIdentifier, Boolean>());
+
+ // changes that have been made visible
+ private final State logged = new State();
+ // changes that are pending
+ private final State staged = new State();
+
+ /**
+ * construct a Transaction for use in an offline operation
+ */
+ public static LifecycleTransaction offline(OperationType operationType, SSTableReader reader)
+ {
+ return offline(operationType, singleton(reader));
+ }
+
+ /**
+ * construct a Transaction for use in an offline operation
+ */
+ public static LifecycleTransaction offline(OperationType operationType, Iterable<SSTableReader> readers)
+ {
+ // if offline, for simplicity we just use a dummy tracker
+ Tracker dummy = new Tracker(null, false);
+ dummy.addInitialSSTables(readers);
+ dummy.apply(updateCompacting(emptySet(), readers));
+ return new LifecycleTransaction(dummy, operationType, readers);
+ }
+
+ LifecycleTransaction(Tracker tracker, OperationType operationType, Iterable<SSTableReader> readers)
+ {
+ this.tracker = tracker;
+ this.operationType = operationType;
+ for (SSTableReader reader : readers)
+ {
+ originals.add(reader);
+ marked.add(reader);
+ identities.add(reader.instanceId);
+ }
+ }
+
+ public void doPrepare()
+ {
+ // note for future: in anticompaction two different operations use the same Transaction, and both prepareToCommit()
+ // separately: the second prepareToCommit is ignored as a "redundant" transition. since it is only a checkpoint
+ // (and these happen anyway) this is fine but if more logic gets inserted here than is performed in a checkpoint,
+ // it may break this use case, and care is needed
+ checkpoint();
+ }
+
+ /**
+ * point of no return: commit all changes, but leave all readers marked as compacting
+ */
+ public Throwable doCommit(Throwable accumulate)
+ {
+ assert staged.isEmpty() : "must be no actions introduced between prepareToCommit and a commit";
+
+ logger.debug("Committing update:{}, obsolete:{}", staged.update, staged.obsolete);
+
+ // this is now the point of no return; we cannot safely rollback, so we ignore exceptions until we're done
+ // we restore state by obsoleting our obsolete files, releasing our references to them, and updating our size
+ // and notification status for the obsolete and new files
+ accumulate = setupDeleteNotification(logged.update, tracker, accumulate);
+ accumulate = markObsolete(logged.obsolete, accumulate);
+ accumulate = tracker.updateSizeTracking(logged.obsolete, logged.update, accumulate);
+ accumulate = release(selfRefs(logged.obsolete), accumulate);
+ accumulate = tracker.notifySSTablesChanged(originals, logged.update, operationType, accumulate);
+ return accumulate;
+ }
+
+ /**
+ * undo all of the changes made by this transaction, resetting the state to its original form
+ */
+ public Throwable doAbort(Throwable accumulate)
+ {
+ if (logger.isDebugEnabled())
+ logger.debug("Aborting transaction over {}, with ({},{}) logged and ({},{}) staged", originals, logged.update, logged.obsolete, staged.update, staged.obsolete);
+
+ if (logged.isEmpty() && staged.isEmpty())
+ return accumulate;
+
+ // mark obsolete all readers that are not versions of those present in the original set
+ Iterable<SSTableReader> obsolete = filterOut(concatUniq(staged.update, logged.update), originals);
+ logger.debug("Obsoleting {}", obsolete);
+ accumulate = markObsolete(obsolete, accumulate);
+
+ // replace all updated readers with a version restored to its original state
+ accumulate = tracker.apply(updateLiveSet(logged.update, restoreUpdatedOriginals()), accumulate);
+ // setReplaced immediately preceding versions that have not been obsoleted
+ accumulate = setReplaced(logged.update, accumulate);
+ // we have replaced all of logged.update and never made visible staged.update,
+ // and the files we have logged as obsolete we clone fresh versions of, so they are no longer needed either
+ // any _staged_ obsoletes should either be in staged.update already, and dealt with there,
+ // or is still in its original form (so left as is); in either case no extra action is needed
+ accumulate = release(selfRefs(concat(staged.update, logged.update, logged.obsolete)), accumulate);
+ logged.clear();
+ staged.clear();
+ return accumulate;
+ }
+
+ @Override
+ protected Throwable doPostCleanup(Throwable accumulate)
+ {
+ return unmarkCompacting(marked, accumulate);
+ }
+
+ public void permitRedundantTransitions()
+ {
+ super.permitRedundantTransitions();
+ }
+
+ /**
+ * call when a consistent batch of changes is ready to be made atomically visible
+ * these will be exposed in the Tracker atomically, or an exception will be thrown; in this case
+ * the transaction should be rolled back
+ */
+ public void checkpoint()
+ {
+ maybeFail(checkpoint(null));
+ }
+ private Throwable checkpoint(Throwable accumulate)
+ {
+ if (logger.isDebugEnabled())
+ logger.debug("Checkpointing update:{}, obsolete:{}", staged.update, staged.obsolete);
+
+ if (staged.isEmpty())
+ return accumulate;
+
+ Set<SSTableReader> toUpdate = toUpdate();
+ Set<SSTableReader> fresh = copyOf(fresh());
+
+ // check the current versions of the readers we're replacing haven't somehow been replaced by someone else
+ checkNotReplaced(filterIn(toUpdate, staged.update));
+
+ // ensure any new readers are in the compacting set, since we aren't done with them yet
+ // and don't want anyone else messing with them
+ // apply atomically along with updating the live set of readers
+ tracker.apply(compose(updateCompacting(emptySet(), fresh),
+ updateLiveSet(toUpdate, staged.update)));
+
+ // log the staged changes and our newly marked readers
+ marked.addAll(fresh);
+ logged.log(staged);
+
+ // setup our tracker, and mark our prior versions replaced, also releasing our references to them
+ // we do not replace/release obsoleted readers, since we may need to restore them on rollback
+ accumulate = setReplaced(filterOut(toUpdate, staged.obsolete), accumulate);
+ accumulate = release(selfRefs(filterOut(toUpdate, staged.obsolete)), accumulate);
+
+ staged.clear();
+ return accumulate;
+ }
+
+ /**
+ * update a reader: if !original, this is a reader that is being introduced by this transaction;
+ * otherwise it must be in the originals() set, i.e. a reader guarded by this transaction
+ */
+ public void update(SSTableReader reader, boolean original)
+ {
+ assert !staged.update.contains(reader) : "each reader may only be updated once per checkpoint: " + reader;
+ assert !identities.contains(reader.instanceId) : "each reader instance may only be provided as an update once: " + reader;
+ // check it isn't obsolete, and that it matches the original flag
+ assert !(logged.obsolete.contains(reader) || staged.obsolete.contains(reader)) : "may not update a reader that has been obsoleted";
+ assert original == originals.contains(reader) : String.format("the 'original' indicator was incorrect (%s provided): %s", original, reader);
+ staged.update.add(reader);
+ identities.add(reader.instanceId);
+ reader.setupKeyCache();
+ }
+
+ /**
+ * mark this reader as for obsoletion. this does not actually obsolete the reader until commit() is called,
+ * but on checkpoint() the reader will be removed from the live set
+ */
+ public void obsolete(SSTableReader reader)
+ {
+ logger.debug("Staging for obsolescence {}", reader);
+ // check this is: a reader guarded by the transaction, an instance we have already worked with
+ // and that we haven't already obsoleted it, nor do we have other changes staged for it
+ assert identities.contains(reader.instanceId) : "only reader instances that have previously been provided may be obsoleted: " + reader;
+ assert originals.contains(reader) : "only readers in the 'original' set may be obsoleted: " + reader + " vs " + originals;
+ assert !(logged.obsolete.contains(reader) || staged.obsolete.contains(reader)) : "may not obsolete a reader that has already been obsoleted: " + reader;
+ assert !staged.update.contains(reader) : "may not obsolete a reader that has a staged update (must checkpoint first): " + reader;
+ assert current(reader) == reader : "may only obsolete the latest version of the reader: " + reader;
+ staged.obsolete.add(reader);
+ }
+
+ /**
+ * obsolete every file in the original transaction
+ */
+ public void obsoleteOriginals()
+ {
+ logger.debug("Staging for obsolescence {}", originals);
+ // if we're obsoleting, we should have no staged updates for the original files
+ assert Iterables.isEmpty(filterIn(staged.update, originals)) : staged.update;
+
+ // stage obsoletes for any currently visible versions of any original readers
+ Iterables.addAll(staged.obsolete, filterIn(current(), originals));
+ }
+
+ /**
+ * return the readers we're replacing in checkpoint(), i.e. the currently visible version of those in staged
+ */
+ private Set<SSTableReader> toUpdate()
+ {
+ return copyOf(filterIn(current(), staged.obsolete, staged.update));
+ }
+
+ /**
+ * new readers that haven't appeared previously (either in the original set or the logged updates)
+ */
+ private Iterable<SSTableReader> fresh()
+ {
+ return filterOut(staged.update,
+ originals, logged.update);
+ }
+
+ /**
+ * returns the currently visible readers managed by this transaction
+ */
+ public Iterable<SSTableReader> current()
+ {
+ // i.e., those that are updates that have been logged (made visible),
+ // and any original readers that have neither been obsoleted nor updated
+ return concat(logged.update, filterOut(originals, logged.update, logged.obsolete));
+ }
+
+ /**
+ * update the current replacement of any original reader back to its original start
+ */
+ private List<SSTableReader> restoreUpdatedOriginals()
+ {
+ Iterable<SSTableReader> torestore = filterIn(originals, logged.update, logged.obsolete);
+ return ImmutableList.copyOf(transform(torestore,
+ new Function<SSTableReader, SSTableReader>()
+ {
+ public SSTableReader apply(SSTableReader reader)
+ {
+ return current(reader).cloneWithNewStart(reader.first, null);
+ }
+ }));
+ }
+
+ /**
+ * the set of readers guarded by this transaction _in their original instance/state_
+ * call current(SSTableReader) on any reader in this set to get the latest instance
+ */
+ public Set<SSTableReader> originals()
+ {
+ return Collections.unmodifiableSet(originals);
+ }
+
+ /**
+ * indicates if the reader has been marked for obsoletion
+ */
+ public boolean isObsolete(SSTableReader reader)
+ {
+ return logged.obsolete.contains(reader) || staged.obsolete.contains(reader);
+ }
+
+ /**
+ * return the current version of the provided reader, whether or not it is visible or staged;
+ * i.e. returns the first version present by testing staged, logged and originals in order.
+ */
+ public SSTableReader current(SSTableReader reader)
+ {
+ Set<SSTableReader> container;
+ if (staged.contains(reader))
+ container = staged.update.contains(reader) ? staged.update : staged.obsolete;
+ else if (logged.contains(reader))
+ container = logged.update.contains(reader) ? logged.update : logged.obsolete;
+ else if (originals.contains(reader))
+ container = originals;
+ else throw new AssertionError();
+ return select(reader, container);
+ }
+
+ /**
+ * remove the reader from the set we're modifying
+ */
+ public void cancel(SSTableReader cancel)
+ {
+ logger.debug("Cancelling {} from transaction", cancel);
+ assert originals.contains(cancel) : "may only cancel a reader in the 'original' set: " + cancel + " vs " + originals;
+ assert !(staged.contains(cancel) || logged.contains(cancel)) : "may only cancel a reader that has not been updated or obsoleted in this transaction: " + cancel;
+ originals.remove(cancel);
+ marked.remove(cancel);
+ maybeFail(unmarkCompacting(singleton(cancel), null));
+ }
+
+ /**
+ * remove the readers from the set we're modifying
+ */
+ public void cancel(Iterable<SSTableReader> cancels)
+ {
+ for (SSTableReader cancel : cancels)
+ cancel(cancel);
+ }
+
+ /**
+ * remove the provided readers from this Transaction, and return a new Transaction to manage them
+ * only permitted to be called if the current Transaction has never been used
+ */
+ public LifecycleTransaction split(Collection<SSTableReader> readers)
+ {
+ logger.debug("Splitting {} into new transaction", readers);
+ checkUnused();
+ for (SSTableReader reader : readers)
+ assert identities.contains(reader.instanceId) : "may only split the same reader instance the transaction was opened with: " + reader;
+
+ for (SSTableReader reader : readers)
+ {
+ identities.remove(reader.instanceId);
+ originals.remove(reader);
+ marked.remove(reader);
+ }
+ return new LifecycleTransaction(tracker, operationType, readers);
+ }
+
+ /**
+ * check this transaction has never been used
+ */
+ private void checkUnused()
+ {
+ assert logged.isEmpty();
+ assert staged.isEmpty();
+ assert identities.size() == originals.size();
+ assert originals.size() == marked.size();
+ }
+
+ private Throwable unmarkCompacting(Set<SSTableReader> unmark, Throwable accumulate)
+ {
+ accumulate = tracker.apply(updateCompacting(unmark, emptySet()), accumulate);
+ // when the CFS is invalidated, it will call unreferenceSSTables(). However, unreferenceSSTables only deals
+ // with sstables that aren't currently being compacted. If there are ongoing compactions that finish or are
+ // interrupted after the CFS is invalidated, those sstables need to be unreferenced as well, so we do that here.
+ accumulate = tracker.dropSSTablesIfInvalid(accumulate);
+ return accumulate;
+ }
+
+ // convenience method for callers that know only one sstable is involved in the transaction
+ public SSTableReader onlyOne()
+ {
+ assert originals.size() == 1;
+ return getFirst(originals, null);
+ }
+
+ // a class representing the current state of the reader within this transaction, encoding the actions both logged
+ // and pending, and the reader instances that are visible now, and will be after the next checkpoint (with null
+ // indicating either obsolescence, or that the reader does not occur in the transaction; which is defined
+ // by the corresponding Action)
+ @VisibleForTesting
+ public static class ReaderState
+ {
+ public static enum Action
+ {
+ UPDATED, OBSOLETED, NONE;
+ public static Action get(boolean updated, boolean obsoleted)
+ {
+ assert !(updated && obsoleted);
+ return updated ? UPDATED : obsoleted ? OBSOLETED : NONE;
+ }
+ }
+
+ final Action staged;
+ final Action logged;
+ final SSTableReader nextVisible;
+ final SSTableReader currentlyVisible;
+ final boolean original;
+
+ public ReaderState(Action logged, Action staged, SSTableReader currentlyVisible, SSTableReader nextVisible, boolean original)
+ {
+ this.staged = staged;
+ this.logged = logged;
+ this.currentlyVisible = currentlyVisible;
+ this.nextVisible = nextVisible;
+ this.original = original;
+ }
+
+ public boolean equals(Object that)
+ {
+ return that instanceof ReaderState && equals((ReaderState) that);
+ }
+
+ public boolean equals(ReaderState that)
+ {
+ return this.staged == that.staged && this.logged == that.logged && this.original == that.original
+ && this.currentlyVisible == that.currentlyVisible && this.nextVisible == that.nextVisible;
+ }
+
+ public String toString()
+ {
+ return String.format("[logged=%s staged=%s original=%s]", logged, staged, original);
+ }
+
+ public static SSTableReader visible(SSTableReader reader, Predicate<SSTableReader> obsolete, Collection<SSTableReader> ... selectFrom)
+ {
+ return obsolete.apply(reader) ? null : selectFirst(reader, selectFrom);
+ }
+ }
+
+ @VisibleForTesting
+ public ReaderState state(SSTableReader reader)
+ {
+ SSTableReader currentlyVisible = ReaderState.visible(reader, in(logged.obsolete), logged.update, originals);
+ SSTableReader nextVisible = ReaderState.visible(reader, orIn(staged.obsolete, logged.obsolete), staged.update, logged.update, originals);
+ return new ReaderState(ReaderState.Action.get(logged.update.contains(reader), logged.obsolete.contains(reader)),
+ ReaderState.Action.get(staged.update.contains(reader), staged.obsolete.contains(reader)),
+ currentlyVisible, nextVisible, originals.contains(reader)
+ );
+ }
+
+ public String toString()
+ {
+ return originals.toString();
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/lifecycle/SSTableIntervalTree.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/SSTableIntervalTree.java b/src/java/org/apache/cassandra/db/lifecycle/SSTableIntervalTree.java
new file mode 100644
index 0000000..ff2abcb
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/lifecycle/SSTableIntervalTree.java
@@ -0,0 +1,40 @@
+package org.apache.cassandra.db.lifecycle;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import com.google.common.collect.Iterables;
+
+import org.apache.cassandra.db.RowPosition;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.utils.Interval;
+import org.apache.cassandra.utils.IntervalTree;
+
+public class SSTableIntervalTree extends IntervalTree<RowPosition, SSTableReader, Interval<RowPosition, SSTableReader>>
+{
+ private static final SSTableIntervalTree EMPTY = new SSTableIntervalTree(null);
+
+ SSTableIntervalTree(Collection<Interval<RowPosition, SSTableReader>> intervals)
+ {
+ super(intervals);
+ }
+
+ public static SSTableIntervalTree empty()
+ {
+ return EMPTY;
+ }
+
+ public static SSTableIntervalTree build(Iterable<SSTableReader> sstables)
+ {
+ return new SSTableIntervalTree(buildIntervals(sstables));
+ }
+
+ public static List<Interval<RowPosition, SSTableReader>> buildIntervals(Iterable<SSTableReader> sstables)
+ {
+ List<Interval<RowPosition, SSTableReader>> intervals = new ArrayList<>(Iterables.size(sstables));
+ for (SSTableReader sstable : sstables)
+ intervals.add(Interval.<RowPosition, SSTableReader>create(sstable.first, sstable.last, sstable));
+ return intervals;
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
new file mode 100644
index 0000000..50f567f
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
@@ -0,0 +1,468 @@
+/*
+ * 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.db.lifecycle;
+
+import java.io.File;
+import java.util.*;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.atomic.AtomicReference;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Function;
+import com.google.common.base.Predicate;
+import com.google.common.base.Predicates;
+import com.google.common.collect.*;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Directories;
+import org.apache.cassandra.db.Memtable;
+import org.apache.cassandra.db.commitlog.ReplayPosition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.metrics.StorageMetrics;
+import org.apache.cassandra.notifications.*;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.concurrent.OpOrder;
+
+import static com.google.common.base.Predicates.and;
+import static com.google.common.collect.ImmutableSet.copyOf;
+import static com.google.common.collect.Iterables.filter;
+import static java.util.Collections.singleton;
+import static org.apache.cassandra.db.lifecycle.Helpers.*;
+import static org.apache.cassandra.db.lifecycle.View.permitCompacting;
+import static org.apache.cassandra.db.lifecycle.View.updateCompacting;
+import static org.apache.cassandra.db.lifecycle.View.updateLiveSet;
+import static org.apache.cassandra.utils.Throwables.maybeFail;
+import static org.apache.cassandra.utils.Throwables.merge;
+import static org.apache.cassandra.utils.concurrent.Refs.release;
+import static org.apache.cassandra.utils.concurrent.Refs.selfRefs;
+
+public class Tracker
+{
+ private static final Logger logger = LoggerFactory.getLogger(Tracker.class);
+
+ public final Collection<INotificationConsumer> subscribers = new CopyOnWriteArrayList<>();
+ public final ColumnFamilyStore cfstore;
+ final AtomicReference<View> view;
+ public final boolean loadsstables;
+
+ public Tracker(ColumnFamilyStore cfstore, boolean loadsstables)
+ {
+ this.cfstore = cfstore;
+ this.view = new AtomicReference<>();
+ this.loadsstables = loadsstables;
+ this.reset();
+ }
+
+ public LifecycleTransaction tryModify(SSTableReader sstable, OperationType operationType)
+ {
+ return tryModify(singleton(sstable), operationType);
+ }
+
+ /**
+ * @return a Transaction over the provided sstables if we are able to mark the given @param sstables as compacted, before anyone else
+ */
+ public LifecycleTransaction tryModify(Iterable<SSTableReader> sstables, OperationType operationType)
+ {
+ if (Iterables.isEmpty(sstables))
+ return new LifecycleTransaction(this, operationType, sstables);
+ if (null == apply(permitCompacting(sstables), updateCompacting(emptySet(), sstables)))
+ return null;
+ return new LifecycleTransaction(this, operationType, sstables);
+ }
+
+
+ // METHODS FOR ATOMICALLY MODIFYING THE VIEW
+
+ Pair<View, View> apply(Function<View, View> function)
+ {
+ return apply(Predicates.<View>alwaysTrue(), function);
+ }
+
+ Throwable apply(Function<View, View> function, Throwable accumulate)
+ {
+ try
+ {
+ apply(function);
+ }
+ catch (Throwable t)
+ {
+ accumulate = merge(accumulate, t);
+ }
+ return accumulate;
+ }
+
+ /**
+ * atomically tests permit against the view and applies function to it, if permit yields true, returning the original;
+ * otherwise the method aborts, returning null
+ */
+ Pair<View, View> apply(Predicate<View> permit, Function<View, View> function)
+ {
+ while (true)
+ {
+ View cur = view.get();
+ if (!permit.apply(cur))
+ return null;
+ View updated = function.apply(cur);
+ if (view.compareAndSet(cur, updated))
+ return Pair.create(cur, updated);
+ }
+ }
+
+ Throwable updateSizeTracking(Iterable<SSTableReader> oldSSTables, Iterable<SSTableReader> newSSTables, Throwable accumulate)
+ {
+ if (cfstore == null)
+ return accumulate;
+
+ long add = 0;
+ for (SSTableReader sstable : newSSTables)
+ {
+ if (logger.isDebugEnabled())
+ logger.debug("adding {} to list of files tracked for {}.{}", sstable.descriptor, cfstore.keyspace.getName(), cfstore.name);
+ try
+ {
+ add += sstable.bytesOnDisk();
+ }
+ catch (Throwable t)
+ {
+ accumulate = merge(accumulate, t);
+ }
+ }
+ long subtract = 0;
+ for (SSTableReader sstable : oldSSTables)
+ {
+ if (logger.isDebugEnabled())
+ logger.debug("removing {} from list of files tracked for {}.{}", sstable.descriptor, cfstore.keyspace.getName(), cfstore.name);
+ try
+ {
+ subtract += sstable.bytesOnDisk();
+ }
+ catch (Throwable t)
+ {
+ accumulate = merge(accumulate, t);
+ }
+ }
+ StorageMetrics.load.inc(add - subtract);
+ cfstore.metric.liveDiskSpaceUsed.inc(add - subtract);
+ // we don't subtract from total until the sstable is deleted
+ cfstore.metric.totalDiskSpaceUsed.inc(add);
+ return accumulate;
+ }
+
+ // SETUP / CLEANUP
+
+ public void addInitialSSTables(Iterable<SSTableReader> sstables)
+ {
+ maybeFail(setupDeleteNotification(sstables, this, null));
+ apply(updateLiveSet(emptySet(), sstables));
+ maybeFail(updateSizeTracking(emptySet(), sstables, null));
+ // no notifications or backup necessary
+ }
+
+ public void addSSTables(Iterable<SSTableReader> sstables)
+ {
+ addInitialSSTables(sstables);
+ for (SSTableReader sstable : sstables)
+ {
+ maybeIncrementallyBackup(sstable);
+ notifyAdded(sstable);
+ }
+ }
+
+ /** (Re)initializes the tracker, purging all references. */
+ @VisibleForTesting
+ public void reset()
+ {
+ view.set(new View(
+ cfstore != null ? ImmutableList.of(new Memtable(cfstore)) : Collections.<Memtable>emptyList(),
+ ImmutableList.<Memtable>of(),
+ Collections.<SSTableReader, SSTableReader>emptyMap(),
+ Collections.<SSTableReader>emptySet(),
+ SSTableIntervalTree.empty()));
+ }
+
+ public Throwable dropSSTablesIfInvalid(Throwable accumulate)
+ {
+ if (cfstore != null && !cfstore.isValid())
+ accumulate = dropSSTables(accumulate);
+ return accumulate;
+ }
+
+ public void dropSSTables()
+ {
+ maybeFail(dropSSTables(null));
+ }
+
+ public Throwable dropSSTables(Throwable accumulate)
+ {
+ return dropSSTables(Predicates.<SSTableReader>alwaysTrue(), OperationType.UNKNOWN, accumulate);
+ }
+
+ /**
+ * removes all sstables that are not busy compacting.
+ */
+ public Throwable dropSSTables(final Predicate<SSTableReader> remove, OperationType operationType, Throwable accumulate)
+ {
+ Pair<View, View> result = apply(new Function<View, View>()
+ {
+ public View apply(View view)
+ {
+ Set<SSTableReader> toremove = copyOf(filter(view.sstables, and(remove, notIn(view.compacting))));
+ return updateLiveSet(toremove, emptySet()).apply(view);
+ }
+ });
+
+ Set<SSTableReader> removed = Sets.difference(result.left.sstables, result.right.sstables);
+ assert Iterables.all(removed, remove);
+
+ if (!removed.isEmpty())
+ {
+ // notifySSTablesChanged -> LeveledManifest.promote doesn't like a no-op "promotion"
+ accumulate = notifySSTablesChanged(removed, Collections.<SSTableReader>emptySet(), operationType, accumulate);
+ accumulate = updateSizeTracking(removed, emptySet(), accumulate);
+ accumulate = markObsolete(removed, accumulate);
+ accumulate = release(selfRefs(removed), accumulate);
+ }
+ return accumulate;
+ }
+
+ /**
+ * Removes every SSTable in the directory from the Tracker's view.
+ * @param directory the unreadable directory, possibly with SSTables in it, but not necessarily.
+ */
+ public void removeUnreadableSSTables(final File directory)
+ {
+ maybeFail(dropSSTables(new Predicate<SSTableReader>()
+ {
+ public boolean apply(SSTableReader reader)
+ {
+ return reader.descriptor.directory.equals(directory);
+ }
+ }, OperationType.UNKNOWN, null));
+ }
+
+
+
+ // FLUSHING
+
+ /**
+ * get the Memtable that the ordered writeOp should be directed to
+ */
+ public Memtable getMemtableFor(OpOrder.Group opGroup, ReplayPosition replayPosition)
+ {
+ // since any new memtables appended to the list after we fetch it will be for operations started
+ // after us, we can safely assume that we will always find the memtable that 'accepts' us;
+ // if the barrier for any memtable is set whilst we are reading the list, it must accept us.
+
+ // there may be multiple memtables in the list that would 'accept' us, however we only ever choose
+ // the oldest such memtable, as accepts() only prevents us falling behind (i.e. ensures we don't
+ // assign operations to a memtable that was retired/queued before we started)
+ for (Memtable memtable : view.get().liveMemtables)
+ {
+ if (memtable.accepts(opGroup, replayPosition))
+ return memtable;
+ }
+ throw new AssertionError(view.get().liveMemtables.toString());
+ }
+
+ /**
+ * Switch the current memtable. This atomically appends a new memtable to the end of the list of active memtables,
+ * returning the previously last memtable. It leaves the previous Memtable in the list of live memtables until
+ * discarding(memtable) is called. These two methods must be synchronized/paired, i.e. m = switchMemtable
+ * must be followed by discarding(m), they cannot be interleaved.
+ *
+ * @return the previously active memtable
+ */
+ public Memtable switchMemtable(boolean truncating)
+ {
+ Memtable newMemtable = new Memtable(cfstore);
+ Pair<View, View> result = apply(View.switchMemtable(newMemtable));
+ if (truncating)
+ notifyRenewed(newMemtable);
+
+ return result.left.getCurrentMemtable();
+ }
+
+ public void markFlushing(Memtable memtable)
+ {
+ apply(View.markFlushing(memtable));
+ }
+
+ public void replaceFlushed(Memtable memtable, SSTableReader sstable)
+ {
+ if (sstable == null)
+ {
+ // sstable may be null if we flushed batchlog and nothing needed to be retained
+ // if it's null, we don't care what state the cfstore is in, we just replace it and continue
+ apply(View.replaceFlushed(memtable, null));
+ return;
+ }
+
+ sstable.setupDeleteNotification(this);
+ sstable.setupKeyCache();
+ // back up before creating a new Snapshot (which makes the new one eligible for compaction)
+ maybeIncrementallyBackup(sstable);
+
+ apply(View.replaceFlushed(memtable, sstable));
+
+ Throwable fail;
+ fail = updateSizeTracking(emptySet(), singleton(sstable), null);
+ // TODO: if we're invalidated, should we notifyadded AND removed, or just skip both?
+ fail = notifyAdded(sstable, fail);
+
+ if (cfstore != null && !cfstore.isValid())
+ dropSSTables();
+
+ maybeFail(fail);
+ }
+
+
+
+ // MISCELLANEOUS public utility calls
+
+ public Set<SSTableReader> getSSTables()
+ {
+ return view.get().sstables;
+ }
+
+ public Set<SSTableReader> getCompacting()
+ {
+ return view.get().compacting;
+ }
+
+ public Set<SSTableReader> getUncompacting()
+ {
+ return view.get().nonCompactingSStables();
+ }
+
+ public Iterable<SSTableReader> getUncompacting(Iterable<SSTableReader> candidates)
+ {
+ return view.get().getUncompacting(candidates);
+ }
+
+ public void maybeIncrementallyBackup(final SSTableReader sstable)
+ {
+ if (!DatabaseDescriptor.isIncrementalBackupsEnabled())
+ return;
+
+ File backupsDir = Directories.getBackupsDirectory(sstable.descriptor);
+ sstable.createLinks(FileUtils.getCanonicalPath(backupsDir));
+ }
+
+ public void spaceReclaimed(long size)
+ {
+ if (cfstore != null)
+ cfstore.metric.totalDiskSpaceUsed.dec(size);
+ }
+
+
+
+ // NOTIFICATION
+
+ Throwable notifySSTablesChanged(Collection<SSTableReader> removed, Collection<SSTableReader> added, OperationType compactionType, Throwable accumulate)
+ {
+ INotification notification = new SSTableListChangedNotification(added, removed, compactionType);
+ for (INotificationConsumer subscriber : subscribers)
+ {
+ try
+ {
+ subscriber.handleNotification(notification, this);
+ }
+ catch (Throwable t)
+ {
+ accumulate = merge(accumulate, t);
+ }
+ }
+ return accumulate;
+ }
+
+ Throwable notifyAdded(SSTableReader added, Throwable accumulate)
+ {
+ INotification notification = new SSTableAddedNotification(added);
+ for (INotificationConsumer subscriber : subscribers)
+ {
+ try
+ {
+ subscriber.handleNotification(notification, this);
+ }
+ catch (Throwable t)
+ {
+ accumulate = merge(accumulate, t);
+ }
+ }
+ return accumulate;
+ }
+
+ public void notifyAdded(SSTableReader added)
+ {
+ maybeFail(notifyAdded(added, null));
+ }
+
+ public void notifySSTableRepairedStatusChanged(Collection<SSTableReader> repairStatusesChanged)
+ {
+ INotification notification = new SSTableRepairStatusChanged(repairStatusesChanged);
+ for (INotificationConsumer subscriber : subscribers)
+ subscriber.handleNotification(notification, this);
+ }
+
+ public void notifyDeleting(SSTableReader deleting)
+ {
+ INotification notification = new SSTableDeletingNotification(deleting);
+ for (INotificationConsumer subscriber : subscribers)
+ subscriber.handleNotification(notification, this);
+ }
+
+ public void notifyRenewed(Memtable renewed)
+ {
+ INotification notification = new MemtableRenewedNotification(renewed);
+ for (INotificationConsumer subscriber : subscribers)
+ subscriber.handleNotification(notification, this);
+ }
+
+ public void notifyTruncated(long truncatedAt)
+ {
+ INotification notification = new TruncationNotification(truncatedAt);
+ for (INotificationConsumer subscriber : subscribers)
+ subscriber.handleNotification(notification, this);
+ }
+
+ public void subscribe(INotificationConsumer consumer)
+ {
+ subscribers.add(consumer);
+ }
+
+ public void unsubscribe(INotificationConsumer consumer)
+ {
+ subscribers.remove(consumer);
+ }
+
+ private static Set<SSTableReader> emptySet()
+ {
+ return Collections.emptySet();
+ }
+
+ public View getView()
+ {
+ return view.get();
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/lifecycle/View.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/View.java b/src/java/org/apache/cassandra/db/lifecycle/View.java
new file mode 100644
index 0000000..0d1100b
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/lifecycle/View.java
@@ -0,0 +1,252 @@
+/*
+ * 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.db.lifecycle;
+
+import java.util.*;
+
+import com.google.common.base.Function;
+import com.google.common.base.Functions;
+import com.google.common.base.Predicate;
+import com.google.common.collect.*;
+
+import org.apache.cassandra.db.Memtable;
+import org.apache.cassandra.db.RowPosition;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.utils.Interval;
+
+import static com.google.common.base.Predicates.equalTo;
+import static com.google.common.base.Predicates.not;
+import static com.google.common.collect.ImmutableList.copyOf;
+import static com.google.common.collect.ImmutableList.of;
+import static com.google.common.collect.Iterables.all;
+import static com.google.common.collect.Iterables.concat;
+import static com.google.common.collect.Iterables.filter;
+import static java.util.Collections.singleton;
+import static org.apache.cassandra.db.lifecycle.Helpers.emptySet;
+import static org.apache.cassandra.db.lifecycle.Helpers.replace;
+
+/**
+ * An immutable structure holding the current memtable, the memtables pending
+ * flush, the sstables for a column family, and the sstables that are active
+ * in compaction (a subset of the sstables).
+ *
+ * Modifications to instances are all performed via a Function produced by the static methods in this class.
+ * These are composed as necessary and provided to the Tracker.apply() methods, which atomically reject or
+ * accept and apply the changes to the View.
+ *
+ */
+public class View
+{
+ /**
+ * ordinarily a list of size 1, but when preparing to flush will contain both the memtable we will flush
+ * and the new replacement memtable, until all outstanding write operations on the old table complete.
+ * The last item in the list is always the "current" memtable.
+ */
+ public final List<Memtable> liveMemtables;
+ /**
+ * contains all memtables that are no longer referenced for writing and are queued for / in the process of being
+ * flushed. In chronologically ascending order.
+ */
+ public final List<Memtable> flushingMemtables;
+ public final Set<SSTableReader> compacting;
+ public final Set<SSTableReader> sstables;
+ // we use a Map here so that we can easily perform identity checks as well as equality checks.
+ // When marking compacting, we now indicate if we expect the sstables to be present (by default we do),
+ // and we then check that not only are they all present in the live set, but that the exact instance present is
+ // the one we made our decision to compact against.
+ public final Map<SSTableReader, SSTableReader> sstablesMap;
+
+ public final SSTableIntervalTree intervalTree;
+
+ View(List<Memtable> liveMemtables, List<Memtable> flushingMemtables, Map<SSTableReader, SSTableReader> sstables, Set<SSTableReader> compacting, SSTableIntervalTree intervalTree)
+ {
+ assert liveMemtables != null;
+ assert flushingMemtables != null;
+ assert sstables != null;
+ assert compacting != null;
+ assert intervalTree != null;
+
+ this.liveMemtables = liveMemtables;
+ this.flushingMemtables = flushingMemtables;
+
+ this.sstablesMap = sstables;
+ this.sstables = sstablesMap.keySet();
+ this.compacting = compacting;
+ this.intervalTree = intervalTree;
+ }
+
+ public Memtable getCurrentMemtable()
+ {
+ return liveMemtables.get(liveMemtables.size() - 1);
+ }
+
+ /**
+ * @return the active memtable and all the memtables that are pending flush.
+ */
+ public Iterable<Memtable> getAllMemtables()
+ {
+ return concat(flushingMemtables, liveMemtables);
+ }
+
+ public Sets.SetView<SSTableReader> nonCompactingSStables()
+ {
+ return Sets.difference(sstables, compacting);
+ }
+
+ public Iterable<SSTableReader> getUncompacting(Iterable<SSTableReader> candidates)
+ {
+ return filter(candidates, new Predicate<SSTableReader>()
+ {
+ public boolean apply(SSTableReader sstable)
+ {
+ return !compacting.contains(sstable);
+ }
+ });
+ }
+
+ @Override
+ public String toString()
+ {
+ return String.format("View(pending_count=%d, sstables=%s, compacting=%s)", liveMemtables.size() + flushingMemtables.size() - 1, sstables, compacting);
+ }
+
+ public List<SSTableReader> sstablesInBounds(AbstractBounds<RowPosition> rowBounds)
+ {
+ if (intervalTree.isEmpty())
+ return Collections.emptyList();
+ RowPosition stopInTree = rowBounds.right.isMinimum() ? intervalTree.max() : rowBounds.right;
+ return intervalTree.search(Interval.<RowPosition, SSTableReader>create(rowBounds.left, stopInTree));
+ }
+
+ // METHODS TO CONSTRUCT FUNCTIONS FOR MODIFYING A VIEW:
+
+ // return a function to un/mark the provided readers compacting in a view
+ static Function<View, View> updateCompacting(final Set<SSTableReader> unmark, final Iterable<SSTableReader> mark)
+ {
+ if (unmark.isEmpty() && Iterables.isEmpty(mark))
+ return Functions.identity();
+ return new Function<View, View>()
+ {
+ public View apply(View view)
+ {
+ assert all(mark, Helpers.idIn(view.sstablesMap));
+ return new View(view.liveMemtables, view.flushingMemtables, view.sstablesMap,
+ replace(view.compacting, unmark, mark),
+ view.intervalTree);
+ }
+ };
+ }
+
+ // construct a predicate to reject views that do not permit us to mark these readers compacting;
+ // i.e. one of them is either already compacting, has been compacted, or has been replaced
+ static Predicate<View> permitCompacting(final Iterable<SSTableReader> readers)
+ {
+ return new Predicate<View>()
+ {
+ public boolean apply(View view)
+ {
+ for (SSTableReader reader : readers)
+ if (view.compacting.contains(reader) || view.sstablesMap.get(reader) != reader || reader.isMarkedCompacted())
+ return false;
+ return true;
+ }
+ };
+ }
+
+ // construct a function to change the liveset in a Snapshot
+ static Function<View, View> updateLiveSet(final Set<SSTableReader> remove, final Iterable<SSTableReader> add)
+ {
+ if (remove.isEmpty() && Iterables.isEmpty(add))
+ return Functions.identity();
+ return new Function<View, View>()
+ {
+ public View apply(View view)
+ {
+ Map<SSTableReader, SSTableReader> sstableMap = replace(view.sstablesMap, remove, add);
+ return new View(view.liveMemtables, view.flushingMemtables, sstableMap, view.compacting,
+ SSTableIntervalTree.build(sstableMap.keySet()));
+ }
+ };
+ }
+
+ // called prior to initiating flush: add newMemtable to liveMemtables, making it the latest memtable
+ static Function<View, View> switchMemtable(final Memtable newMemtable)
+ {
+ return new Function<View, View>()
+ {
+ public View apply(View view)
+ {
+ List<Memtable> newLive = ImmutableList.<Memtable>builder().addAll(view.liveMemtables).add(newMemtable).build();
+ assert newLive.size() == view.liveMemtables.size() + 1;
+ return new View(newLive, view.flushingMemtables, view.sstablesMap, view.compacting, view.intervalTree);
+ }
+ };
+ }
+
+ // called before flush: move toFlush from liveMemtables to flushingMemtables
+ static Function<View, View> markFlushing(final Memtable toFlush)
+ {
+ return new Function<View, View>()
+ {
+ public View apply(View view)
+ {
+ List<Memtable> live = view.liveMemtables, flushing = view.flushingMemtables;
+ List<Memtable> newLive = copyOf(filter(live, not(equalTo(toFlush))));
+ List<Memtable> newFlushing = copyOf(concat(filter(flushing, lessThan(toFlush)),
+ of(toFlush),
+ filter(flushing, not(lessThan(toFlush)))));
+ assert newLive.size() == live.size() - 1;
+ assert newFlushing.size() == flushing.size() + 1;
+ return new View(newLive, newFlushing, view.sstablesMap, view.compacting, view.intervalTree);
+ }
+ };
+ }
+
+ // called after flush: removes memtable from flushingMemtables, and inserts flushed into the live sstable set
+ static Function<View, View> replaceFlushed(final Memtable memtable, final SSTableReader flushed)
+ {
+ return new Function<View, View>()
+ {
+ public View apply(View view)
+ {
+ List<Memtable> flushingMemtables = copyOf(filter(view.flushingMemtables, not(equalTo(memtable))));
+ assert flushingMemtables.size() == view.flushingMemtables.size() - 1;
+
+ if (flushed == null)
+ return new View(view.liveMemtables, flushingMemtables, view.sstablesMap,
+ view.compacting, view.intervalTree);
+
+ Map<SSTableReader, SSTableReader> sstableMap = replace(view.sstablesMap, emptySet(), singleton(flushed));
+ return new View(view.liveMemtables, flushingMemtables, sstableMap, view.compacting,
+ SSTableIntervalTree.build(sstableMap.keySet()));
+ }
+ };
+ }
+
+ private static <T extends Comparable<T>> Predicate<T> lessThan(final T lessThan)
+ {
+ return new Predicate<T>()
+ {
+ public boolean apply(T t)
+ {
+ return t.compareTo(lessThan) < 0;
+ }
+ };
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java b/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
index a6c7a8b..c994a3d 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
@@ -403,7 +403,7 @@ public class CompressionMetadata
count = chunkIndex;
}
- protected Throwable doCleanup(Throwable failed)
+ protected Throwable doPreCleanup(Throwable failed)
{
return offsets.close(failed);
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java b/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
index 81e487c..9bfbc99 100644
--- a/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
+++ b/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
@@ -19,24 +19,15 @@ package org.apache.cassandra.io.sstable;
import java.io.IOException;
import java.lang.management.ManagementFactory;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
+import java.util.*;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import javax.management.MBeanServer;
import javax.management.ObjectName;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Multimap;
-import com.google.common.collect.Sets;
+import com.google.common.collect.*;
+
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -44,8 +35,10 @@ import org.slf4j.LoggerFactory;
import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.DataTracker;
+import org.apache.cassandra.db.compaction.OperationType;
import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.db.lifecycle.View;
import org.apache.cassandra.utils.Pair;
import org.apache.cassandra.utils.WrappedRunnable;
@@ -209,22 +202,25 @@ public class IndexSummaryManager implements IndexSummaryManagerMBean
* Returns a Pair of all compacting and non-compacting sstables. Non-compacting sstables will be marked as
* compacting.
*/
- private Pair<List<SSTableReader>, Multimap<DataTracker, SSTableReader>> getCompactingAndNonCompactingSSTables()
+ private Pair<List<SSTableReader>, Map<UUID, LifecycleTransaction>> getCompactingAndNonCompactingSSTables()
{
List<SSTableReader> allCompacting = new ArrayList<>();
- Multimap<DataTracker, SSTableReader> allNonCompacting = HashMultimap.create();
+ Map<UUID, LifecycleTransaction> allNonCompacting = new HashMap<>();
for (Keyspace ks : Keyspace.all())
{
for (ColumnFamilyStore cfStore: ks.getColumnFamilyStores())
{
Set<SSTableReader> nonCompacting, allSSTables;
+ LifecycleTransaction txn = null;
do
{
- allSSTables = cfStore.getDataTracker().getSSTables();
- nonCompacting = Sets.newHashSet(cfStore.getDataTracker().getUncompactingSSTables(allSSTables));
+ View view = cfStore.getTracker().getView();
+ allSSTables = view.sstables;
+ nonCompacting = ImmutableSet.copyOf(view.getUncompacting(allSSTables));
}
- while (!(nonCompacting.isEmpty() || cfStore.getDataTracker().markCompacting(nonCompacting)));
- allNonCompacting.putAll(cfStore.getDataTracker(), nonCompacting);
+ while (null == (txn = cfStore.getTracker().tryModify(nonCompacting, OperationType.UNKNOWN)));
+
+ allNonCompacting.put(cfStore.metadata.cfId, txn);
allCompacting.addAll(Sets.difference(allSSTables, nonCompacting));
}
}
@@ -233,50 +229,57 @@ public class IndexSummaryManager implements IndexSummaryManagerMBean
public void redistributeSummaries() throws IOException
{
- Pair<List<SSTableReader>, Multimap<DataTracker, SSTableReader>> compactingAndNonCompacting = getCompactingAndNonCompactingSSTables();
+ Pair<List<SSTableReader>, Map<UUID, LifecycleTransaction>> compactingAndNonCompacting = getCompactingAndNonCompactingSSTables();
try
{
- redistributeSummaries(compactingAndNonCompacting.left, Lists.newArrayList(compactingAndNonCompacting.right.values()), this.memoryPoolBytes);
+ redistributeSummaries(compactingAndNonCompacting.left, compactingAndNonCompacting.right, this.memoryPoolBytes);
}
finally
{
- for(DataTracker tracker : compactingAndNonCompacting.right.keySet())
- tracker.unmarkCompacting(compactingAndNonCompacting.right.get(tracker));
+ for (LifecycleTransaction modifier : compactingAndNonCompacting.right.values())
+ modifier.close();
}
}
/**
* Attempts to fairly distribute a fixed pool of memory for index summaries across a set of SSTables based on
* their recent read rates.
- * @param nonCompacting a list of sstables to share the memory pool across
+ * @param transactions containing the sstables we are to redistribute the memory pool across
* @param memoryPoolBytes a size (in bytes) that the total index summary space usage should stay close to or
* under, if possible
* @return a list of new SSTableReader instances
*/
@VisibleForTesting
- public static List<SSTableReader> redistributeSummaries(List<SSTableReader> compacting, List<SSTableReader> nonCompacting, long memoryPoolBytes) throws IOException
+ public static List<SSTableReader> redistributeSummaries(List<SSTableReader> compacting, Map<UUID, LifecycleTransaction> transactions, long memoryPoolBytes) throws IOException
{
- long total = 0;
- for (SSTableReader sstable : Iterables.concat(compacting, nonCompacting))
- total += sstable.getIndexSummaryOffHeapSize();
-
List<SSTableReader> oldFormatSSTables = new ArrayList<>();
- for (SSTableReader sstable : nonCompacting)
+ List<SSTableReader> redistribute = new ArrayList<>();
+ for (LifecycleTransaction txn : transactions.values())
{
- // We can't change the sampling level of sstables with the old format, because the serialization format
- // doesn't include the sampling level. Leave this one as it is. (See CASSANDRA-8993 for details.)
- logger.trace("SSTable {} cannot be re-sampled due to old sstable format", sstable);
- if (!sstable.descriptor.version.hasSamplingLevel())
- oldFormatSSTables.add(sstable);
+ for (SSTableReader sstable : ImmutableList.copyOf(txn.originals()))
+ {
+ // We can't change the sampling level of sstables with the old format, because the serialization format
+ // doesn't include the sampling level. Leave this one as it is. (See CASSANDRA-8993 for details.)
+ logger.trace("SSTable {} cannot be re-sampled due to old sstable format", sstable);
+ if (!sstable.descriptor.version.hasSamplingLevel())
+ {
+ oldFormatSSTables.add(sstable);
+ txn.cancel(sstable);
+ }
+ }
+ redistribute.addAll(txn.originals());
}
- nonCompacting.removeAll(oldFormatSSTables);
+
+ long total = 0;
+ for (SSTableReader sstable : Iterables.concat(compacting, redistribute))
+ total += sstable.getIndexSummaryOffHeapSize();
logger.debug("Beginning redistribution of index summaries for {} sstables with memory pool size {} MB; current spaced used is {} MB",
- nonCompacting.size(), memoryPoolBytes / 1024L / 1024L, total / 1024.0 / 1024.0);
+ redistribute.size(), memoryPoolBytes / 1024L / 1024L, total / 1024.0 / 1024.0);
- final Map<SSTableReader, Double> readRates = new HashMap<>(nonCompacting.size());
+ final Map<SSTableReader, Double> readRates = new HashMap<>(redistribute.size());
double totalReadsPerSec = 0.0;
- for (SSTableReader sstable : nonCompacting)
+ for (SSTableReader sstable : redistribute)
{
if (sstable.getReadMeter() != null)
{
@@ -288,7 +291,7 @@ public class IndexSummaryManager implements IndexSummaryManagerMBean
logger.trace("Total reads/sec across all sstables in index summary resize process: {}", totalReadsPerSec);
// copy and sort by read rates (ascending)
- List<SSTableReader> sstablesByHotness = new ArrayList<>(nonCompacting);
+ List<SSTableReader> sstablesByHotness = new ArrayList<>(redistribute);
Collections.sort(sstablesByHotness, new ReadRateComparator(readRates));
long remainingBytes = memoryPoolBytes;
@@ -297,7 +300,10 @@ public class IndexSummaryManager implements IndexSummaryManagerMBean
logger.trace("Index summaries for compacting SSTables are using {} MB of space",
(memoryPoolBytes - remainingBytes) / 1024.0 / 1024.0);
- List<SSTableReader> newSSTables = adjustSamplingLevels(sstablesByHotness, totalReadsPerSec, remainingBytes);
+ List<SSTableReader> newSSTables = adjustSamplingLevels(sstablesByHotness, transactions, totalReadsPerSec, remainingBytes);
+
+ for (LifecycleTransaction txn : transactions.values())
+ txn.finish();
total = 0;
for (SSTableReader sstable : Iterables.concat(compacting, oldFormatSSTables, newSSTables))
@@ -308,7 +314,7 @@ public class IndexSummaryManager implements IndexSummaryManagerMBean
return newSSTables;
}
- private static List<SSTableReader> adjustSamplingLevels(List<SSTableReader> sstables,
+ private static List<SSTableReader> adjustSamplingLevels(List<SSTableReader> sstables, Map<UUID, LifecycleTransaction> transactions,
double totalReadsPerSec, long memoryPoolCapacity) throws IOException
{
@@ -410,26 +416,16 @@ public class IndexSummaryManager implements IndexSummaryManagerMBean
toDownsample.addAll(forceResample);
toDownsample.addAll(toUpsample);
toDownsample.addAll(forceUpsample);
- Multimap<DataTracker, SSTableReader> replacedByTracker = HashMultimap.create();
- Multimap<DataTracker, SSTableReader> replacementsByTracker = HashMultimap.create();
for (ResampleEntry entry : toDownsample)
{
SSTableReader sstable = entry.sstable;
logger.debug("Re-sampling index summary for {} from {}/{} to {}/{} of the original number of entries",
sstable, sstable.getIndexSummarySamplingLevel(), Downsampling.BASE_SAMPLING_LEVEL,
entry.newSamplingLevel, Downsampling.BASE_SAMPLING_LEVEL);
- ColumnFamilyStore cfs = Keyspace.open(sstable.getKeyspaceName()).getColumnFamilyStore(sstable.getColumnFamilyName());
+ ColumnFamilyStore cfs = Keyspace.open(sstable.metadata.ksName).getColumnFamilyStore(sstable.metadata.cfId);
SSTableReader replacement = sstable.cloneWithNewSummarySamplingLevel(cfs, entry.newSamplingLevel);
- DataTracker tracker = cfs.getDataTracker();
-
- replacedByTracker.put(tracker, sstable);
- replacementsByTracker.put(tracker, replacement);
- }
-
- for (DataTracker tracker : replacedByTracker.keySet())
- {
- tracker.replaceWithNewInstances(replacedByTracker.get(tracker), replacementsByTracker.get(tracker));
- newSSTables.addAll(replacementsByTracker.get(tracker));
+ newSSTables.add(replacement);
+ transactions.get(sstable.metadata.cfId).update(replacement, true);
}
return newSSTables;
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/io/sstable/SSTableDeletingTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableDeletingTask.java b/src/java/org/apache/cassandra/io/sstable/SSTableDeletingTask.java
index db54557..cc837ba 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableDeletingTask.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableDeletingTask.java
@@ -23,15 +23,16 @@ import java.util.Set;
import java.util.concurrent.CopyOnWriteArraySet;
import java.util.concurrent.TimeUnit;
+import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Sets;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.cassandra.concurrent.ScheduledExecutors;
-import org.apache.cassandra.db.DataTracker;
-import org.apache.cassandra.db.SystemKeyspace;
+import org.apache.cassandra.db.lifecycle.Tracker;
import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.concurrent.Blocker;
public class SSTableDeletingTask implements Runnable
{
@@ -42,11 +43,12 @@ public class SSTableDeletingTask implements Runnable
// Additionally, we need to make sure to delete the data file first, so on restart the others
// will be recognized as GCable.
private static final Set<SSTableDeletingTask> failedTasks = new CopyOnWriteArraySet<>();
+ private static final Blocker blocker = new Blocker();
private final SSTableReader referent;
private final Descriptor desc;
private final Set<Component> components;
- private DataTracker tracker;
+ private Tracker tracker;
/**
* realDescriptor is the actual descriptor for the sstable, the descriptor inside
@@ -70,13 +72,18 @@ public class SSTableDeletingTask implements Runnable
}
}
- public void setTracker(DataTracker tracker)
+ public void setTracker(Tracker tracker)
{
// the tracker is used only to notify listeners of deletion of the sstable;
// since deletion of a non-final file is not really deletion of the sstable,
// we don't want to notify the listeners in this event
- if (desc.type == Descriptor.Type.FINAL)
- this.tracker = tracker;
+ assert desc.type == Descriptor.Type.FINAL;
+ this.tracker = tracker;
+ }
+
+ public Tracker getTracker()
+ {
+ return tracker;
}
public void schedule()
@@ -86,6 +93,7 @@ public class SSTableDeletingTask implements Runnable
public void run()
{
+ blocker.ask();
long size = referent.bytesOnDisk();
if (tracker != null)
@@ -119,6 +127,7 @@ public class SSTableDeletingTask implements Runnable
}
/** for tests */
+ @VisibleForTesting
public static void waitForDeletions()
{
Runnable runnable = new Runnable()
@@ -130,5 +139,11 @@ public class SSTableDeletingTask implements Runnable
FBUtilities.waitOnFuture(ScheduledExecutors.nonPeriodicTasks.schedule(runnable, 0, TimeUnit.MILLISECONDS));
}
+
+ @VisibleForTesting
+ public static void pauseDeletions(boolean stop)
+ {
+ blocker.block(stop);
+ }
}
[3/6] cassandra git commit: Extend Transactional API to sstable
lifecycle management
Posted by be...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
index a526ec9..8029075 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
@@ -20,33 +20,28 @@ package org.apache.cassandra.io.sstable;
import java.util.*;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.ImmutableList;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.DataTracker;
import org.apache.cassandra.db.DecoratedKey;
import org.apache.cassandra.db.RowIndexEntry;
import org.apache.cassandra.db.compaction.AbstractCompactedRow;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.utils.CLibrary;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.concurrent.Refs;
import org.apache.cassandra.utils.concurrent.Transactional;
-import static org.apache.cassandra.utils.Throwables.merge;
-
/**
* Wraps one or more writers as output for rewriting one or more readers: every sstable_preemptive_open_interval_in_mb
* we look in the summary we're collecting for the latest writer for the penultimate key that we know to have been fully
* flushed to the index file, and then double check that the key is fully present in the flushed data file.
- * Then we move the starts of each reader forwards to that point, replace them in the datatracker, and attach a runnable
+ * Then we move the starts of each reader forwards to that point, replace them in the Tracker, and attach a runnable
* for on-close (i.e. when all references expire) that drops the page cache prior to that key position
*
* hard-links are created for each partially written sstable so that readers opened against them continue to work past
* the rename of the temporary file, which is deleted once all readers against the hard-link have been closed.
- * If for any reason the writer is rolled over, we immediately rename and fully expose the completed file in the DataTracker.
+ * If for any reason the writer is rolled over, we immediately rename and fully expose the completed file in the Tracker.
*
* On abort we restore the original lower bounds to the existing readers and delete any temporary files we had in progress,
* but leave any hard-links in place for the readers we opened to cleanup when they're finished as we would had we finished
@@ -74,26 +69,19 @@ public class SSTableRewriter extends Transactional.AbstractTransactional impleme
return preemptiveOpenInterval;
}
- private final DataTracker dataTracker;
private final ColumnFamilyStore cfs;
private final long maxAge;
private long repairedAt = -1;
// the set of final readers we will expose on commit
+ private final LifecycleTransaction transaction; // the readers we are rewriting (updated as they are replaced)
private final List<SSTableReader> preparedForCommit = new ArrayList<>();
- private final Set<SSTableReader> rewriting; // the readers we are rewriting (updated as they are replaced)
- private final Map<Descriptor, DecoratedKey> originalStarts = new HashMap<>(); // the start key for each reader we are rewriting
private final Map<Descriptor, Integer> fileDescriptors = new HashMap<>(); // the file descriptors for each reader descriptor we are rewriting
- private SSTableReader currentlyOpenedEarly; // the reader for the most recent (re)opening of the target file
private long currentlyOpenedEarlyAt; // the position (in MB) in the target file we last (re)opened at
- private final List<Finished> finishedWriters = new ArrayList<>();
- // as writers are closed from finishedWriters, their last readers are moved into discard, so that abort can cleanup
- // after us safely; we use a set so we can add in both prepareToCommit and abort
- private final Set<SSTableReader> discard = new HashSet<>();
- // true for operations that are performed without Cassandra running (prevents updates of DataTracker)
- private final boolean isOffline;
+ private final List<SSTableWriter> writers = new ArrayList<>();
+ private final boolean isOffline; // true for operations that are performed without Cassandra running (prevents updates of Tracker)
private SSTableWriter writer;
private Map<DecoratedKey, RowIndexEntry> cachedKeys = new HashMap<>();
@@ -101,15 +89,11 @@ public class SSTableRewriter extends Transactional.AbstractTransactional impleme
// for testing (TODO: remove when have byteman setup)
private boolean throwEarly, throwLate;
- public SSTableRewriter(ColumnFamilyStore cfs, Set<SSTableReader> rewriting, long maxAge, boolean isOffline)
+ public SSTableRewriter(ColumnFamilyStore cfs, LifecycleTransaction transaction, long maxAge, boolean isOffline)
{
- this.rewriting = rewriting;
- for (SSTableReader sstable : rewriting)
- {
- originalStarts.put(sstable.descriptor, sstable.first);
+ this.transaction = transaction;
+ for (SSTableReader sstable : this.transaction.originals())
fileDescriptors.put(sstable.descriptor, CLibrary.getfd(sstable.getFilename()));
- }
- this.dataTracker = cfs.getDataTracker();
this.cfs = cfs;
this.maxAge = maxAge;
this.isOffline = isOffline;
@@ -134,7 +118,7 @@ public class SSTableRewriter extends Transactional.AbstractTransactional impleme
else
{
boolean save = false;
- for (SSTableReader reader : rewriting)
+ for (SSTableReader reader : transaction.originals())
{
if (reader.getCachedPosition(row.key, false) != null)
{
@@ -170,7 +154,7 @@ public class SSTableRewriter extends Transactional.AbstractTransactional impleme
{
if (isOffline)
{
- for (SSTableReader reader : rewriting)
+ for (SSTableReader reader : transaction.originals())
{
RowIndexEntry index = reader.getPosition(key, SSTableReader.Operator.GE);
CLibrary.trySkipCache(fileDescriptors.get(reader.descriptor), 0, index == null ? 0 : index.position);
@@ -181,10 +165,10 @@ public class SSTableRewriter extends Transactional.AbstractTransactional impleme
SSTableReader reader = writer.setMaxDataAge(maxAge).openEarly();
if (reader != null)
{
- replaceEarlyOpenedFile(currentlyOpenedEarly, reader);
- currentlyOpenedEarly = reader;
+ transaction.update(reader, false);
currentlyOpenedEarlyAt = writer.getFilePointer();
- moveStarts(reader, reader.last, false);
+ moveStarts(reader, reader.last);
+ transaction.checkpoint();
}
}
}
@@ -192,59 +176,19 @@ public class SSTableRewriter extends Transactional.AbstractTransactional impleme
protected Throwable doAbort(Throwable accumulate)
{
- try
- {
- moveStarts(null, null, true);
- }
- catch (Throwable t)
- {
- accumulate = merge(accumulate, t);
- }
-
- // cleanup any sstables we prepared for commit
- for (SSTableReader sstable : preparedForCommit)
- {
- try
- {
- sstable.markObsolete();
- sstable.selfRef().release();
- }
- catch (Throwable t)
- {
- accumulate = merge(accumulate , t);
- }
- }
-
- // abort the writers, and add the early opened readers to our discard pile
-
- if (writer != null)
- finishedWriters.add(new Finished(writer, currentlyOpenedEarly));
-
- for (Finished finished : finishedWriters)
- {
- accumulate = finished.writer.abort(accumulate);
-
- // if we've already been opened, add ourselves to the discard pile
- if (finished.reader != null)
- discard.add(finished.reader);
- }
-
- accumulate = replaceWithFinishedReaders(Collections.<SSTableReader>emptyList(), accumulate);
+ // abort the writers
+ for (SSTableWriter writer : writers)
+ accumulate = writer.abort(accumulate);
+ // abort the lifecycle transaction
+ accumulate = transaction.abort(accumulate);
return accumulate;
}
protected Throwable doCommit(Throwable accumulate)
{
- for (Finished f : finishedWriters)
- accumulate = f.writer.commit(accumulate);
- accumulate = replaceWithFinishedReaders(preparedForCommit, accumulate);
-
- return accumulate;
- }
-
- protected Throwable doCleanup(Throwable accumulate)
- {
- // we have no state of our own to cleanup; Transactional objects cleanup their own state in abort or commit
+ for (SSTableWriter writer : writers)
+ accumulate = writer.commit(accumulate);
+ accumulate = transaction.commit(accumulate);
return accumulate;
}
@@ -260,100 +204,70 @@ public class SSTableRewriter extends Transactional.AbstractTransactional impleme
*
* @param newReader the rewritten reader that replaces them for this region
* @param lowerbound if !reset, must be non-null, and marks the exclusive lowerbound of the start for each sstable
- * @param reset true iff we are restoring earlier starts (increasing the range over which they are valid)
*/
- private void moveStarts(SSTableReader newReader, DecoratedKey lowerbound, boolean reset)
+ private void moveStarts(SSTableReader newReader, DecoratedKey lowerbound)
{
if (isOffline)
return;
if (preemptiveOpenInterval == Long.MAX_VALUE)
return;
- List<SSTableReader> toReplace = new ArrayList<>();
- List<SSTableReader> replaceWith = new ArrayList<>();
final List<DecoratedKey> invalidateKeys = new ArrayList<>();
- if (!reset)
- {
- invalidateKeys.addAll(cachedKeys.keySet());
- for (Map.Entry<DecoratedKey, RowIndexEntry> cacheKey : cachedKeys.entrySet())
- newReader.cacheKey(cacheKey.getKey(), cacheKey.getValue());
- }
+ invalidateKeys.addAll(cachedKeys.keySet());
+ for (Map.Entry<DecoratedKey, RowIndexEntry> cacheKey : cachedKeys.entrySet())
+ newReader.cacheKey(cacheKey.getKey(), cacheKey.getValue());
cachedKeys = new HashMap<>();
- for (SSTableReader sstable : ImmutableList.copyOf(rewriting))
+ for (SSTableReader sstable : transaction.originals())
{
// we call getCurrentReplacement() to support multiple rewriters operating over the same source readers at once.
// note: only one such writer should be written to at any moment
- final SSTableReader latest = sstable.getCurrentReplacement();
- SSTableReader replacement;
- if (reset)
- {
- DecoratedKey newStart = originalStarts.get(sstable.descriptor);
- replacement = latest.cloneWithNewStart(newStart, null);
- }
- else
- {
- // skip any sstables that we know to already be shadowed
- if (latest.openReason == SSTableReader.OpenReason.SHADOWED)
- continue;
- if (latest.first.compareTo(lowerbound) > 0)
- continue;
+ final SSTableReader latest = transaction.current(sstable);
- final Runnable runOnClose = new Runnable()
- {
- public void run()
- {
- // this is somewhat racey, in that we could theoretically be closing this old reader
- // when an even older reader is still in use, but it's not likely to have any major impact
- for (DecoratedKey key : invalidateKeys)
- latest.invalidateCacheKey(key);
- }
- };
+ // skip any sstables that we know to already be shadowed
+ if (latest.first.compareTo(lowerbound) > 0)
+ continue;
- if (lowerbound.compareTo(latest.last) >= 0)
+ final Runnable runOnClose = new Runnable()
+ {
+ public void run()
{
- replacement = latest.cloneAsShadowed(runOnClose);
+ // this is somewhat racey, in that we could theoretically be closing this old reader
+ // when an even older reader is still in use, but it's not likely to have any major impact
+ for (DecoratedKey key : invalidateKeys)
+ latest.invalidateCacheKey(key);
}
- else
+ };
+
+ if (lowerbound.compareTo(latest.last) >= 0)
+ {
+ if (!transaction.isObsolete(latest))
{
- DecoratedKey newStart = latest.firstKeyBeyond(lowerbound);
- assert newStart != null;
- replacement = latest.cloneWithNewStart(newStart, runOnClose);
+ latest.runOnClose(runOnClose);
+ transaction.obsolete(latest);
}
+ continue;
}
- toReplace.add(latest);
- replaceWith.add(replacement);
- rewriting.remove(sstable);
- rewriting.add(replacement);
+ DecoratedKey newStart = latest.firstKeyBeyond(lowerbound);
+ assert newStart != null;
+ SSTableReader replacement = latest.cloneWithNewStart(newStart, runOnClose);
+ transaction.update(replacement, true);
}
- cfs.getDataTracker().replaceWithNewInstances(toReplace, replaceWith);
- }
-
- private void replaceEarlyOpenedFile(SSTableReader toReplace, SSTableReader replaceWith)
- {
- if (isOffline)
- return;
- Set<SSTableReader> toReplaceSet;
- if (toReplace != null)
- {
- toReplace.setReplacedBy(replaceWith);
- toReplaceSet = Collections.singleton(toReplace);
- }
- else
- {
- dataTracker.markCompacting(Collections.singleton(replaceWith), true, isOffline);
- toReplaceSet = Collections.emptySet();
- }
- dataTracker.replaceEarlyOpenedFiles(toReplaceSet, Collections.singleton(replaceWith));
}
public void switchWriter(SSTableWriter newWriter)
{
+ if (newWriter != null)
+ writers.add(newWriter.setMaxDataAge(maxAge));
+
if (writer == null || writer.getFilePointer() == 0)
{
if (writer != null)
+ {
writer.abort();
+ writers.remove(writer);
+ }
writer = newWriter;
return;
}
@@ -361,14 +275,13 @@ public class SSTableRewriter extends Transactional.AbstractTransactional impleme
SSTableReader reader = null;
if (preemptiveOpenInterval != Long.MAX_VALUE)
{
- // we leave it as a tmp file, but we open it and add it to the dataTracker
+ // we leave it as a tmp file, but we open it and add it to the Tracker
reader = writer.setMaxDataAge(maxAge).openFinalEarly();
- replaceEarlyOpenedFile(currentlyOpenedEarly, reader);
- moveStarts(reader, reader.last, false);
+ transaction.update(reader, false);
+ moveStarts(reader, reader.last);
+ transaction.checkpoint();
}
- finishedWriters.add(new Finished(writer, reader));
- currentlyOpenedEarly = null;
currentlyOpenedEarlyAt = 0;
writer = newWriter;
}
@@ -387,12 +300,12 @@ public class SSTableRewriter extends Transactional.AbstractTransactional impleme
/**
* Finishes the new file(s)
*
- * Creates final files, adds the new files to the dataTracker (via replaceReader).
+ * Creates final files, adds the new files to the Tracker (via replaceReader).
*
* We add them to the tracker to be able to get rid of the tmpfiles
*
* It is up to the caller to do the compacted sstables replacement
- * gymnastics (ie, call DataTracker#markCompactedSSTablesReplaced(..))
+ * gymnastics (ie, call Tracker#markCompactedSSTablesReplaced(..))
*
*
*/
@@ -402,6 +315,7 @@ public class SSTableRewriter extends Transactional.AbstractTransactional impleme
return finished();
}
+ // returns, in list form, the
public List<SSTableReader> finished()
{
assert state() == State.COMMITTED || state() == State.READY_TO_COMMIT;
@@ -416,82 +330,31 @@ public class SSTableRewriter extends Transactional.AbstractTransactional impleme
throw new RuntimeException("exception thrown early in finish, for testing");
// No early open to finalize and replace
- for (Finished f : finishedWriters)
+ for (SSTableWriter writer : writers)
{
- if (f.reader != null)
- discard.add(f.reader);
-
- f.writer.setRepairedAt(repairedAt).setMaxDataAge(maxAge).setOpenResult(true).prepareToCommit();
- SSTableReader newReader = f.writer.finished();
-
- if (f.reader != null)
- f.reader.setReplacedBy(newReader);
-
- preparedForCommit.add(newReader);
+ assert writer.getFilePointer() > 0;
+ writer.setRepairedAt(repairedAt).setOpenResult(true).prepareToCommit();
+ SSTableReader reader = writer.finished();
+ transaction.update(reader, false);
+ preparedForCommit.add(reader);
}
+ transaction.checkpoint();
if (throwLate)
throw new RuntimeException("exception thrown after all sstables finished, for testing");
- }
- @VisibleForTesting
- void throwDuringPrepare(boolean throwEarly)
- {
- this.throwEarly = throwEarly;
- this.throwLate = !throwEarly;
- }
+ // TODO: do we always want to avoid obsoleting if offline?
+ if (!isOffline)
+ transaction.obsoleteOriginals();
- // cleanup all our temporary readers and swap in our new ones
- private Throwable replaceWithFinishedReaders(List<SSTableReader> finished, Throwable accumulate)
- {
- if (isOffline)
- {
- for (SSTableReader reader : discard)
- {
- try
- {
- if (reader.getCurrentReplacement() == reader)
- reader.markObsolete();
- }
- catch (Throwable t)
- {
- accumulate = merge(accumulate, t);
- }
- }
- accumulate = Refs.release(Refs.selfRefs(discard), accumulate);
- }
- else
- {
- try
- {
- dataTracker.replaceEarlyOpenedFiles(discard, finished);
- }
- catch (Throwable t)
- {
- accumulate = merge(accumulate, t);
- }
- try
- {
- dataTracker.unmarkCompacting(discard);
- }
- catch (Throwable t)
- {
- accumulate = merge(accumulate, t);
- }
- }
- discard.clear();
- return accumulate;
+ transaction.prepareToCommit();
}
- private static final class Finished
+ public void throwDuringPrepare(boolean earlyException)
{
- final SSTableWriter writer;
- final SSTableReader reader;
-
- private Finished(SSTableWriter writer, SSTableReader reader)
- {
- this.writer = writer;
- this.reader = reader;
- }
+ if (earlyException)
+ throwEarly = true;
+ else
+ throwLate = true;
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index 23c27b0..8e701b3 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -46,6 +46,7 @@ import org.apache.cassandra.db.commitlog.ReplayPosition;
import org.apache.cassandra.db.composites.CellName;
import org.apache.cassandra.db.filter.ColumnSlice;
import org.apache.cassandra.db.index.SecondaryIndex;
+import org.apache.cassandra.db.lifecycle.Tracker;
import org.apache.cassandra.dht.*;
import org.apache.cassandra.io.compress.CompressionMetadata;
import org.apache.cassandra.io.sstable.*;
@@ -122,7 +123,7 @@ import static org.apache.cassandra.db.Directories.SECONDARY_INDEX_NAME_SEPARATOR
* managed as another resource each instance tracks its own Ref instance to, to ensure all of these resources are
* cleaned up safely and can be debugged otherwise.
*
- * TODO: fill in details about DataTracker and lifecycle interactions for tools, and for compaction strategies
+ * TODO: fill in details about Tracker and lifecycle interactions for tools, and for compaction strategies
*/
public abstract class SSTableReader extends SSTable implements SelfRefCounted<SSTableReader>
{
@@ -141,6 +142,9 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
}
};
+ // it's just an object, which we use regular Object equality on; we introduce a special class just for easy recognition
+ public static final class UniqueIdentifier {}
+
public static final Comparator<SSTableReader> sstableComparator = new Comparator<SSTableReader>()
{
public int compare(SSTableReader o1, SSTableReader o2)
@@ -170,11 +174,11 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
NORMAL,
EARLY,
METADATA_CHANGE,
- MOVED_START,
- SHADOWED // => MOVED_START past end
+ MOVED_START
}
public final OpenReason openReason;
+ public final UniqueIdentifier instanceId = new UniqueIdentifier();
// indexfile and datafile: might be null before a call to load()
protected SegmentedFile ifile;
@@ -594,9 +598,22 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
return ifile.path();
}
- public void setTrackedBy(DataTracker tracker)
+ // this is only used for restoring tracker state at delete (and wiring up the keycache) and so
+ // should only be called once it is actually added to the tracker
+ public void setupDeleteNotification(Tracker tracker)
{
tidy.type.deletingTask.setTracker(tracker);
+ setupKeyCache();
+ }
+
+ @VisibleForTesting
+ public boolean isDeleteNotificationSetup()
+ {
+ return tidy.type.deletingTask.getTracker() != null;
+ }
+
+ public void setupKeyCache()
+ {
// under normal operation we can do this at any time, but SSTR is also used outside C* proper,
// e.g. by BulkLoader, which does not initialize the cache. As a kludge, we set up the cache
// here when we know we're being wired into the rest of the server infrastructure.
@@ -908,15 +925,38 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
}
}
- public void setReplacedBy(SSTableReader replacement)
+ public void setReplaced()
{
synchronized (tidy.global)
{
- assert replacement != null;
assert !tidy.isReplaced;
- assert tidy.global.live == this;
tidy.isReplaced = true;
- tidy.global.live = replacement;
+ }
+ }
+
+ public boolean isReplaced()
+ {
+ synchronized (tidy.global)
+ {
+ return tidy.isReplaced;
+ }
+ }
+
+ public void runOnClose(final Runnable runOnClose)
+ {
+ synchronized (tidy.global)
+ {
+ final Runnable existing = tidy.runOnClose;
+ tidy.runOnClose = existing == null
+ ? runOnClose
+ : new Runnable()
+ {
+ public void run()
+ {
+ existing.run();
+ runOnClose.run();
+ }
+ };
}
}
@@ -948,32 +988,6 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
replacement.first = newStart;
replacement.last = this.last;
- setReplacedBy(replacement);
- return replacement;
- }
- }
-
- public SSTableReader cloneAsShadowed(final Runnable runOnClose)
- {
- synchronized (tidy.global)
- {
- assert openReason != OpenReason.EARLY;
- this.tidy.runOnClose = new Runnable()
- {
- public void run()
- {
- dfile.dropPageCache(0);
- ifile.dropPageCache(0);
- runOnClose.run();
- }
- };
-
- SSTableReader replacement = internalOpen(descriptor, components, metadata, partitioner, ifile.sharedCopy(),
- dfile.sharedCopy(), indexSummary.sharedCopy(), bf.sharedCopy(),
- maxDataAge, sstableMetadata, OpenReason.SHADOWED);
- replacement.first = first;
- replacement.last = last;
- setReplacedBy(replacement);
return replacement;
}
}
@@ -1036,7 +1050,6 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
sstableMetadata, OpenReason.METADATA_CHANGE);
replacement.first = this.first;
replacement.last = this.last;
- setReplacedBy(replacement);
return replacement;
}
}
@@ -1520,7 +1533,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
* except for threads holding a reference.
*
* @return true if the this is the first time the file was marked obsolete. Calling this
- * multiple times is usually buggy (see exceptions in DataTracker.unmarkCompacting and removeOldSSTablesSize).
+ * multiple times is usually buggy (see exceptions in Tracker.unmarkCompacting and removeOldSSTablesSize).
*/
public boolean markObsolete()
{
@@ -1638,11 +1651,6 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
return sstableMetadata.repairedAt != ActiveRepairService.UNREPAIRED_SSTABLE;
}
- public SSTableReader getCurrentReplacement()
- {
- return tidy.global.live;
- }
-
/**
* TODO: Move someplace reusable
*/
@@ -2048,8 +2056,6 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
static final ConcurrentMap<Descriptor, Ref<GlobalTidy>> lookup = new ConcurrentHashMap<>();
private final Descriptor desc;
- // a single convenience property for getting the most recent version of an sstable, not related to tidying
- private SSTableReader live;
// the readMeter that is shared between all instances of the sstable, and can be overridden in all of them
// at once also, for testing purposes
private RestorableMeter readMeter;
@@ -2064,7 +2070,6 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
{
this.desc = reader.descriptor;
this.isCompacted = new AtomicBoolean();
- this.live = reader;
}
void ensureReadMeter()
@@ -2128,6 +2133,13 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
}
}
+ @VisibleForTesting
+ public static void resetTidying()
+ {
+ GlobalTidy.lookup.clear();
+ DescriptorTypeTidy.lookup.clear();
+ }
+
public static abstract class Factory
{
public abstract SSTableReader open(final Descriptor descriptor,
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
index fa17c20..a7a7fcc 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
@@ -377,7 +377,8 @@ public class BigTableWriter extends SSTableWriter
return accumulate;
}
- protected Throwable doCleanup(Throwable accumulate)
+ @Override
+ protected Throwable doPreCleanup(Throwable accumulate)
{
accumulate = dbuilder.close(accumulate);
return accumulate;
@@ -562,7 +563,8 @@ public class BigTableWriter extends SSTableWriter
return indexFile.abort(accumulate);
}
- protected Throwable doCleanup(Throwable accumulate)
+ @Override
+ protected Throwable doPreCleanup(Throwable accumulate)
{
accumulate = summary.close(accumulate);
accumulate = bf.close(accumulate);
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/io/util/SequentialWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/SequentialWriter.java b/src/java/org/apache/cassandra/io/util/SequentialWriter.java
index d63be31..3c35a34 100644
--- a/src/java/org/apache/cassandra/io/util/SequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/util/SequentialWriter.java
@@ -77,7 +77,7 @@ public class SequentialWriter extends OutputStream implements WritableByteChanne
protected class TransactionalProxy extends AbstractTransactional
{
@Override
- protected Throwable doCleanup(Throwable accumulate)
+ protected Throwable doPreCleanup(Throwable accumulate)
{
if (directoryFD >= 0)
{
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java b/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java
index 842d06d..4ab4446 100644
--- a/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java
@@ -215,28 +215,28 @@ public class ColumnFamilyMetrics
{
public Long getValue()
{
- return cfs.getDataTracker().getView().getCurrentMemtable().getOperations();
+ return cfs.getTracker().getView().getCurrentMemtable().getOperations();
}
});
memtableOnHeapSize = createColumnFamilyGauge("MemtableOnHeapSize", new Gauge<Long>()
{
public Long getValue()
{
- return cfs.getDataTracker().getView().getCurrentMemtable().getAllocator().onHeap().owns();
+ return cfs.getTracker().getView().getCurrentMemtable().getAllocator().onHeap().owns();
}
});
memtableOffHeapSize = createColumnFamilyGauge("MemtableOffHeapSize", new Gauge<Long>()
{
public Long getValue()
{
- return cfs.getDataTracker().getView().getCurrentMemtable().getAllocator().offHeap().owns();
+ return cfs.getTracker().getView().getCurrentMemtable().getAllocator().offHeap().owns();
}
});
memtableLiveDataSize = createColumnFamilyGauge("MemtableLiveDataSize", new Gauge<Long>()
{
public Long getValue()
{
- return cfs.getDataTracker().getView().getCurrentMemtable().getLiveDataSize();
+ return cfs.getTracker().getView().getCurrentMemtable().getLiveDataSize();
}
});
allMemtablesOnHeapSize = createColumnFamilyGauge("AllMemtablesHeapSize", new Gauge<Long>()
@@ -245,7 +245,7 @@ public class ColumnFamilyMetrics
{
long size = 0;
for (ColumnFamilyStore cfs2 : cfs.concatWithIndexes())
- size += cfs2.getDataTracker().getView().getCurrentMemtable().getAllocator().onHeap().owns();
+ size += cfs2.getTracker().getView().getCurrentMemtable().getAllocator().onHeap().owns();
return size;
}
});
@@ -255,7 +255,7 @@ public class ColumnFamilyMetrics
{
long size = 0;
for (ColumnFamilyStore cfs2 : cfs.concatWithIndexes())
- size += cfs2.getDataTracker().getView().getCurrentMemtable().getAllocator().offHeap().owns();
+ size += cfs2.getTracker().getView().getCurrentMemtable().getAllocator().offHeap().owns();
return size;
}
});
@@ -265,7 +265,7 @@ public class ColumnFamilyMetrics
{
long size = 0;
for (ColumnFamilyStore cfs2 : cfs.concatWithIndexes())
- size += cfs2.getDataTracker().getView().getCurrentMemtable().getLiveDataSize();
+ size += cfs2.getTracker().getView().getCurrentMemtable().getLiveDataSize();
return size;
}
});
@@ -288,7 +288,7 @@ public class ColumnFamilyMetrics
public Long getValue()
{
long memtablePartitions = 0;
- for (Memtable memtable : cfs.getDataTracker().getView().getAllMemtables())
+ for (Memtable memtable : cfs.getTracker().getView().getAllMemtables())
memtablePartitions += memtable.partitionCount();
return SSTableReader.getApproximateKeyCount(cfs.getSSTables()) + memtablePartitions;
}
@@ -358,7 +358,7 @@ public class ColumnFamilyMetrics
{
public Integer getValue()
{
- return cfs.getDataTracker().getSSTables().size();
+ return cfs.getTracker().getSSTables().size();
}
});
liveDiskSpaceUsed = createColumnFamilyCounter("LiveDiskSpaceUsed");
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/streaming/StreamSession.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamSession.java b/src/java/org/apache/cassandra/streaming/StreamSession.java
index 6a70692..44522db 100644
--- a/src/java/org/apache/cassandra/streaming/StreamSession.java
+++ b/src/java/org/apache/cassandra/streaming/StreamSession.java
@@ -28,13 +28,14 @@ import javax.annotation.Nullable;
import com.google.common.base.Function;
import com.google.common.collect.*;
+
+import org.apache.cassandra.db.lifecycle.View;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.DataTracker;
import org.apache.cassandra.db.Keyspace;
import org.apache.cassandra.db.RowPosition;
import org.apache.cassandra.dht.AbstractBounds;
@@ -318,9 +319,9 @@ public class StreamSession implements IEndpointStateChangeSubscriber
final List<AbstractBounds<RowPosition>> rowBoundsList = new ArrayList<>(ranges.size());
for (Range<Token> range : ranges)
rowBoundsList.add(Range.makeRowRange(range));
- refs.addAll(cfStore.selectAndReference(new Function<DataTracker.View, List<SSTableReader>>()
+ refs.addAll(cfStore.selectAndReference(new Function<View, List<SSTableReader>>()
{
- public List<SSTableReader> apply(DataTracker.View view)
+ public List<SSTableReader> apply(View view)
{
List<SSTableReader> filteredSSTables = ColumnFamilyStore.CANONICAL_SSTABLES.apply(view);
Set<SSTableReader> sstables = Sets.newHashSet();
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
index 9f26637..d32ef88 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
@@ -32,12 +32,8 @@ import org.apache.cassandra.config.Schema;
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.Directories;
import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.compaction.AbstractCompactionStrategy;
-import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.db.compaction.LeveledCompactionStrategy;
-import org.apache.cassandra.db.compaction.LeveledManifest;
-import org.apache.cassandra.db.compaction.Scrubber;
-import org.apache.cassandra.db.compaction.WrappingCompactionStrategy;
+import org.apache.cassandra.db.compaction.*;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.sstable.*;
import org.apache.cassandra.utils.JVMStabilityInspector;
@@ -121,9 +117,9 @@ public class StandaloneScrubber
{
for (SSTableReader sstable : sstables)
{
- try
+ try (LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.SCRUB, sstable))
{
- Scrubber scrubber = new Scrubber(cfs, sstable, options.skipCorrupted, handler, true, !options.noValidate);
+ Scrubber scrubber = new Scrubber(cfs, txn, options.skipCorrupted, handler, true, !options.noValidate);
try
{
scrubber.scrub();
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/StandaloneSplitter.java b/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
index 2541d6e..e881133 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
@@ -27,10 +27,12 @@ import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.commons.cli.*;
import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.compaction.OperationType;
import org.apache.cassandra.db.Directories;
import org.apache.cassandra.db.Keyspace;
import org.apache.cassandra.db.compaction.CompactionManager;
import org.apache.cassandra.db.compaction.SSTableSplitter;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.io.sstable.*;
import org.apache.cassandra.utils.JVMStabilityInspector;
import org.apache.cassandra.utils.Pair;
@@ -145,12 +147,11 @@ public class StandaloneSplitter
if (options.snapshot)
System.out.println(String.format("Pre-split sstables snapshotted into snapshot %s", snapshotName));
- cfs.getDataTracker().markCompacting(sstables, false, true);
for (SSTableReader sstable : sstables)
{
- try
+ try (LifecycleTransaction transaction = LifecycleTransaction.offline(OperationType.UNKNOWN, sstable))
{
- new SSTableSplitter(cfs, sstable, options.sizeInMB).split();
+ new SSTableSplitter(cfs, transaction, options.sizeInMB).split();
// Remove the sstable (it's been copied by split and snapshotted)
sstable.markObsolete();
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java b/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
index 409a5f0..626d429 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
@@ -29,7 +29,9 @@ import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.Directories;
import org.apache.cassandra.db.Keyspace;
import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.compaction.OperationType;
import org.apache.cassandra.db.compaction.Upgrader;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.io.sstable.*;
import org.apache.cassandra.utils.JVMStabilityInspector;
import org.apache.cassandra.utils.OutputHandler;
@@ -98,9 +100,9 @@ public class StandaloneUpgrader
for (SSTableReader sstable : readers)
{
- try
+ try (LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.UPGRADE_SSTABLES, sstable))
{
- Upgrader upgrader = new Upgrader(cfs, sstable, handler);
+ Upgrader upgrader = new Upgrader(cfs, txn, handler);
upgrader.upgrade();
if (!options.keepSource)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/utils/concurrent/Blocker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/concurrent/Blocker.java b/src/java/org/apache/cassandra/utils/concurrent/Blocker.java
new file mode 100644
index 0000000..5192e98
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/concurrent/Blocker.java
@@ -0,0 +1,63 @@
+/*
+* 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.utils.concurrent;
+
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+public class Blocker
+{
+ private final ReentrantLock lock = new ReentrantLock();
+ private final Condition unblocked = lock.newCondition();
+ private volatile boolean block = false;
+
+ public void block(boolean block)
+ {
+ this.block = block;
+ if (!block)
+ {
+ lock.lock();
+ try
+ {
+ unblocked.signalAll();
+ }
+ finally
+ {
+ lock.unlock();
+ }
+ }
+ }
+
+ public void ask()
+ {
+ if (block)
+ {
+ lock.lock();
+ try
+ {
+ while (block)
+ unblocked.awaitUninterruptibly();
+ }
+ finally
+ {
+ lock.unlock();
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/utils/concurrent/Transactional.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/concurrent/Transactional.java b/src/java/org/apache/cassandra/utils/concurrent/Transactional.java
index bcf5095..5b0eb8e 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/Transactional.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/Transactional.java
@@ -70,6 +70,7 @@ public interface Transactional extends AutoCloseable
ABORTED;
}
+ private boolean permitRedundantTransitions;
private State state = State.IN_PROGRESS;
// the methods for actually performing the necessary behaviours, that are themselves protected against
@@ -79,9 +80,18 @@ public interface Transactional extends AutoCloseable
protected abstract Throwable doCommit(Throwable accumulate);
protected abstract Throwable doAbort(Throwable accumulate);
- // this only needs to perform cleanup of state unique to this instance; any internal
+ // these only needs to perform cleanup of state unique to this instance; any internal
// Transactional objects will perform cleanup in the commit() or abort() calls
- protected abstract Throwable doCleanup(Throwable accumulate);
+
+ /**
+ * perform an exception-safe pre-abort cleanup; this will still be run *after* commit
+ */
+ protected Throwable doPreCleanup(Throwable accumulate){ return accumulate; }
+
+ /**
+ * perform an exception-safe post-abort cleanup
+ */
+ protected Throwable doPostCleanup(Throwable accumulate){ return accumulate; }
/**
* Do any preparatory work prior to commit. This method should throw any exceptions that can be encountered
@@ -94,10 +104,13 @@ public interface Transactional extends AutoCloseable
*/
public final Throwable commit(Throwable accumulate)
{
+ if (permitRedundantTransitions && state == State.COMMITTED)
+ return accumulate;
if (state != State.READY_TO_COMMIT)
- throw new IllegalStateException("Commit attempted before prepared to commit");
+ throw new IllegalStateException("Cannot commit unless READY_TO_COMMIT; state is " + state);
accumulate = doCommit(accumulate);
- accumulate = doCleanup(accumulate);
+ accumulate = doPreCleanup(accumulate);
+ accumulate = doPostCleanup(accumulate);
state = State.COMMITTED;
return accumulate;
}
@@ -123,8 +136,9 @@ public interface Transactional extends AutoCloseable
}
state = State.ABORTED;
// we cleanup first so that, e.g., file handles can be released prior to deletion
- accumulate = doCleanup(accumulate);
+ accumulate = doPreCleanup(accumulate);
accumulate = doAbort(accumulate);
+ accumulate = doPostCleanup(accumulate);
return accumulate;
}
@@ -147,6 +161,8 @@ public interface Transactional extends AutoCloseable
*/
public final void prepareToCommit()
{
+ if (permitRedundantTransitions && state == State.READY_TO_COMMIT)
+ return;
if (state != State.IN_PROGRESS)
throw new IllegalStateException("Cannot prepare to commit unless IN_PROGRESS; state is " + state);
@@ -183,6 +199,11 @@ public interface Transactional extends AutoCloseable
{
return state;
}
+
+ protected void permitRedundantTransitions()
+ {
+ permitRedundantTransitions = true;
+ }
}
// commit should generally never throw an exception, and preferably never generate one,
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java b/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
index bf71639..e6c8f56 100644
--- a/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
+++ b/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
@@ -18,9 +18,7 @@
*/
package org.apache.cassandra.db.compaction;
-import java.io.IOException;
import java.util.*;
-import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
@@ -35,6 +33,7 @@ import org.apache.cassandra.Util;
import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.db.*;
import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.io.sstable.SSTableUtils;
import org.apache.cassandra.locator.SimpleStrategy;
import org.apache.cassandra.utils.ByteBufferUtil;
@@ -126,8 +125,11 @@ public class LongCompactionsTest
long start = System.nanoTime();
final int gcBefore = (int) (System.currentTimeMillis() / 1000) - Schema.instance.getCFMetaData(KEYSPACE1, "Standard1").getGcGraceSeconds();
- assert store.getDataTracker().markCompacting(sstables): "Cannot markCompacting all sstables";
- new CompactionTask(store, sstables, gcBefore, false).execute(null);
+ try (LifecycleTransaction txn = store.getTracker().tryModify(sstables, OperationType.COMPACTION))
+ {
+ assert txn != null : "Cannot markCompacting all sstables";
+ new CompactionTask(store, txn, gcBefore, false).execute(null);
+ }
System.out.println(String.format("%s: sstables=%d rowsper=%d colsper=%d: %d ms",
this.getClass().getName(),
sstableCount,
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/test/unit/org/apache/cassandra/MockSchema.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/MockSchema.java b/test/unit/org/apache/cassandra/MockSchema.java
new file mode 100644
index 0000000..bc236e1
--- /dev/null
+++ b/test/unit/org/apache/cassandra/MockSchema.java
@@ -0,0 +1,167 @@
+/*
+* 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;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+
+import org.apache.cassandra.cache.CachingOptions;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.composites.SimpleSparseCellNameType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.IndexSummary;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.util.BufferedSegmentedFile;
+import org.apache.cassandra.io.util.ChannelProxy;
+import org.apache.cassandra.io.util.Memory;
+import org.apache.cassandra.io.util.SegmentedFile;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.utils.AlwaysPresentFilter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class MockSchema
+{
+ static
+ {
+ Memory offsets = Memory.allocate(4);
+ offsets.setInt(0, 0);
+ indexSummary = new IndexSummary(Murmur3Partitioner.instance, offsets, 0, Memory.allocate(4), 0, 0, 0, 1);
+ }
+ private static final AtomicInteger id = new AtomicInteger();
+ public static final Keyspace ks = Keyspace.mockKS(new KSMetaData("mockks", SimpleStrategy.class, ImmutableMap.of("replication_factor", "1"), false));
+ public static final ColumnFamilyStore cfs = newCFS();
+
+ private static final IndexSummary indexSummary;
+ private static final SegmentedFile segmentedFile = new BufferedSegmentedFile(new ChannelProxy(temp("mocksegmentedfile")), 0);
+
+ public static Memtable memtable()
+ {
+ return new Memtable(cfs.metadata);
+ }
+
+ public static SSTableReader sstable(int generation)
+ {
+ return sstable(generation, false);
+ }
+
+ public static SSTableReader sstable(int generation, boolean keepRef)
+ {
+ return sstable(generation, 0, keepRef);
+ }
+
+ public static SSTableReader sstable(int generation, int size)
+ {
+ return sstable(generation, size, false);
+ }
+
+ public static SSTableReader sstable(int generation, int size, boolean keepRef)
+ {
+ return sstable(generation, size, keepRef, cfs);
+ }
+ public static SSTableReader sstable(int generation, int size, boolean keepRef, ColumnFamilyStore cfs)
+ {
+ Descriptor descriptor = new Descriptor(temp("mockcfdir").getParentFile(), "mockks", "mockcf", generation, Descriptor.Type.FINAL);
+ Set<Component> components = ImmutableSet.of(Component.DATA, Component.PRIMARY_INDEX, Component.FILTER, Component.TOC);
+ for (Component component : components)
+ {
+ File file = new File(descriptor.filenameFor(component));
+ try
+ {
+ file.createNewFile();
+ }
+ catch (IOException e)
+ {
+ }
+ file.deleteOnExit();
+ }
+ if (size > 0)
+ {
+ try
+ {
+ File file = new File(descriptor.filenameFor(Component.DATA));
+ try (RandomAccessFile raf = new RandomAccessFile(file, "rw"))
+ {
+ raf.setLength(size);
+ }
+ }
+ catch (IOException e)
+ {
+ throw new RuntimeException(e);
+ }
+ }
+ StatsMetadata metadata = (StatsMetadata) new MetadataCollector(cfs.metadata.comparator)
+ .finalizeMetadata(Murmur3Partitioner.instance.getClass().getCanonicalName(), 0.01f, -1)
+ .get(MetadataType.STATS);
+ SSTableReader reader = SSTableReader.internalOpen(descriptor, components, cfs.metadata, Murmur3Partitioner.instance,
+ segmentedFile.sharedCopy(), segmentedFile.sharedCopy(), indexSummary.sharedCopy(),
+ new AlwaysPresentFilter(), 1L, metadata, SSTableReader.OpenReason.NORMAL);
+ reader.first = reader.last = readerBounds(generation);
+ if (!keepRef)
+ reader.selfRef().release();
+ return reader;
+ }
+
+ public static ColumnFamilyStore newCFS()
+ {
+ String cfname = "mockcf" + (id.incrementAndGet());
+ CFMetaData metadata = newCFMetaData(cfname);
+ return new ColumnFamilyStore(ks, cfname, Murmur3Partitioner.instance, 0, metadata, new Directories(metadata), false, false);
+ }
+
+ private static CFMetaData newCFMetaData(String cfname)
+ {
+ CFMetaData metadata = new CFMetaData("mockks", cfname, ColumnFamilyType.Standard, new SimpleSparseCellNameType(UTF8Type.instance));
+ metadata.caching(CachingOptions.NONE);
+ return metadata;
+ }
+
+ public static BufferDecoratedKey readerBounds(int generation)
+ {
+ return new BufferDecoratedKey(new Murmur3Partitioner.LongToken(generation), ByteBufferUtil.EMPTY_BYTE_BUFFER);
+ }
+
+ private static File temp(String id)
+ {
+ try
+ {
+ File file = File.createTempFile(id, "tmp");
+ file.deleteOnExit();
+ return file;
+ }
+ catch (IOException e)
+ {
+ throw new RuntimeException(e);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/test/unit/org/apache/cassandra/Util.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/Util.java b/test/unit/org/apache/cassandra/Util.java
index 9b8e5df..c2205c4 100644
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@ -24,16 +24,17 @@ import java.io.*;
import java.net.InetAddress;
import java.net.UnknownHostException;
import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.SortedSet;
-import java.util.TreeSet;
-import java.util.UUID;
+import java.nio.channels.FileChannel;
+import java.util.*;
import java.util.concurrent.Callable;
import java.util.concurrent.Future;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+
+import org.apache.cassandra.cache.CachingOptions;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.KSMetaData;
import org.apache.cassandra.db.*;
import org.apache.cassandra.db.composites.*;
import org.apache.cassandra.db.compaction.AbstractCompactionTask;
@@ -44,17 +45,27 @@ import org.apache.cassandra.db.filter.QueryFilter;
import org.apache.cassandra.db.filter.SliceQueryFilter;
import org.apache.cassandra.db.filter.NamesQueryFilter;
import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.UTF8Type;
import org.apache.cassandra.dht.*;
import org.apache.cassandra.dht.RandomPartitioner.BigIntegerToken;
import org.apache.cassandra.gms.ApplicationState;
import org.apache.cassandra.gms.Gossiper;
import org.apache.cassandra.gms.VersionedValue;
+import org.apache.cassandra.io.sstable.Component;
import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.IndexSummary;
import org.apache.cassandra.io.sstable.format.SSTableReader;
-import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.sstable.format.big.BigTableReader;
+import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.util.*;
+import org.apache.cassandra.locator.SimpleStrategy;
import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.AlwaysPresentFilter;
import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.utils.CounterId;
+import org.apache.hadoop.fs.FileUtil;
import static org.junit.Assert.assertTrue;
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/test/unit/org/apache/cassandra/db/KeyCacheTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/KeyCacheTest.java b/test/unit/org/apache/cassandra/db/KeyCacheTest.java
index e5fd470..27e7a2b 100644
--- a/test/unit/org/apache/cassandra/db/KeyCacheTest.java
+++ b/test/unit/org/apache/cassandra/db/KeyCacheTest.java
@@ -43,7 +43,6 @@ import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.locator.SimpleStrategy;
import org.apache.cassandra.service.CacheService;
import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.concurrent.Refs;
import static org.junit.Assert.assertEquals;
@@ -171,7 +170,7 @@ public class KeyCacheTest
assertKeyCacheSize(2, KEYSPACE1, COLUMN_FAMILY1);
- Set<SSTableReader> readers = cfs.getDataTracker().getSSTables();
+ Set<SSTableReader> readers = cfs.getTracker().getSSTables();
Refs<SSTableReader> refs = Refs.tryRef(readers);
if (refs == null)
throw new IllegalStateException();
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ScrubTest.java b/test/unit/org/apache/cassandra/db/ScrubTest.java
index a5af823..dbbce9e 100644
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@ -20,57 +20,63 @@ package org.apache.cassandra.db;
*
*/
-import java.io.*;
import java.lang.reflect.Field;
import java.lang.reflect.Modifier;
import java.nio.ByteBuffer;
import java.util.*;
import java.util.concurrent.ExecutionException;
+import java.io.File;
+import java.io.IOError;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+
+import org.apache.cassandra.config.KSMetaData;
import org.apache.cassandra.cql3.QueryProcessor;
import org.apache.cassandra.db.composites.CellNameType;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.CounterColumnType;
+import org.apache.cassandra.db.marshal.UUIDType;
import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.db.marshal.LongType;
-import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.exceptions.RequestExecutionException;
import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.utils.UUIDGen;
import org.apache.commons.lang3.StringUtils;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.KSMetaData;
import org.apache.cassandra.config.Schema;
import org.apache.cassandra.cql3.Operator;
-import org.apache.cassandra.cql3.QueryProcessor;
import org.apache.cassandra.cql3.UntypedResultSet;
import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
import org.apache.cassandra.db.compaction.CompactionManager;
import org.apache.cassandra.db.compaction.Scrubber;
import org.apache.cassandra.db.index.SecondaryIndex;
import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.RequestExecutionException;
import org.apache.cassandra.exceptions.WriteTimeoutException;
import org.apache.cassandra.io.sstable.Component;
import org.apache.cassandra.io.sstable.Descriptor;
import org.apache.cassandra.io.sstable.SSTableRewriter;
-import org.apache.cassandra.io.sstable.format.SSTableFormat;
-import org.apache.cassandra.io.sstable.format.SSTableReader;
-import org.apache.cassandra.locator.SimpleStrategy;
import org.apache.cassandra.OrderedJUnit4ClassRunner;
import org.apache.cassandra.SchemaLoader;
import org.apache.cassandra.service.StorageService;
import org.apache.cassandra.Util;
import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.UUIDGen;
import static org.junit.Assert.*;
import static org.junit.Assume.assumeTrue;
import static org.apache.cassandra.Util.cellname;
import static org.apache.cassandra.Util.column;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
@RunWith(OrderedJUnit4ClassRunner.class)
public class ScrubTest
@@ -155,7 +161,8 @@ public class ScrubTest
overrideWithGarbage(sstable, ByteBufferUtil.bytes("0"), ByteBufferUtil.bytes("1"));
// with skipCorrupted == false, the scrub is expected to fail
- try(Scrubber scrubber = new Scrubber(cfs, sstable, false, false, true))
+ try (LifecycleTransaction txn = cfs.getTracker().tryModify(Arrays.asList(sstable), OperationType.SCRUB);
+ Scrubber scrubber = new Scrubber(cfs, txn, false, false, true);)
{
scrubber.scrub();
fail("Expected a CorruptSSTableException to be thrown");
@@ -164,7 +171,8 @@ public class ScrubTest
// with skipCorrupted == true, the corrupt rows will be skipped
Scrubber.ScrubResult scrubResult;
- try(Scrubber scrubber = new Scrubber(cfs, sstable, true, false, true))
+ try (LifecycleTransaction txn = cfs.getTracker().tryModify(Arrays.asList(sstable), OperationType.SCRUB);
+ Scrubber scrubber = new Scrubber(cfs, txn, true, false, true);)
{
scrubResult = scrubber.scrubWithResult();
}
@@ -213,20 +221,24 @@ public class ScrubTest
overrideWithGarbage(sstable, ByteBufferUtil.bytes("0"), ByteBufferUtil.bytes("1"));
// with skipCorrupted == false, the scrub is expected to fail
- Scrubber scrubber = new Scrubber(cfs, sstable, false, false, true);
- try
+ try (LifecycleTransaction txn = cfs.getTracker().tryModify(Arrays.asList(sstable), OperationType.SCRUB);
+ Scrubber scrubber = new Scrubber(cfs, txn, false, false, true))
{
+ // with skipCorrupted == true, the corrupt row will be skipped
scrubber.scrub();
fail("Expected a CorruptSSTableException to be thrown");
}
catch (IOError err) {}
- // with skipCorrupted == true, the corrupt row will be skipped
- scrubber = new Scrubber(cfs, sstable, true, false, true);
- scrubber.scrub();
- scrubber.close();
- assertEquals(1, cfs.getSSTables().size());
+ try (LifecycleTransaction txn = cfs.getTracker().tryModify(Arrays.asList(sstable), OperationType.SCRUB);
+ Scrubber scrubber = new Scrubber(cfs, txn, true, false, true))
+ {
+ // with skipCorrupted == true, the corrupt row will be skipped
+ scrubber.scrub();
+ scrubber.close();
+ }
+ assertEquals(1, cfs.getSSTables().size());
// verify that we can read all of the rows, and there is now one less row
rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
assertEquals(1, rows.size());
@@ -369,9 +381,13 @@ public class ScrubTest
components.add(Component.STATS);
components.add(Component.SUMMARY);
components.add(Component.TOC);
+
SSTableReader sstable = SSTableReader.openNoValidation(desc, components, cfs);
+ if (sstable.last.compareTo(sstable.first) < 0)
+ sstable.last = sstable.first;
- try(Scrubber scrubber = new Scrubber(cfs, sstable, false, true, true))
+ try (LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.SCRUB, sstable);
+ Scrubber scrubber = new Scrubber(cfs, txn, false, true, true);)
{
scrubber.scrub();
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java b/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
index 1dc72ae..235462b 100644
--- a/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
@@ -17,7 +17,7 @@
*/
package org.apache.cassandra.db.compaction;
-import junit.framework.Assert;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.utils.concurrent.Refs;
import static org.hamcrest.CoreMatchers.is;
import static org.junit.Assert.assertEquals;
@@ -30,7 +30,6 @@ import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
-import java.util.concurrent.ExecutionException;
import org.apache.cassandra.config.KSMetaData;
import org.apache.cassandra.exceptions.ConfigurationException;
@@ -91,13 +90,18 @@ public class AntiCompactionTest
Range<Token> range = new Range<Token>(new BytesToken("0".getBytes()), new BytesToken("4".getBytes()));
List<Range<Token>> ranges = Arrays.asList(range);
- Refs<SSTableReader> refs = Refs.ref(sstables);
- long repairedAt = 1000;
- CompactionManager.instance.performAnticompaction(store, ranges, refs, repairedAt);
-
- assertEquals(2, store.getSSTables().size());
int repairedKeys = 0;
int nonRepairedKeys = 0;
+ try (LifecycleTransaction txn = store.getTracker().tryModify(sstables, OperationType.ANTICOMPACTION);
+ Refs<SSTableReader> refs = Refs.ref(sstables))
+ {
+ if (txn == null)
+ throw new IllegalStateException();
+ long repairedAt = 1000;
+ CompactionManager.instance.performAnticompaction(store, ranges, refs, txn, repairedAt);
+ }
+
+ assertEquals(2, store.getSSTables().size());
for (SSTableReader sstable : store.getSSTables())
{
try (ISSTableScanner scanner = sstable.getScanner())
@@ -123,7 +127,7 @@ public class AntiCompactionTest
assertFalse(sstable.isMarkedCompacted());
assertEquals(1, sstable.selfRef().globalCount());
}
- assertEquals(0, store.getDataTracker().getCompacting().size());
+ assertEquals(0, store.getTracker().getCompacting().size());
assertEquals(repairedKeys, 4);
assertEquals(nonRepairedKeys, 6);
}
@@ -139,13 +143,16 @@ public class AntiCompactionTest
long origSize = s.bytesOnDisk();
Range<Token> range = new Range<Token>(new BytesToken(ByteBufferUtil.bytes(0)), new BytesToken(ByteBufferUtil.bytes(500)));
Collection<SSTableReader> sstables = cfs.getSSTables();
- CompactionManager.instance.performAnticompaction(cfs, Arrays.asList(range), Refs.tryRef(sstables), 12345);
+ try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.ANTICOMPACTION);
+ Refs<SSTableReader> refs = Refs.ref(sstables))
+ {
+ CompactionManager.instance.performAnticompaction(cfs, Arrays.asList(range), refs, txn, 12345);
+ }
long sum = 0;
for (SSTableReader x : cfs.getSSTables())
sum += x.bytesOnDisk();
assertEquals(sum, cfs.metric.liveDiskSpaceUsed.getCount());
assertEquals(origSize, cfs.metric.liveDiskSpaceUsed.getCount(), 100000);
-
}
private SSTableReader writeFile(ColumnFamilyStore cfs, int count)
@@ -210,10 +217,12 @@ public class AntiCompactionTest
Range<Token> range = new Range<Token>(new BytesToken("0".getBytes()), new BytesToken("4".getBytes()));
List<Range<Token>> ranges = Arrays.asList(range);
- Refs<SSTableReader> refs = Refs.tryRef(sstables);
- Assert.assertNotNull(refs);
long repairedAt = 1000;
- CompactionManager.instance.performAnticompaction(store, ranges, refs, repairedAt);
+ try (LifecycleTransaction txn = store.getTracker().tryModify(sstables, OperationType.ANTICOMPACTION);
+ Refs<SSTableReader> refs = Refs.ref(sstables))
+ {
+ CompactionManager.instance.performAnticompaction(store, ranges, refs, txn, repairedAt);
+ }
/*
Anticompaction will be anti-compacting 10 SSTables but will be doing this two at a time
so there will be no net change in the number of sstables
@@ -256,12 +265,16 @@ public class AntiCompactionTest
Range<Token> range = new Range<Token>(new BytesToken("0".getBytes()), new BytesToken("9999".getBytes()));
List<Range<Token>> ranges = Arrays.asList(range);
- CompactionManager.instance.performAnticompaction(store, ranges, Refs.tryRef(sstables), 1);
+ try (LifecycleTransaction txn = store.getTracker().tryModify(sstables, OperationType.ANTICOMPACTION);
+ Refs<SSTableReader> refs = Refs.ref(sstables))
+ {
+ CompactionManager.instance.performAnticompaction(store, ranges, refs, txn, 1);
+ }
assertThat(store.getSSTables().size(), is(1));
assertThat(Iterables.get(store.getSSTables(), 0).isRepaired(), is(true));
assertThat(Iterables.get(store.getSSTables(), 0).selfRef().globalCount(), is(1));
- assertThat(store.getDataTracker().getCompacting().size(), is(0));
+ assertThat(store.getTracker().getCompacting().size(), is(0));
}
@@ -282,8 +295,12 @@ public class AntiCompactionTest
Range<Token> range = new Range<Token>(new BytesToken("-10".getBytes()), new BytesToken("-1".getBytes()));
List<Range<Token>> ranges = Arrays.asList(range);
- Refs<SSTableReader> refs = Refs.ref(sstables);
- CompactionManager.instance.performAnticompaction(store, ranges, refs, 0);
+
+ try (LifecycleTransaction txn = store.getTracker().tryModify(sstables, OperationType.ANTICOMPACTION);
+ Refs<SSTableReader> refs = Refs.ref(sstables))
+ {
+ CompactionManager.instance.performAnticompaction(store, ranges, refs, txn, 0);
+ }
assertThat(store.getSSTables().size(), is(10));
assertThat(Iterables.get(store.getSSTables(), 0).isRepaired(), is(false));
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/test/unit/org/apache/cassandra/db/compaction/CompactionAwareWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionAwareWriterTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionAwareWriterTest.java
index 88074af..235fd49 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionAwareWriterTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionAwareWriterTest.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.db.compaction;
import java.nio.ByteBuffer;
import java.util.*;
-import java.util.concurrent.ExecutionException;
import com.google.common.primitives.Longs;
import org.junit.Before;
@@ -41,11 +40,12 @@ import org.apache.cassandra.db.compaction.writers.MajorLeveledCompactionWriter;
import org.apache.cassandra.db.compaction.writers.MaxSSTableSizeWriter;
import org.apache.cassandra.db.compaction.writers.SplittingSizeTieredCompactionWriter;
import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.io.sstable.ISSTableScanner;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.locator.SimpleStrategy;
-import org.apache.cassandra.utils.ByteBufferUtil;
+
import static org.junit.Assert.assertEquals;
public class CompactionAwareWriterTest
@@ -81,10 +81,10 @@ public class CompactionAwareWriterTest
int rowCount = 1000;
cfs.disableAutoCompaction();
populate(cfs, rowCount);
- Set<SSTableReader> sstables = new HashSet<>(cfs.getSSTables());
- long beforeSize = sstables.iterator().next().onDiskLength();
- CompactionAwareWriter writer = new DefaultCompactionWriter(cfs, sstables, sstables, false, OperationType.COMPACTION);
- int rows = compact(cfs, sstables, writer);
+ LifecycleTransaction txn = cfs.getTracker().tryModify(cfs.getSSTables(), OperationType.COMPACTION);
+ long beforeSize = txn.originals().iterator().next().onDiskLength();
+ CompactionAwareWriter writer = new DefaultCompactionWriter(cfs, txn, txn.originals(), false, OperationType.COMPACTION);
+ int rows = compact(cfs, txn, writer);
assertEquals(1, cfs.getSSTables().size());
assertEquals(rowCount, rows);
assertEquals(beforeSize, cfs.getSSTables().iterator().next().onDiskLength());
@@ -100,11 +100,11 @@ public class CompactionAwareWriterTest
cfs.disableAutoCompaction();
int rowCount = 1000;
populate(cfs, rowCount);
- Set<SSTableReader> sstables = new HashSet<>(cfs.getSSTables());
- long beforeSize = sstables.iterator().next().onDiskLength();
+ LifecycleTransaction txn = cfs.getTracker().tryModify(cfs.getSSTables(), OperationType.COMPACTION);
+ long beforeSize = txn.originals().iterator().next().onDiskLength();
int sstableSize = (int)beforeSize/10;
- CompactionAwareWriter writer = new MaxSSTableSizeWriter(cfs, sstables, sstables, sstableSize, 0, false, OperationType.COMPACTION);
- int rows = compact(cfs, sstables, writer);
+ CompactionAwareWriter writer = new MaxSSTableSizeWriter(cfs, txn, txn.originals(), sstableSize, 0, false, OperationType.COMPACTION);
+ int rows = compact(cfs, txn, writer);
assertEquals(10, cfs.getSSTables().size());
assertEquals(rowCount, rows);
validateData(cfs, rowCount);
@@ -118,10 +118,10 @@ public class CompactionAwareWriterTest
cfs.disableAutoCompaction();
int rowCount = 10000;
populate(cfs, rowCount);
- Set<SSTableReader> sstables = new HashSet<>(cfs.getSSTables());
- long beforeSize = sstables.iterator().next().onDiskLength();
- CompactionAwareWriter writer = new SplittingSizeTieredCompactionWriter(cfs, sstables, sstables, OperationType.COMPACTION, 0);
- int rows = compact(cfs, sstables, writer);
+ LifecycleTransaction txn = cfs.getTracker().tryModify(cfs.getSSTables(), OperationType.COMPACTION);
+ long beforeSize = txn.originals().iterator().next().onDiskLength();
+ CompactionAwareWriter writer = new SplittingSizeTieredCompactionWriter(cfs, txn, txn.originals(), OperationType.COMPACTION, 0);
+ int rows = compact(cfs, txn, writer);
long expectedSize = beforeSize / 2;
List<SSTableReader> sortedSSTables = new ArrayList<>(cfs.getSSTables());
@@ -154,11 +154,11 @@ public class CompactionAwareWriterTest
int rowCount = 20000;
int targetSSTableCount = 50;
populate(cfs, rowCount);
- Set<SSTableReader> sstables = new HashSet<>(cfs.getSSTables());
- long beforeSize = sstables.iterator().next().onDiskLength();
+ LifecycleTransaction txn = cfs.getTracker().tryModify(cfs.getSSTables(), OperationType.COMPACTION);
+ long beforeSize = txn.originals().iterator().next().onDiskLength();
int sstableSize = (int)beforeSize/targetSSTableCount;
- CompactionAwareWriter writer = new MajorLeveledCompactionWriter(cfs, sstables, sstables, sstableSize, false, OperationType.COMPACTION);
- int rows = compact(cfs, sstables, writer);
+ CompactionAwareWriter writer = new MajorLeveledCompactionWriter(cfs, txn, txn.originals(), sstableSize, false, OperationType.COMPACTION);
+ int rows = compact(cfs, txn, writer);
assertEquals(targetSSTableCount, cfs.getSSTables().size());
int [] levelCounts = new int[5];
assertEquals(rowCount, rows);
@@ -175,13 +175,13 @@ public class CompactionAwareWriterTest
cfs.truncateBlocking();
}
- private int compact(ColumnFamilyStore cfs, Set<SSTableReader> sstables, CompactionAwareWriter writer)
+ private int compact(ColumnFamilyStore cfs, LifecycleTransaction txn, CompactionAwareWriter writer)
{
- assert sstables.size() == 1;
+ assert txn.originals().size() == 1;
int rowsWritten = 0;
- try (AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables))
+ try (AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(txn.originals()))
{
- CompactionController controller = new CompactionController(cfs, sstables, cfs.gcBefore(System.currentTimeMillis()));
+ CompactionController controller = new CompactionController(cfs, txn.originals(), cfs.gcBefore(System.currentTimeMillis()));
ISSTableScanner scanner = scanners.scanners.get(0);
while(scanner.hasNext())
{
@@ -191,7 +191,6 @@ public class CompactionAwareWriterTest
}
}
Collection<SSTableReader> newSSTables = writer.finish();
- cfs.getDataTracker().markCompactedSSTablesReplaced(sstables, newSSTables, OperationType.COMPACTION);
return rowsWritten;
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
index f1d016b..64e4465 100644
--- a/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
@@ -324,10 +324,10 @@ public class DateTieredCompactionStrategyTest extends SchemaLoader
Thread.sleep(2000);
AbstractCompactionTask t = dtcs.getNextBackgroundTask((int) (System.currentTimeMillis()/1000));
assertNotNull(t);
- assertEquals(1, Iterables.size(t.sstables));
- SSTableReader sstable = t.sstables.iterator().next();
+ assertEquals(1, Iterables.size(t.transaction.originals()));
+ SSTableReader sstable = t.transaction.originals().iterator().next();
assertEquals(sstable, expiredSSTable);
- cfs.getDataTracker().unmarkCompacting(cfs.getSSTables());
+ t.transaction.abort();
}
}
[5/6] cassandra git commit: Extend Transactional API to sstable
lifecycle management
Posted by be...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index d79b835..004e893 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -54,14 +54,10 @@ import org.apache.cassandra.concurrent.NamedThreadFactory;
import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.db.Cell;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.OnDiskAtom;
-import org.apache.cassandra.db.SystemKeyspace;
+import org.apache.cassandra.db.*;
import org.apache.cassandra.db.compaction.CompactionInfo.Holder;
import org.apache.cassandra.db.index.SecondaryIndexBuilder;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.dht.Bounds;
import org.apache.cassandra.dht.Range;
import org.apache.cassandra.dht.Token;
@@ -82,12 +78,14 @@ import org.apache.cassandra.utils.UUIDGen;
import org.apache.cassandra.utils.concurrent.OpOrder;
import org.apache.cassandra.utils.concurrent.Refs;
+import static java.util.Collections.singleton;
+
/**
* <p>
* A singleton which manages a private executor of ongoing compactions.
* </p>
* Scheduling for compaction is accomplished by swapping sstables to be compacted into
- * a set via DataTracker. New scheduling attempts will ignore currently compacting
+ * a set via Tracker. New scheduling attempts will ignore currently compacting
* sstables.
*/
public class CompactionManager implements CompactionManagerMBean
@@ -195,7 +193,7 @@ public class CompactionManager implements CompactionManagerMBean
public boolean isCompacting(Iterable<ColumnFamilyStore> cfses)
{
for (ColumnFamilyStore cfs : cfses)
- if (!cfs.getDataTracker().getCompacting().isEmpty())
+ if (!cfs.getTracker().getCompacting().isEmpty())
return true;
return false;
}
@@ -245,22 +243,22 @@ public class CompactionManager implements CompactionManagerMBean
}
}
- private AllSSTableOpStatus parallelAllSSTableOperation(final ColumnFamilyStore cfs, final OneSSTableOperation operation) throws ExecutionException, InterruptedException
+ private AllSSTableOpStatus parallelAllSSTableOperation(final ColumnFamilyStore cfs, final OneSSTableOperation operation, OperationType operationType) throws ExecutionException, InterruptedException
{
- Iterable<SSTableReader> compactingSSTables = cfs.markAllCompacting();
- if (compactingSSTables == null)
- {
- logger.info("Aborting operation on {}.{} after failing to interrupt other compaction operations", cfs.keyspace.getName(), cfs.name);
- return AllSSTableOpStatus.ABORTED;
- }
- if (Iterables.isEmpty(compactingSSTables))
+ try (LifecycleTransaction compacting = cfs.markAllCompacting(operationType);)
{
- logger.info("No sstables for {}.{}", cfs.keyspace.getName(), cfs.name);
- return AllSSTableOpStatus.SUCCESSFUL;
- }
- try
- {
- Iterable<SSTableReader> sstables = operation.filterSSTables(compactingSSTables);
+ if (compacting == null)
+ {
+ logger.info("Aborting operation on {}.{} after failing to interrupt other compaction operations", cfs.keyspace.getName(), cfs.name);
+ return AllSSTableOpStatus.ABORTED;
+ }
+ if (compacting.originals().isEmpty())
+ {
+ logger.info("No sstables for {}.{}", cfs.keyspace.getName(), cfs.name);
+ return AllSSTableOpStatus.SUCCESSFUL;
+ }
+
+ Iterable<SSTableReader> sstables = operation.filterSSTables(compacting.originals());
List<Future<Object>> futures = new ArrayList<>();
for (final SSTableReader sstable : sstables)
@@ -271,31 +269,30 @@ public class CompactionManager implements CompactionManagerMBean
return AllSSTableOpStatus.ABORTED;
}
+ final LifecycleTransaction txn = compacting.split(singleton(sstable));
futures.add(executor.submit(new Callable<Object>()
{
@Override
public Object call() throws Exception
{
- operation.execute(sstable);
+ operation.execute(txn);
return this;
}
}));
}
+ assert compacting.originals().isEmpty();
+
for (Future<Object> f : futures)
f.get();
+ return AllSSTableOpStatus.SUCCESSFUL;
}
- finally
- {
- cfs.getDataTracker().unmarkCompacting(compactingSSTables);
- }
- return AllSSTableOpStatus.SUCCESSFUL;
}
private static interface OneSSTableOperation
{
Iterable<SSTableReader> filterSSTables(Iterable<SSTableReader> input);
- void execute(SSTableReader input) throws IOException;
+ void execute(LifecycleTransaction input) throws IOException;
}
public enum AllSSTableOpStatus { ABORTED(1), SUCCESSFUL(0);
@@ -318,11 +315,11 @@ public class CompactionManager implements CompactionManagerMBean
}
@Override
- public void execute(SSTableReader input) throws IOException
+ public void execute(LifecycleTransaction input) throws IOException
{
scrubOne(cfs, input, skipCorrupted, checkData);
}
- });
+ }, OperationType.SCRUB);
}
public AllSSTableOpStatus performVerify(final ColumnFamilyStore cfs, final boolean extendedVerify) throws InterruptedException, ExecutionException
@@ -337,11 +334,11 @@ public class CompactionManager implements CompactionManagerMBean
}
@Override
- public void execute(SSTableReader input) throws IOException
+ public void execute(LifecycleTransaction input) throws IOException
{
- verifyOne(cfs, input, extendedVerify);
+ verifyOne(cfs, input.onlyOne(), extendedVerify);
}
- });
+ }, OperationType.VERIFY);
}
public AllSSTableOpStatus performSSTableRewrite(final ColumnFamilyStore cfs, final boolean excludeCurrentVersion) throws InterruptedException, ExecutionException
@@ -362,14 +359,14 @@ public class CompactionManager implements CompactionManagerMBean
}
@Override
- public void execute(SSTableReader input) throws IOException
+ public void execute(LifecycleTransaction txn) throws IOException
{
- AbstractCompactionTask task = cfs.getCompactionStrategy().getCompactionTask(Collections.singleton(input), NO_GC, Long.MAX_VALUE);
+ AbstractCompactionTask task = cfs.getCompactionStrategy().getCompactionTask(txn, NO_GC, Long.MAX_VALUE);
task.setUserDefined(true);
task.setCompactionType(OperationType.UPGRADE_SSTABLES);
task.execute(metrics);
}
- });
+ }, OperationType.UPGRADE_SSTABLES);
}
public AllSSTableOpStatus performCleanup(final ColumnFamilyStore cfStore) throws InterruptedException, ExecutionException
@@ -395,12 +392,12 @@ public class CompactionManager implements CompactionManagerMBean
}
@Override
- public void execute(SSTableReader input) throws IOException
+ public void execute(LifecycleTransaction txn) throws IOException
{
CleanupStrategy cleanupStrategy = CleanupStrategy.get(cfStore, ranges);
- doCleanupOne(cfStore, input, cleanupStrategy, ranges, hasIndexes);
+ doCleanupOne(cfStore, txn, cleanupStrategy, ranges, hasIndexes);
}
- });
+ }, OperationType.CLEANUP);
}
public ListenableFuture<?> submitAntiCompaction(final ColumnFamilyStore cfs,
@@ -412,19 +409,19 @@ public class CompactionManager implements CompactionManagerMBean
@Override
public void runMayThrow() throws Exception
{
- boolean success = false;
- while (!success)
+ LifecycleTransaction modifier = null;
+ while (modifier == null)
{
- for (SSTableReader compactingSSTable : cfs.getDataTracker().getCompacting())
+ for (SSTableReader compactingSSTable : cfs.getTracker().getCompacting())
sstables.releaseIfHolds(compactingSSTable);
Set<SSTableReader> compactedSSTables = new HashSet<>();
for (SSTableReader sstable : sstables)
if (sstable.isMarkedCompacted())
compactedSSTables.add(sstable);
sstables.release(compactedSSTables);
- success = sstables.isEmpty() || cfs.getDataTracker().markCompacting(sstables);
+ modifier = cfs.getTracker().tryModify(sstables, OperationType.ANTICOMPACTION);
}
- performAnticompaction(cfs, ranges, sstables, repairedAt);
+ performAnticompaction(cfs, ranges, sstables, modifier, repairedAt);
}
};
if (executor.isShutdown())
@@ -452,6 +449,7 @@ public class CompactionManager implements CompactionManagerMBean
public void performAnticompaction(ColumnFamilyStore cfs,
Collection<Range<Token>> ranges,
Refs<SSTableReader> validatedForRepair,
+ LifecycleTransaction txn,
long repairedAt) throws InterruptedException, IOException
{
logger.info("Starting anticompaction for {}.{} on {}/{} sstables", cfs.keyspace.getName(), cfs.getColumnFamilyName(), validatedForRepair.size(), cfs.getSSTables().size());
@@ -490,16 +488,18 @@ public class CompactionManager implements CompactionManagerMBean
}
}
}
- cfs.getDataTracker().notifySSTableRepairedStatusChanged(mutatedRepairStatuses);
- cfs.getDataTracker().unmarkCompacting(Sets.union(nonAnticompacting, mutatedRepairStatuses));
+ cfs.getTracker().notifySSTableRepairedStatusChanged(mutatedRepairStatuses);
+ txn.cancel(Sets.union(nonAnticompacting, mutatedRepairStatuses));
validatedForRepair.release(Sets.union(nonAnticompacting, mutatedRepairStatuses));
+ assert txn.originals().equals(sstables);
if (!sstables.isEmpty())
- doAntiCompaction(cfs, ranges, sstables, repairedAt);
+ doAntiCompaction(cfs, ranges, txn, repairedAt);
+ txn.finish();
}
finally
{
validatedForRepair.release();
- cfs.getDataTracker().unmarkCompacting(sstables);
+ txn.close();
}
logger.info("Completed anticompaction successfully");
@@ -657,9 +657,9 @@ public class CompactionManager implements CompactionManagerMBean
}
}
- private void scrubOne(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean checkData) throws IOException
+ private void scrubOne(ColumnFamilyStore cfs, LifecycleTransaction modifier, boolean skipCorrupted, boolean checkData) throws IOException
{
- Scrubber scrubber = new Scrubber(cfs, sstable, skipCorrupted, false, checkData);
+ Scrubber scrubber = new Scrubber(cfs, modifier, skipCorrupted, false, checkData);
CompactionInfo.Holder scrubInfo = scrubber.getScrubInfo();
metrics.beginCompaction(scrubInfo);
@@ -750,15 +750,16 @@ public class CompactionManager implements CompactionManagerMBean
*
* @throws IOException
*/
- private void doCleanupOne(final ColumnFamilyStore cfs, SSTableReader sstable, CleanupStrategy cleanupStrategy, Collection<Range<Token>> ranges, boolean hasIndexes) throws IOException
+ private void doCleanupOne(final ColumnFamilyStore cfs, LifecycleTransaction txn, CleanupStrategy cleanupStrategy, Collection<Range<Token>> ranges, boolean hasIndexes) throws IOException
{
assert !cfs.isIndex();
- Set<SSTableReader> sstableSet = Collections.singleton(sstable);
+ SSTableReader sstable = txn.onlyOne();
if (!hasIndexes && !new Bounds<>(sstable.first.getToken(), sstable.last.getToken()).intersects(ranges))
{
- cfs.getDataTracker().markCompactedSSTablesReplaced(sstableSet, Collections.<SSTableReader>emptyList(), OperationType.CLEANUP);
+ txn.obsoleteOriginals();
+ txn.finish();
return;
}
if (!needsCleanup(sstable, ranges))
@@ -772,13 +773,13 @@ public class CompactionManager implements CompactionManagerMBean
long totalkeysWritten = 0;
int expectedBloomFilterSize = Math.max(cfs.metadata.getMinIndexInterval(),
- (int) (SSTableReader.getApproximateKeyCount(sstableSet)));
+ (int) (SSTableReader.getApproximateKeyCount(txn.originals())));
if (logger.isDebugEnabled())
logger.debug("Expected bloom filter size : {}", expectedBloomFilterSize);
logger.info("Cleaning up {}", sstable);
- File compactionFileLocation = cfs.directories.getWriteableLocationAsFile(cfs.getExpectedCompactedFileSize(sstableSet, OperationType.CLEANUP));
+ File compactionFileLocation = cfs.directories.getWriteableLocationAsFile(cfs.getExpectedCompactedFileSize(txn.originals(), OperationType.CLEANUP));
if (compactionFileLocation == null)
throw new IOException("disk full");
@@ -786,10 +787,9 @@ public class CompactionManager implements CompactionManagerMBean
CleanupInfo ci = new CleanupInfo(sstable, scanner);
metrics.beginCompaction(ci);
- Set<SSTableReader> oldSSTable = Sets.newHashSet(sstable);
List<SSTableReader> finished;
- try (SSTableRewriter writer = new SSTableRewriter(cfs, oldSSTable, sstable.maxDataAge, false);
- CompactionController controller = new CompactionController(cfs, sstableSet, getDefaultGcBefore(cfs)))
+ try (SSTableRewriter writer = new SSTableRewriter(cfs, txn, sstable.maxDataAge, false);
+ CompactionController controller = new CompactionController(cfs, txn.originals(), getDefaultGcBefore(cfs)))
{
writer.switchWriter(createWriter(cfs, compactionFileLocation, expectedBloomFilterSize, sstable.getSSTableMetadata().repairedAt, sstable));
@@ -811,7 +811,6 @@ public class CompactionManager implements CompactionManagerMBean
cfs.indexManager.flushIndexesBlocking();
finished = writer.finish();
- cfs.getDataTracker().markCompactedSSTablesReplaced(oldSSTable, finished, OperationType.CLEANUP);
}
finally
{
@@ -970,11 +969,11 @@ public class CompactionManager implements CompactionManagerMBean
}
}
return SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(compactionFileLocation)),
- (long)expectedBloomFilterSize,
- repairedAt,
- cfs.metadata,
- cfs.partitioner,
- new MetadataCollector(sstables, cfs.metadata.comparator, minLevel));
+ (long) expectedBloomFilterSize,
+ repairedAt,
+ cfs.metadata,
+ cfs.partitioner,
+ new MetadataCollector(sstables, cfs.metadata.comparator, minLevel));
}
@@ -1057,7 +1056,7 @@ public class CompactionManager implements CompactionManagerMBean
long numPartitions = 0;
for (SSTableReader sstable : sstables)
{
- numPartitions += sstable.estimatedKeysForRanges(Collections.singleton(validator.desc.range));
+ numPartitions += sstable.estimatedKeysForRanges(singleton(validator.desc.range));
}
// determine tree depth from number of partitions, but cap at 20 to prevent large tree.
int depth = numPartitions > 0 ? (int) Math.min(Math.floor(Math.log(numPartitions)), 20) : 0;
@@ -1119,37 +1118,39 @@ public class CompactionManager implements CompactionManagerMBean
* will store the non-repaired ranges. Once anticompation is completed, the original sstable is marked as compacted
* and subsequently deleted.
* @param cfs
- * @param repairedSSTables
+ * @param repaired a transaction over the repaired sstables to anticompacy
* @param ranges Repaired ranges to be placed into one of the new sstables. The repaired table will be tracked via
* the {@link org.apache.cassandra.io.sstable.metadata.StatsMetadata#repairedAt} field.
*/
- private void doAntiCompaction(ColumnFamilyStore cfs, Collection<Range<Token>> ranges,
- Collection<SSTableReader> repairedSSTables, long repairedAt)
+ private void doAntiCompaction(ColumnFamilyStore cfs, Collection<Range<Token>> ranges, LifecycleTransaction repaired, long repairedAt)
{
- logger.info("Performing anticompaction on {} sstables", repairedSSTables.size());
+ logger.info("Performing anticompaction on {} sstables", repaired.originals().size());
//Group SSTables
- Collection<Collection<SSTableReader>> groupedSSTables = cfs.getCompactionStrategy().groupSSTablesForAntiCompaction(repairedSSTables);
+ Collection<Collection<SSTableReader>> groupedSSTables = cfs.getCompactionStrategy().groupSSTablesForAntiCompaction(repaired.originals());
// iterate over sstables to check if the repaired / unrepaired ranges intersect them.
int antiCompactedSSTableCount = 0;
for (Collection<SSTableReader> sstableGroup : groupedSSTables)
{
- int antiCompacted = antiCompactGroup(cfs, ranges, sstableGroup, repairedAt);
- antiCompactedSSTableCount += antiCompacted;
+ try (LifecycleTransaction txn = repaired.split(sstableGroup))
+ {
+ int antiCompacted = antiCompactGroup(cfs, ranges, txn, repairedAt);
+ antiCompactedSSTableCount += antiCompacted;
+ }
}
String format = "Anticompaction completed successfully, anticompacted from {} to {} sstable(s).";
- logger.info(format, repairedSSTables.size(), antiCompactedSSTableCount);
+ logger.info(format, repaired.originals().size(), antiCompactedSSTableCount);
}
private int antiCompactGroup(ColumnFamilyStore cfs, Collection<Range<Token>> ranges,
- Collection<SSTableReader> anticompactionGroup, long repairedAt)
+ LifecycleTransaction anticompactionGroup, long repairedAt)
{
long groupMaxDataAge = -1;
// check that compaction hasn't stolen any sstables used in previous repair sessions
// if we need to skip the anticompaction, it will be carried out by the next repair
- for (Iterator<SSTableReader> i = anticompactionGroup.iterator(); i.hasNext();)
+ for (Iterator<SSTableReader> i = anticompactionGroup.originals().iterator(); i.hasNext();)
{
SSTableReader sstable = i.next();
if (!new File(sstable.getFilename()).exists())
@@ -1162,26 +1163,25 @@ public class CompactionManager implements CompactionManagerMBean
groupMaxDataAge = sstable.maxDataAge;
}
-
- if (anticompactionGroup.size() == 0)
+ if (anticompactionGroup.originals().size() == 0)
{
logger.info("No valid anticompactions for this group, All sstables were compacted and are no longer available");
return 0;
}
logger.info("Anticompacting {}", anticompactionGroup);
- Set<SSTableReader> sstableAsSet = new HashSet<>(anticompactionGroup);
+ Set<SSTableReader> sstableAsSet = anticompactionGroup.originals();
File destination = cfs.directories.getWriteableLocationAsFile(cfs.getExpectedCompactedFileSize(sstableAsSet, OperationType.ANTICOMPACTION));
long repairedKeyCount = 0;
long unrepairedKeyCount = 0;
AbstractCompactionStrategy strategy = cfs.getCompactionStrategy();
- try (SSTableRewriter repairedSSTableWriter = new SSTableRewriter(cfs, sstableAsSet, groupMaxDataAge, false);
- SSTableRewriter unRepairedSSTableWriter = new SSTableRewriter(cfs, sstableAsSet, groupMaxDataAge, false);
- AbstractCompactionStrategy.ScannerList scanners = strategy.getScanners(anticompactionGroup);
+ try (SSTableRewriter repairedSSTableWriter = new SSTableRewriter(cfs, anticompactionGroup, groupMaxDataAge, false);
+ SSTableRewriter unRepairedSSTableWriter = new SSTableRewriter(cfs, anticompactionGroup, groupMaxDataAge, false);
+ AbstractCompactionStrategy.ScannerList scanners = strategy.getScanners(anticompactionGroup.originals());
CompactionController controller = new CompactionController(cfs, sstableAsSet, getDefaultGcBefore(cfs)))
{
- int expectedBloomFilterSize = Math.max(cfs.metadata.getMinIndexInterval(), (int)(SSTableReader.getApproximateKeyCount(anticompactionGroup)));
+ int expectedBloomFilterSize = Math.max(cfs.metadata.getMinIndexInterval(), (int)(SSTableReader.getApproximateKeyCount(sstableAsSet)));
repairedSSTableWriter.switchWriter(CompactionManager.createWriterForAntiCompaction(cfs, destination, expectedBloomFilterSize, repairedAt, sstableAsSet));
unRepairedSSTableWriter.switchWriter(CompactionManager.createWriterForAntiCompaction(cfs, destination, expectedBloomFilterSize, ActiveRepairService.UNREPAIRED_SSTABLE, sstableAsSet));
@@ -1212,12 +1212,18 @@ public class CompactionManager implements CompactionManagerMBean
{
metrics.finishCompaction(ci);
}
- // we have the same readers being rewritten by both writers, so we ask the first one NOT to close them
- // so that the second one can do so safely, without leaving us with references < 0 or any other ugliness
+
List<SSTableReader> anticompactedSSTables = new ArrayList<>();
- anticompactedSSTables.addAll(repairedSSTableWriter.setRepairedAt(repairedAt).finish());
- anticompactedSSTables.addAll(unRepairedSSTableWriter.setRepairedAt(ActiveRepairService.UNREPAIRED_SSTABLE).finish());
- cfs.getDataTracker().markCompactedSSTablesReplaced(sstableAsSet, anticompactedSSTables, OperationType.ANTICOMPACTION);
+ // since both writers are operating over the same Transaction, we cannot use the convenience Transactional.finish() method,
+ // as on the second finish() we would prepareToCommit() on a Transaction that has already been committed, which is forbidden by the API
+ // (since it indicates misuse). We call permitRedundantTransitions so that calls that transition to a state already occupied are permitted.
+ anticompactionGroup.permitRedundantTransitions();
+ repairedSSTableWriter.setRepairedAt(repairedAt).prepareToCommit();
+ unRepairedSSTableWriter.prepareToCommit();
+ anticompactedSSTables.addAll(repairedSSTableWriter.finished());
+ anticompactedSSTables.addAll(unRepairedSSTableWriter.finished());
+ repairedSSTableWriter.commit();
+ unRepairedSSTableWriter.commit();
logger.debug("Repaired {} keys out of {} for {}/{} in {}", repairedKeyCount,
repairedKeyCount + unrepairedKeyCount,
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
index 34f57c1..e593ec0 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@ -17,8 +17,6 @@
*/
package org.apache.cassandra.db.compaction;
-import java.io.File;
-import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
import java.util.Iterator;
@@ -44,6 +42,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.SystemKeyspace;
import org.apache.cassandra.db.compaction.CompactionManager.CompactionExecutorStatsCollector;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.service.ActiveRepairService;
import org.apache.cassandra.utils.UUIDGen;
@@ -55,9 +54,9 @@ public class CompactionTask extends AbstractCompactionTask
protected static long totalBytesCompacted = 0;
private CompactionExecutorStatsCollector collector;
- public CompactionTask(ColumnFamilyStore cfs, Iterable<SSTableReader> sstables, int gcBefore, boolean offline)
+ public CompactionTask(ColumnFamilyStore cfs, LifecycleTransaction txn, int gcBefore, boolean offline)
{
- super(cfs, Sets.newHashSet(sstables));
+ super(cfs, txn);
this.gcBefore = gcBefore;
this.offline = offline;
}
@@ -71,23 +70,20 @@ public class CompactionTask extends AbstractCompactionTask
{
this.collector = collector;
run();
- return sstables.size();
+ return transaction.originals().size();
}
public boolean reduceScopeForLimitedSpace()
{
- if (partialCompactionsAcceptable() && sstables.size() > 1)
+ if (partialCompactionsAcceptable() && transaction.originals().size() > 1)
{
// Try again w/o the largest one.
- logger.warn("insufficient space to compact all requested files {}", StringUtils.join(sstables, ", "));
+ logger.warn("insufficient space to compact all requested files {}", StringUtils.join(transaction.originals(), ", "));
// Note that we have removed files that are still marked as compacting.
// This suboptimal but ok since the caller will unmark all the sstables at the end.
- SSTableReader removedSSTable = cfs.getMaxSizeFile(sstables);
- if (sstables.remove(removedSSTable))
- {
- cfs.getDataTracker().unmarkCompacting(Arrays.asList(removedSSTable));
- return true;
- }
+ SSTableReader removedSSTable = cfs.getMaxSizeFile(transaction.originals());
+ transaction.cancel(removedSSTable);
+ return true;
}
return false;
}
@@ -101,9 +97,9 @@ public class CompactionTask extends AbstractCompactionTask
{
// The collection of sstables passed may be empty (but not null); even if
// it is not empty, it may compact down to nothing if all rows are deleted.
- assert sstables != null;
+ assert transaction != null;
- if (sstables.size() == 0)
+ if (transaction.originals().isEmpty())
return;
// Note that the current compaction strategy, is not necessarily the one this task was created under.
@@ -115,12 +111,12 @@ public class CompactionTask extends AbstractCompactionTask
// note that we need to do a rough estimate early if we can fit the compaction on disk - this is pessimistic, but
// since we might remove sstables from the compaction in checkAvailableDiskSpace it needs to be done here
- long expectedWriteSize = cfs.getExpectedCompactedFileSize(sstables, compactionType);
+ long expectedWriteSize = cfs.getExpectedCompactedFileSize(transaction.originals(), compactionType);
long earlySSTableEstimate = Math.max(1, expectedWriteSize / strategy.getMaxSSTableBytes());
checkAvailableDiskSpace(earlySSTableEstimate, expectedWriteSize);
// sanity check: all sstables must belong to the same cfs
- assert !Iterables.any(sstables, new Predicate<SSTableReader>()
+ assert !Iterables.any(transaction.originals(), new Predicate<SSTableReader>()
{
@Override
public boolean apply(SSTableReader sstable)
@@ -129,13 +125,13 @@ public class CompactionTask extends AbstractCompactionTask
}
});
- UUID taskId = SystemKeyspace.startCompaction(cfs, sstables);
+ UUID taskId = SystemKeyspace.startCompaction(cfs, transaction.originals());
// new sstables from flush can be added during a compaction, but only the compaction can remove them,
// so in our single-threaded compaction world this is a valid way of determining if we're compacting
// all the sstables (that existed when we started)
StringBuilder ssTableLoggerMsg = new StringBuilder("[");
- for (SSTableReader sstr : sstables)
+ for (SSTableReader sstr : transaction.originals())
{
ssTableLoggerMsg.append(String.format("%s:level=%d, ", sstr.getFilename(), sstr.getSSTableLevel()));
}
@@ -148,11 +144,11 @@ public class CompactionTask extends AbstractCompactionTask
long totalKeysWritten = 0;
long estimatedKeys = 0;
- try (CompactionController controller = getCompactionController(sstables))
+ try (CompactionController controller = getCompactionController(transaction.originals()))
{
- Set<SSTableReader> actuallyCompact = Sets.difference(sstables, controller.getFullyExpiredSSTables());
+ Set<SSTableReader> actuallyCompact = Sets.difference(transaction.originals(), controller.getFullyExpiredSSTables());
- SSTableFormat.Type sstableFormat = getFormatType(sstables);
+ SSTableFormat.Type sstableFormat = getFormatType(transaction.originals());
List<SSTableReader> newSStables;
AbstractCompactionIterable ci;
@@ -171,7 +167,7 @@ public class CompactionTask extends AbstractCompactionTask
if (!controller.cfs.getCompactionStrategy().isActive)
throw new CompactionInterruptedException(ci.getCompactionInfo());
- try (CompactionAwareWriter writer = getCompactionAwareWriter(cfs, sstables, actuallyCompact))
+ try (CompactionAwareWriter writer = getCompactionAwareWriter(cfs, transaction, actuallyCompact))
{
estimatedKeys = writer.estimatedKeys();
while (iter.hasNext())
@@ -205,13 +201,9 @@ public class CompactionTask extends AbstractCompactionTask
}
}
- Collection<SSTableReader> oldSStables = this.sstables;
- if (!offline)
- cfs.getDataTracker().markCompactedSSTablesReplaced(oldSStables, newSStables, compactionType);
-
// log a bunch of statistics about the result and save to system table compaction_history
long dTime = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
- long startsize = SSTableReader.getTotalBytes(oldSStables);
+ long startsize = SSTableReader.getTotalBytes(transaction.originals());
long endsize = SSTableReader.getTotalBytes(newSStables);
double ratio = (double) endsize / (double) startsize;
@@ -223,7 +215,7 @@ public class CompactionTask extends AbstractCompactionTask
long totalSourceRows = 0;
String mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), ci, startsize, endsize);
logger.info(String.format("Compacted (%s) %d sstables to [%s] to level=%d. %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s. %,d total partitions merged to %,d. Partition merge counts were {%s}",
- taskIdLoggerMsg, oldSStables.size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, totalSourceRows, totalKeysWritten, mergeSummary));
+ taskIdLoggerMsg, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, totalSourceRows, totalKeysWritten, mergeSummary));
logger.debug(String.format("CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize)));
logger.debug("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double)(totalKeysWritten - estimatedKeys)/totalKeysWritten));
@@ -236,9 +228,9 @@ public class CompactionTask extends AbstractCompactionTask
}
@Override
- public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables)
+ public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, LifecycleTransaction transaction, Set<SSTableReader> nonExpiredSSTables)
{
- return new DefaultCompactionWriter(cfs, allSSTables, nonExpiredSSTables, offline, compactionType);
+ return new DefaultCompactionWriter(cfs, transaction, nonExpiredSSTables, offline, compactionType);
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java
index 6385671..18d5f7b 100644
--- a/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java
@@ -28,6 +28,7 @@ import org.slf4j.LoggerFactory;
import org.apache.cassandra.cql3.statements.CFPropDefs;
import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.utils.Pair;
@@ -67,8 +68,9 @@ public class DateTieredCompactionStrategy extends AbstractCompactionStrategy
if (latestBucket.isEmpty())
return null;
- if (cfs.getDataTracker().markCompacting(latestBucket))
- return new CompactionTask(cfs, latestBucket, gcBefore, false);
+ LifecycleTransaction modifier = cfs.getTracker().tryModify(latestBucket, OperationType.COMPACTION);
+ if (modifier != null)
+ return new CompactionTask(cfs, modifier, gcBefore, false);
}
}
@@ -366,11 +368,11 @@ public class DateTieredCompactionStrategy extends AbstractCompactionStrategy
@Override
public synchronized Collection<AbstractCompactionTask> getMaximalTask(int gcBefore, boolean splitOutput)
{
- Iterable<SSTableReader> sstables = cfs.markAllCompacting();
- if (sstables == null)
+ LifecycleTransaction modifier = cfs.markAllCompacting(OperationType.COMPACTION);
+ if (modifier == null)
return null;
- return Arrays.<AbstractCompactionTask>asList(new CompactionTask(cfs, sstables, gcBefore, false));
+ return Arrays.<AbstractCompactionTask>asList(new CompactionTask(cfs, modifier, gcBefore, false));
}
@Override
@@ -378,13 +380,14 @@ public class DateTieredCompactionStrategy extends AbstractCompactionStrategy
{
assert !sstables.isEmpty(); // checked for by CM.submitUserDefined
- if (!cfs.getDataTracker().markCompacting(sstables))
+ LifecycleTransaction modifier = cfs.getTracker().tryModify(sstables, OperationType.COMPACTION);
+ if (modifier == null)
{
logger.debug("Unable to mark {} for compaction; probably a background compaction got to it first. You can disable background compactions temporarily if this is a problem", sstables);
return null;
}
- return new CompactionTask(cfs, sstables, gcBefore, false).setUserDefined(true);
+ return new CompactionTask(cfs, modifier, gcBefore, false).setUserDefined(true);
}
public int getEstimatedRemainingTasks()
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
index 6b82ad3..c434d31 100644
--- a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
@@ -33,6 +33,7 @@ import org.slf4j.LoggerFactory;
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.dht.Range;
import org.apache.cassandra.dht.Token;
import org.apache.cassandra.exceptions.ConfigurationException;
@@ -115,9 +116,10 @@ public class LeveledCompactionStrategy extends AbstractCompactionStrategy
op = OperationType.COMPACTION;
}
- if (cfs.getDataTracker().markCompacting(candidate.sstables))
+ LifecycleTransaction txn = cfs.getTracker().tryModify(candidate.sstables, OperationType.COMPACTION);
+ if (txn != null)
{
- LeveledCompactionTask newTask = new LeveledCompactionTask(cfs, candidate.sstables, candidate.level, gcBefore, candidate.maxSSTableBytes, false);
+ LeveledCompactionTask newTask = new LeveledCompactionTask(cfs, txn, candidate.level, gcBefore, candidate.maxSSTableBytes, false);
newTask.setCompactionType(op);
return newTask;
}
@@ -131,9 +133,10 @@ public class LeveledCompactionStrategy extends AbstractCompactionStrategy
Iterable<SSTableReader> filteredSSTables = filterSuspectSSTables(sstables);
if (Iterables.isEmpty(sstables))
return null;
- if (!cfs.getDataTracker().markCompacting(filteredSSTables))
+ LifecycleTransaction txn = cfs.getTracker().tryModify(filteredSSTables, OperationType.COMPACTION);
+ if (txn == null)
return null;
- return Arrays.<AbstractCompactionTask>asList(new LeveledCompactionTask(cfs, filteredSSTables, 0, gcBefore, getMaxSSTableBytes(), true));
+ return Arrays.<AbstractCompactionTask>asList(new LeveledCompactionTask(cfs, txn, 0, gcBefore, getMaxSSTableBytes(), true));
}
@@ -144,19 +147,19 @@ public class LeveledCompactionStrategy extends AbstractCompactionStrategy
}
@Override
- public AbstractCompactionTask getCompactionTask(Collection<SSTableReader> sstables, int gcBefore, long maxSSTableBytes)
+ public AbstractCompactionTask getCompactionTask(LifecycleTransaction txn, int gcBefore, long maxSSTableBytes)
{
- assert sstables.size() > 0;
+ assert txn.originals().size() > 0;
int level = -1;
// if all sstables are in the same level, we can set that level:
- for (SSTableReader sstable : sstables)
+ for (SSTableReader sstable : txn.originals())
{
if (level == -1)
level = sstable.getSSTableLevel();
if (level != sstable.getSSTableLevel())
level = 0;
}
- return new LeveledCompactionTask(cfs, sstables, level, gcBefore, maxSSTableBytes, false);
+ return new LeveledCompactionTask(cfs, txn, level, gcBefore, maxSSTableBytes, false);
}
/**
@@ -226,7 +229,7 @@ public class LeveledCompactionStrategy extends AbstractCompactionStrategy
{
for (Integer level : byLevel.keySet())
{
- // level can be -1 when sstables are added to DataTracker but not to LeveledManifest
+ // level can be -1 when sstables are added to Tracker but not to LeveledManifest
// since we don't know which level those sstable belong yet, we simply do the same as L0 sstables.
if (level <= 0)
{
@@ -402,7 +405,7 @@ public class LeveledCompactionStrategy extends AbstractCompactionStrategy
if (sstables.isEmpty())
continue;
- Set<SSTableReader> compacting = cfs.getDataTracker().getCompacting();
+ Set<SSTableReader> compacting = cfs.getTracker().getCompacting();
for (SSTableReader sstable : sstables)
{
if (sstable.getEstimatedDroppableTombstoneRatio(gcBefore) <= tombstoneThreshold)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/LeveledCompactionTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionTask.java b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionTask.java
index ce9dfaf..1c3b686 100644
--- a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionTask.java
@@ -24,6 +24,7 @@ import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter;
import org.apache.cassandra.db.compaction.writers.MajorLeveledCompactionWriter;
import org.apache.cassandra.db.compaction.writers.MaxSSTableSizeWriter;
import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
public class LeveledCompactionTask extends CompactionTask
{
@@ -31,20 +32,20 @@ public class LeveledCompactionTask extends CompactionTask
private final long maxSSTableBytes;
private final boolean majorCompaction;
- public LeveledCompactionTask(ColumnFamilyStore cfs, Iterable<SSTableReader> sstables, int level, final int gcBefore, long maxSSTableBytes, boolean majorCompaction)
+ public LeveledCompactionTask(ColumnFamilyStore cfs, LifecycleTransaction txn, int level, int gcBefore, long maxSSTableBytes, boolean majorCompaction)
{
- super(cfs, sstables, gcBefore, false);
+ super(cfs, txn, gcBefore, false);
this.level = level;
this.maxSSTableBytes = maxSSTableBytes;
this.majorCompaction = majorCompaction;
}
@Override
- public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables)
+ public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables)
{
if (majorCompaction)
- return new MajorLeveledCompactionWriter(cfs, sstables, nonExpiredSSTables, maxSSTableBytes, false, compactionType);
- return new MaxSSTableSizeWriter(cfs, sstables, nonExpiredSSTables, maxSSTableBytes, getLevel(), false, compactionType);
+ return new MajorLeveledCompactionWriter(cfs, txn, nonExpiredSSTables, maxSSTableBytes, false, compactionType);
+ return new MaxSSTableSizeWriter(cfs, txn, nonExpiredSSTables, maxSSTableBytes, getLevel(), false, compactionType);
}
@Override
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java b/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
index daff131..0d0928f 100644
--- a/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
+++ b/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
@@ -23,7 +23,6 @@ import java.util.*;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Predicate;
import com.google.common.base.Predicates;
-import com.google.common.collect.ImmutableSet;
import com.google.common.collect.ImmutableSortedSet;
import com.google.common.collect.Iterables;
import com.google.common.collect.Sets;
@@ -316,7 +315,7 @@ public class LeveledManifest
continue; // mostly this just avoids polluting the debug log with zero scores
// we want to calculate score excluding compacting ones
Set<SSTableReader> sstablesInLevel = Sets.newHashSet(sstables);
- Set<SSTableReader> remaining = Sets.difference(sstablesInLevel, cfs.getDataTracker().getCompacting());
+ Set<SSTableReader> remaining = Sets.difference(sstablesInLevel, cfs.getTracker().getCompacting());
double score = (double) SSTableReader.getTotalBytes(remaining) / (double)maxBytesForLevel(i, maxSSTableSizeInBytes);
logger.debug("Compaction score for level {} is {}", i, score);
@@ -361,7 +360,7 @@ public class LeveledManifest
private List<SSTableReader> getSSTablesForSTCS(Collection<SSTableReader> sstables)
{
- Iterable<SSTableReader> candidates = cfs.getDataTracker().getUncompactingSSTables(sstables);
+ Iterable<SSTableReader> candidates = cfs.getTracker().getUncompacting(sstables);
List<Pair<SSTableReader,Long>> pairs = SizeTieredCompactionStrategy.createSSTableAndLengthPairs(AbstractCompactionStrategy.filterSuspectSSTables(candidates));
List<List<SSTableReader>> buckets = SizeTieredCompactionStrategy.getBuckets(pairs,
options.bucketHigh,
@@ -415,7 +414,7 @@ public class LeveledManifest
}
if (min == null || max == null || min.equals(max)) // single partition sstables - we cannot include a high level sstable.
return candidates;
- Set<SSTableReader> compacting = cfs.getDataTracker().getCompacting();
+ Set<SSTableReader> compacting = cfs.getTracker().getCompacting();
Range<RowPosition> boundaries = new Range<>(min, max);
for (SSTableReader sstable : getLevel(i))
{
@@ -542,7 +541,7 @@ public class LeveledManifest
assert !getLevel(level).isEmpty();
logger.debug("Choosing candidates for L{}", level);
- final Set<SSTableReader> compacting = cfs.getDataTracker().getCompacting();
+ final Set<SSTableReader> compacting = cfs.getTracker().getCompacting();
if (level == 0)
{
@@ -650,7 +649,7 @@ public class LeveledManifest
{
Set<SSTableReader> sstables = new HashSet<>();
Set<SSTableReader> levelSSTables = new HashSet<>(getLevel(level));
- for (SSTableReader sstable : cfs.getDataTracker().getCompacting())
+ for (SSTableReader sstable : cfs.getTracker().getCompacting())
{
if (levelSSTables.contains(sstable))
sstables.add(sstable);
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java b/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java
index 8d7b0e9..e9a4f05 100644
--- a/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java
+++ b/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java
@@ -23,6 +23,7 @@ import org.apache.cassandra.db.*;
import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter;
import org.apache.cassandra.db.compaction.writers.MaxSSTableSizeWriter;
import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
public class SSTableSplitter {
@@ -30,9 +31,9 @@ public class SSTableSplitter {
private CompactionInfo.Holder info;
- public SSTableSplitter(ColumnFamilyStore cfs, SSTableReader sstable, int sstableSizeInMB)
+ public SSTableSplitter(ColumnFamilyStore cfs, LifecycleTransaction transaction, int sstableSizeInMB)
{
- this.task = new SplittingCompactionTask(cfs, sstable, sstableSizeInMB);
+ this.task = new SplittingCompactionTask(cfs, transaction, sstableSizeInMB);
}
public void split()
@@ -57,9 +58,9 @@ public class SSTableSplitter {
{
private final int sstableSizeInMB;
- public SplittingCompactionTask(ColumnFamilyStore cfs, SSTableReader sstable, int sstableSizeInMB)
+ public SplittingCompactionTask(ColumnFamilyStore cfs, LifecycleTransaction transaction, int sstableSizeInMB)
{
- super(cfs, Collections.singletonList(sstable), CompactionManager.NO_GC, true);
+ super(cfs, transaction, CompactionManager.NO_GC, true);
this.sstableSizeInMB = sstableSizeInMB;
if (sstableSizeInMB <= 0)
@@ -73,9 +74,9 @@ public class SSTableSplitter {
}
@Override
- public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables)
+ public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables)
{
- return new MaxSSTableSizeWriter(cfs, sstables, nonExpiredSSTables, sstableSizeInMB * 1024L * 1024L, 0, true, compactionType);
+ return new MaxSSTableSizeWriter(cfs, txn, nonExpiredSSTables, sstableSizeInMB * 1024L * 1024L, 0, true, compactionType);
}
@Override
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/Scrubber.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/Scrubber.java b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
index 1e014ed..b7c149c 100644
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@ -23,9 +23,9 @@ import java.util.*;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Throwables;
-import com.google.common.collect.Sets;
import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.io.sstable.*;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.sstable.format.SSTableWriter;
@@ -41,6 +41,7 @@ public class Scrubber implements Closeable
{
private final ColumnFamilyStore cfs;
private final SSTableReader sstable;
+ private final LifecycleTransaction transaction;
private final File destination;
private final boolean skipCorrupted;
@@ -80,15 +81,16 @@ public class Scrubber implements Closeable
};
private final SortedSet<Row> outOfOrderRows = new TreeSet<>(rowComparator);
- public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean isOffline, boolean checkData) throws IOException
+ public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, boolean isOffline, boolean checkData) throws IOException
{
- this(cfs, sstable, skipCorrupted, new OutputHandler.LogOutput(), isOffline, checkData);
+ this(cfs, transaction, skipCorrupted, new OutputHandler.LogOutput(), isOffline, checkData);
}
- public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, OutputHandler outputHandler, boolean isOffline, boolean checkData) throws IOException
+ public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, OutputHandler outputHandler, boolean isOffline, boolean checkData) throws IOException
{
this.cfs = cfs;
- this.sstable = sstable;
+ this.transaction = transaction;
+ this.sstable = transaction.onlyOne();
this.outputHandler = outputHandler;
this.skipCorrupted = skipCorrupted;
this.isOffline = isOffline;
@@ -127,9 +129,7 @@ public class Scrubber implements Closeable
public void scrub()
{
outputHandler.output(String.format("Scrubbing %s (%s bytes)", sstable, dataFile.length()));
- Set<SSTableReader> oldSSTable = Sets.newHashSet(sstable);
-
- try (SSTableRewriter writer = new SSTableRewriter(cfs, oldSSTable, sstable.maxDataAge, isOffline);)
+ try (SSTableRewriter writer = new SSTableRewriter(cfs, transaction, sstable.maxDataAge, isOffline);)
{
nextIndexKey = ByteBufferUtil.readWithShortLength(indexFile);
{
@@ -278,8 +278,7 @@ public class Scrubber implements Closeable
inOrderWriter.append(row.key, row.cf);
newInOrderSstable = inOrderWriter.finish(-1, sstable.maxDataAge, true);
}
- if (!isOffline)
- cfs.getDataTracker().addSSTables(Collections.singleton(newInOrderSstable));
+ transaction.update(newInOrderSstable, false);
outputHandler.warn(String.format("%d out of order rows found while scrubbing %s; Those have been written (in order) to a new sstable (%s)", outOfOrderRows.size(), sstable, newInOrderSstable));
}
@@ -287,8 +286,6 @@ public class Scrubber implements Closeable
List<SSTableReader> finished = writer.setRepairedAt(badRows > 0 ? ActiveRepairService.UNREPAIRED_SSTABLE : sstable.getSSTableMetadata().repairedAt).finish();
if (!finished.isEmpty())
newSstable = finished.get(0);
- if (!isOffline)
- cfs.getDataTracker().markCompactedSSTablesReplaced(oldSSTable, finished, OperationType.SCRUB);
}
catch (IOException e)
{
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
index 722536c..94c3daf 100644
--- a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
@@ -21,22 +21,19 @@ import java.util.*;
import java.util.Map.Entry;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter;
import org.apache.cassandra.db.compaction.writers.SplittingSizeTieredCompactionWriter;
import org.apache.cassandra.io.sstable.format.SSTableReader;
-import com.google.common.primitives.Longs;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.cassandra.cql3.statements.CFPropDefs;
import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.io.sstable.ColumnNameHelper;
import org.apache.cassandra.utils.Pair;
public class SizeTieredCompactionStrategy extends AbstractCompactionStrategy
@@ -190,8 +187,9 @@ public class SizeTieredCompactionStrategy extends AbstractCompactionStrategy
if (hottestBucket.isEmpty())
return null;
- if (cfs.getDataTracker().markCompacting(hottestBucket))
- return new CompactionTask(cfs, hottestBucket, gcBefore, false);
+ LifecycleTransaction transaction = cfs.getTracker().tryModify(hottestBucket, OperationType.COMPACTION);
+ if (transaction != null)
+ return new CompactionTask(cfs, transaction, gcBefore, false);
}
}
@@ -200,24 +198,26 @@ public class SizeTieredCompactionStrategy extends AbstractCompactionStrategy
Iterable<SSTableReader> filteredSSTables = filterSuspectSSTables(sstables);
if (Iterables.isEmpty(filteredSSTables))
return null;
- if (!cfs.getDataTracker().markCompacting(ImmutableList.copyOf(filteredSSTables)))
+ LifecycleTransaction txn = cfs.getTracker().tryModify(filteredSSTables, OperationType.COMPACTION);
+ if (txn == null)
return null;
if (splitOutput)
- return Arrays.<AbstractCompactionTask>asList(new SplittingCompactionTask(cfs, filteredSSTables, gcBefore, false));
- return Arrays.<AbstractCompactionTask>asList(new CompactionTask(cfs, filteredSSTables, gcBefore, false));
+ return Arrays.<AbstractCompactionTask>asList(new SplittingCompactionTask(cfs, txn, gcBefore, false));
+ return Arrays.<AbstractCompactionTask>asList(new CompactionTask(cfs, txn, gcBefore, false));
}
public AbstractCompactionTask getUserDefinedTask(Collection<SSTableReader> sstables, final int gcBefore)
{
assert !sstables.isEmpty(); // checked for by CM.submitUserDefined
- if (!cfs.getDataTracker().markCompacting(sstables))
+ LifecycleTransaction transaction = cfs.getTracker().tryModify(sstables, OperationType.COMPACTION);
+ if (transaction == null)
{
logger.debug("Unable to mark {} for compaction; probably a background compaction got to it first. You can disable background compactions temporarily if this is a problem", sstables);
return null;
}
- return new CompactionTask(cfs, sstables, gcBefore, false).setUserDefined(true);
+ return new CompactionTask(cfs, transaction, gcBefore, false).setUserDefined(true);
}
public int getEstimatedRemainingTasks()
@@ -338,15 +338,15 @@ public class SizeTieredCompactionStrategy extends AbstractCompactionStrategy
private static class SplittingCompactionTask extends CompactionTask
{
- public SplittingCompactionTask(ColumnFamilyStore cfs, Iterable<SSTableReader> sstables, int gcBefore, boolean offline)
+ public SplittingCompactionTask(ColumnFamilyStore cfs, LifecycleTransaction txn, int gcBefore, boolean offline)
{
- super(cfs, sstables, gcBefore, offline);
+ super(cfs, txn, gcBefore, offline);
}
@Override
- public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables)
+ public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables)
{
- return new SplittingSizeTieredCompactionWriter(cfs, allSSTables, nonExpiredSSTables, compactionType);
+ return new SplittingSizeTieredCompactionWriter(cfs, txn, nonExpiredSSTables, compactionType);
}
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/Upgrader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/Upgrader.java b/src/java/org/apache/cassandra/db/compaction/Upgrader.java
index 5bb1530..6556a71 100644
--- a/src/java/org/apache/cassandra/db/compaction/Upgrader.java
+++ b/src/java/org/apache/cassandra/db/compaction/Upgrader.java
@@ -20,12 +20,10 @@ package org.apache.cassandra.db.compaction;
import java.io.File;
import java.util.*;
-import com.google.common.base.Throwables;
-import com.google.common.collect.Sets;
-
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.io.sstable.*;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.sstable.format.SSTableWriter;
@@ -37,6 +35,7 @@ public class Upgrader
{
private final ColumnFamilyStore cfs;
private final SSTableReader sstable;
+ private final LifecycleTransaction transaction;
private final File directory;
private final OperationType compactionType = OperationType.UPGRADE_SSTABLES;
@@ -46,10 +45,11 @@ public class Upgrader
private final OutputHandler outputHandler;
- public Upgrader(ColumnFamilyStore cfs, SSTableReader sstable, OutputHandler outputHandler)
+ public Upgrader(ColumnFamilyStore cfs, LifecycleTransaction txn, OutputHandler outputHandler)
{
this.cfs = cfs;
- this.sstable = sstable;
+ this.transaction = txn;
+ this.sstable = txn.onlyOne();
this.outputHandler = outputHandler;
this.directory = new File(sstable.getFilename()).getParentFile();
@@ -81,10 +81,9 @@ public class Upgrader
public void upgrade()
{
outputHandler.output("Upgrading " + sstable);
- Set<SSTableReader> toUpgrade = Sets.newHashSet(sstable);
- try (SSTableRewriter writer = new SSTableRewriter(cfs, toUpgrade, CompactionTask.getMaxDataAge(toUpgrade), true);
- AbstractCompactionStrategy.ScannerList scanners = strategy.getScanners(toUpgrade))
+ try (SSTableRewriter writer = new SSTableRewriter(cfs, transaction, CompactionTask.getMaxDataAge(transaction.originals()), true);
+ AbstractCompactionStrategy.ScannerList scanners = strategy.getScanners(transaction.originals()))
{
Iterator<AbstractCompactedRow> iter = new CompactionIterable(compactionType, scanners.scanners, controller, DatabaseDescriptor.getSSTableFormat(), UUIDGen.getTimeUUID()).iterator();
writer.switchWriter(createCompactionWriter(sstable.getSSTableMetadata().repairedAt));
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/WrappingCompactionStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/WrappingCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/WrappingCompactionStrategy.java
index 5345d8d..c511bcd 100644
--- a/src/java/org/apache/cassandra/db/compaction/WrappingCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/WrappingCompactionStrategy.java
@@ -50,7 +50,7 @@ public final class WrappingCompactionStrategy extends AbstractCompactionStrategy
{
super(cfs, cfs.metadata.compactionStrategyOptions);
reloadCompactionStrategy(cfs.metadata);
- cfs.getDataTracker().subscribe(this);
+ cfs.getTracker().subscribe(this);
logger.debug("{} subscribed to the data tracker.", this);
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java
index fe43186..20c96d6 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java
@@ -25,6 +25,7 @@ import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.Directories;
import org.apache.cassandra.db.compaction.AbstractCompactedRow;
import org.apache.cassandra.db.compaction.CompactionTask;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.io.sstable.SSTableRewriter;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.utils.concurrent.Transactional;
@@ -43,14 +44,14 @@ public abstract class CompactionAwareWriter extends Transactional.AbstractTransa
protected final long minRepairedAt;
protected final SSTableRewriter sstableWriter;
- public CompactionAwareWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables, boolean offline)
+ public CompactionAwareWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, boolean offline)
{
this.cfs = cfs;
this.nonExpiredSSTables = nonExpiredSSTables;
this.estimatedTotalKeys = SSTableReader.getApproximateKeyCount(nonExpiredSSTables);
this.maxAge = CompactionTask.getMaxDataAge(nonExpiredSSTables);
this.minRepairedAt = CompactionTask.getMinRepairedAt(nonExpiredSSTables);
- this.sstableWriter = new SSTableRewriter(cfs, allSSTables, maxAge, offline);
+ this.sstableWriter = new SSTableRewriter(cfs, txn, maxAge, offline);
}
/**
@@ -67,12 +68,6 @@ public abstract class CompactionAwareWriter extends Transactional.AbstractTransa
}
@Override
- protected Throwable doCleanup(Throwable accumulate)
- {
- return accumulate;
- }
-
- @Override
protected Throwable doCommit(Throwable accumulate)
{
return sstableWriter.commit(accumulate);
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java
index 3589b54..0b31061 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.db.compaction.writers;
import java.io.File;
-import java.util.List;
import java.util.Set;
import org.slf4j.Logger;
@@ -28,14 +27,12 @@ import org.slf4j.LoggerFactory;
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.compaction.AbstractCompactedRow;
import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.SSTableRewriter;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.sstable.format.SSTableWriter;
import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
-import static org.apache.cassandra.utils.Throwables.maybeFail;
-
/**
* The default compaction writer - creates one output file in L0
@@ -44,9 +41,9 @@ public class DefaultCompactionWriter extends CompactionAwareWriter
{
protected static final Logger logger = LoggerFactory.getLogger(DefaultCompactionWriter.class);
- public DefaultCompactionWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables, boolean offline, OperationType compactionType)
+ public DefaultCompactionWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, boolean offline, OperationType compactionType)
{
- super(cfs, allSSTables, nonExpiredSSTables, offline);
+ super(cfs, txn, nonExpiredSSTables, offline);
logger.debug("Expected bloom filter size : {}", estimatedTotalKeys);
long expectedWriteSize = cfs.getExpectedCompactedFileSize(nonExpiredSSTables, compactionType);
File sstableDirectory = cfs.directories.getLocationForDisk(getWriteDirectory(expectedWriteSize));
@@ -55,7 +52,7 @@ public class DefaultCompactionWriter extends CompactionAwareWriter
minRepairedAt,
cfs.metadata,
cfs.partitioner,
- new MetadataCollector(allSSTables, cfs.metadata.comparator, 0));
+ new MetadataCollector(txn.originals(), cfs.metadata.comparator, 0));
sstableWriter.switchWriter(writer);
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java
index d48140e..014b4af 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java
@@ -18,8 +18,6 @@
package org.apache.cassandra.db.compaction.writers;
import java.io.File;
-import java.util.Collections;
-import java.util.List;
import java.util.Set;
import org.slf4j.Logger;
@@ -28,11 +26,10 @@ import org.slf4j.LoggerFactory;
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.RowIndexEntry;
import org.apache.cassandra.db.compaction.AbstractCompactedRow;
-import org.apache.cassandra.db.compaction.CompactionTask;
import org.apache.cassandra.db.compaction.LeveledManifest;
import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.SSTableRewriter;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.sstable.format.SSTableWriter;
import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
@@ -50,11 +47,11 @@ public class MajorLeveledCompactionWriter extends CompactionAwareWriter
private int sstablesWritten = 0;
private final boolean skipAncestors;
- public MajorLeveledCompactionWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables, long maxSSTableSize, boolean offline, OperationType compactionType)
+ public MajorLeveledCompactionWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, long maxSSTableSize, boolean offline, OperationType compactionType)
{
- super(cfs, allSSTables, nonExpiredSSTables, offline);
+ super(cfs, txn, nonExpiredSSTables, offline);
this.maxSSTableSize = maxSSTableSize;
- this.allSSTables = allSSTables;
+ this.allSSTables = txn.originals();
expectedWriteSize = Math.min(maxSSTableSize, cfs.getExpectedCompactedFileSize(nonExpiredSSTables, compactionType));
long estimatedSSTables = Math.max(1, SSTableReader.getTotalBytes(nonExpiredSSTables) / maxSSTableSize);
long keysPerSSTable = estimatedTotalKeys / estimatedSSTables;
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java
index ab24bf8..8903ff7 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java
@@ -18,16 +18,14 @@
package org.apache.cassandra.db.compaction.writers;
import java.io.File;
-import java.util.List;
import java.util.Set;
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.RowIndexEntry;
import org.apache.cassandra.db.compaction.AbstractCompactedRow;
-import org.apache.cassandra.db.compaction.CompactionTask;
import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.SSTableRewriter;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.sstable.format.SSTableWriter;
import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
@@ -41,10 +39,10 @@ public class MaxSSTableSizeWriter extends CompactionAwareWriter
private final long estimatedSSTables;
private final Set<SSTableReader> allSSTables;
- public MaxSSTableSizeWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables, long maxSSTableSize, int level, boolean offline, OperationType compactionType)
+ public MaxSSTableSizeWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, long maxSSTableSize, int level, boolean offline, OperationType compactionType)
{
- super(cfs, allSSTables, nonExpiredSSTables, offline);
- this.allSSTables = allSSTables;
+ super(cfs, txn, nonExpiredSSTables, offline);
+ this.allSSTables = txn.originals();
this.level = level;
this.maxSSTableSize = maxSSTableSize;
long totalSize = cfs.getExpectedCompactedFileSize(nonExpiredSSTables, compactionType);
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
index 2a452c7..81ea6b1 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.db.compaction.writers;
import java.io.File;
import java.util.Arrays;
-import java.util.List;
import java.util.Set;
import org.slf4j.Logger;
@@ -28,10 +27,9 @@ import org.slf4j.LoggerFactory;
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.RowIndexEntry;
import org.apache.cassandra.db.compaction.AbstractCompactedRow;
-import org.apache.cassandra.db.compaction.CompactionTask;
import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.SSTableRewriter;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.sstable.format.SSTableWriter;
import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
@@ -53,15 +51,15 @@ public class SplittingSizeTieredCompactionWriter extends CompactionAwareWriter
private long currentBytesToWrite;
private int currentRatioIndex = 0;
- public SplittingSizeTieredCompactionWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables, OperationType compactionType)
+ public SplittingSizeTieredCompactionWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, OperationType compactionType)
{
- this(cfs, allSSTables, nonExpiredSSTables, compactionType, DEFAULT_SMALLEST_SSTABLE_BYTES);
+ this(cfs, txn, nonExpiredSSTables, compactionType, DEFAULT_SMALLEST_SSTABLE_BYTES);
}
- public SplittingSizeTieredCompactionWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables, OperationType compactionType, long smallestSSTable)
+ public SplittingSizeTieredCompactionWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, OperationType compactionType, long smallestSSTable)
{
- super(cfs, allSSTables, nonExpiredSSTables, false);
- this.allSSTables = allSSTables;
+ super(cfs, txn, nonExpiredSSTables, false);
+ this.allSSTables = txn.originals();
totalSize = cfs.getExpectedCompactedFileSize(nonExpiredSSTables, compactionType);
double[] potentialRatios = new double[20];
double currentRatio = 1;
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java b/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
index f8b3aba..ba48350 100644
--- a/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
+++ b/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
@@ -57,7 +57,7 @@ public abstract class AbstractSimplePerColumnSecondaryIndex extends PerColumnSec
indexedCfMetadata.cfName,
new LocalPartitioner(getIndexKeyComparator()),
indexedCfMetadata,
- baseCfs.getDataTracker().loadsstables);
+ baseCfs.getTracker().loadsstables);
}
protected AbstractType<?> getIndexKeyComparator()
@@ -143,7 +143,7 @@ public abstract class AbstractSimplePerColumnSecondaryIndex extends PerColumnSec
{
Future<?> wait;
// we synchronise on the baseCfs to make sure we are ordered correctly with other flushes to the base CFS
- synchronized (baseCfs.getDataTracker())
+ synchronized (baseCfs.getTracker())
{
wait = indexCfs.forceFlush();
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java b/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
index dda532d..4c1bf45 100644
--- a/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
+++ b/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
@@ -346,7 +346,7 @@ public class SecondaryIndexManager
{
// despatch flushes for all CFS backed indexes
List<Future<?>> wait = new ArrayList<>();
- synchronized (baseCfs.getDataTracker())
+ synchronized (baseCfs.getTracker())
{
for (SecondaryIndex index : allIndexes)
if (index.getIndexCfs() != null)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/lifecycle/Helpers.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/Helpers.java b/src/java/org/apache/cassandra/db/lifecycle/Helpers.java
new file mode 100644
index 0000000..05f7531
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/lifecycle/Helpers.java
@@ -0,0 +1,241 @@
+/*
+ * 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.db.lifecycle;
+
+import java.util.*;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.*;
+
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+
+import static com.google.common.base.Predicates.*;
+import static com.google.common.collect.Iterables.any;
+import static com.google.common.collect.Iterables.concat;
+import static com.google.common.collect.Iterables.filter;
+import static com.google.common.collect.Iterables.getFirst;
+import static org.apache.cassandra.utils.Throwables.merge;
+
+class Helpers
+{
+ /**
+ * update the contents of a set with the provided sets, ensuring that the items to remove are
+ * really present, and that the items to add are not (unless we're also removing them)
+ * @return a new set with the contents of the provided one modified
+ */
+ static <T> Set<T> replace(Set<T> original, Set<T> remove, Iterable<T> add)
+ {
+ return ImmutableSet.copyOf(replace(identityMap(original), remove, add).keySet());
+ }
+
+ /**
+ * update the contents of an "identity map" with the provided sets, ensuring that the items to remove are
+ * really present, and that the items to add are not (unless we're also removing them)
+ * @return a new identity map with the contents of the provided one modified
+ */
+ static <T> Map<T, T> replace(Map<T, T> original, Set<T> remove, Iterable<T> add)
+ {
+ // ensure the ones being removed are the exact same ones present
+ for (T reader : remove)
+ assert original.get(reader) == reader;
+
+ // ensure we don't already contain any we're adding, that we aren't also removing
+ assert !any(add, and(not(in(remove)), in(original.keySet()))) : String.format("original:%s remove:%s add:%s", original.keySet(), remove, add);
+
+ Map<T, T> result =
+ identityMap(concat(add, filter(original.keySet(), not(in(remove)))));
+
+ assert result.size() == original.size() - remove.size() + Iterables.size(add) :
+ String.format("Expecting new size of %d, got %d while replacing %s by %s in %s",
+ original.size() - remove.size() + Iterables.size(add), result.size(), remove, add, original.keySet());
+ return result;
+ }
+
+ /**
+ * A convenience method for encapsulating this action over multiple SSTableReader with exception-safety
+ * @return accumulate if not null (with any thrown exception attached), or any thrown exception otherwise
+ */
+ static Throwable setupDeleteNotification(Iterable<SSTableReader> readers, Tracker tracker, Throwable accumulate)
+ {
+ try
+ {
+ for (SSTableReader reader : readers)
+ reader.setupDeleteNotification(tracker);
+ }
+ catch (Throwable t)
+ {
+ // shouldn't be possible, but in case the contract changes in future and we miss it...
+ accumulate = merge(accumulate, t);
+ }
+ return accumulate;
+ }
+
+ /**
+ * A convenience method for encapsulating this action over multiple SSTableReader with exception-safety
+ * @return accumulate if not null (with any thrown exception attached), or any thrown exception otherwise
+ */
+ static Throwable setReplaced(Iterable<SSTableReader> readers, Throwable accumulate)
+ {
+ for (SSTableReader reader : readers)
+ {
+ try
+ {
+ reader.setReplaced();
+ }
+ catch (Throwable t)
+ {
+ accumulate = merge(accumulate, t);
+ }
+ }
+ return accumulate;
+ }
+
+ /**
+ * assert that none of these readers have been replaced
+ */
+ static void checkNotReplaced(Iterable<SSTableReader> readers)
+ {
+ for (SSTableReader reader : readers)
+ assert !reader.isReplaced();
+ }
+
+ /**
+ * A convenience method for encapsulating this action over multiple SSTableReader with exception-safety
+ * @return accumulate if not null (with any thrown exception attached), or any thrown exception otherwise
+ */
+ static Throwable markObsolete(Iterable<SSTableReader> readers, Throwable accumulate)
+ {
+ for (SSTableReader reader : readers)
+ {
+ try
+ {
+ boolean firstToCompact = reader.markObsolete();
+ assert firstToCompact : reader + " was already marked compacted";
+ }
+ catch (Throwable t)
+ {
+ accumulate = merge(accumulate, t);
+ }
+ }
+ return accumulate;
+ }
+
+ /**
+ * @return the identity function, as a Map, with domain of the provided values
+ */
+ static <T> Map<T, T> identityMap(Iterable<T> values)
+ {
+ ImmutableMap.Builder<T, T> builder = ImmutableMap.<T, T>builder();
+ for (T t : values)
+ builder.put(t, t);
+ return builder.build();
+ }
+
+ /**
+ * @return an Iterable of the union if the sets, with duplicates being represented by their first encountered instance
+ * (as defined by the order of set provision)
+ */
+ static <T> Iterable<T> concatUniq(Set<T>... sets)
+ {
+ List<Predicate<T>> notIn = new ArrayList<>(sets.length);
+ for (Set<T> set : sets)
+ notIn.add(not(in(set)));
+ List<Iterable<T>> results = new ArrayList<>(sets.length);
+ for (int i = 0 ; i < sets.length ; i++)
+ results.add(filter(sets[i], and(notIn.subList(0, i))));
+ return concat(results);
+ }
+
+ /**
+ * @return a Predicate yielding true for an item present in NONE of the provided sets
+ */
+ static <T> Predicate<T> notIn(Set<T>... sets)
+ {
+ return not(orIn(sets));
+ }
+
+ /**
+ * @return a Predicate yielding true for an item present in ANY of the provided sets
+ */
+ static <T> Predicate<T> orIn(Collection<T>... sets)
+ {
+ Predicate<T>[] orIn = new Predicate[sets.length];
+ for (int i = 0 ; i < orIn.length ; i++)
+ orIn[i] = in(sets[i]);
+ return or(orIn);
+ }
+
+ /**
+ * filter out (i.e. remove) matching elements
+ * @return filter, filtered to only those elements that *are not* present in *any* of the provided sets (are present in none)
+ */
+ static <T> Iterable<T> filterOut(Iterable<T> filter, Set<T>... inNone)
+ {
+ return filter(filter, notIn(inNone));
+ }
+
+ /**
+ * filter in (i.e. retain)
+ *
+ * @return filter, filtered to only those elements that *are* present in *any* of the provided sets
+ */
+ static <T> Iterable<T> filterIn(Iterable<T> filter, Set<T>... inAny)
+ {
+ return filter(filter, orIn(inAny));
+ }
+
+ static Set<SSTableReader> emptySet()
+ {
+ return Collections.emptySet();
+ }
+
+ static <T> T select(T t, Collection<T> col)
+ {
+ if (col instanceof Set && !col.contains(t))
+ return null;
+ return getFirst(filter(col, equalTo(t)), null);
+ }
+
+ static <T> T selectFirst(T t, Collection<T> ... sets)
+ {
+ for (Collection<T> set : sets)
+ {
+ T select = select(t, set);
+ if (select != null)
+ return select;
+ }
+ return null;
+ }
+
+ static <T> Predicate<T> idIn(Set<T> set)
+ {
+ return idIn(identityMap(set));
+ }
+
+ static <T> Predicate<T> idIn(final Map<T, T> identityMap)
+ {
+ return new Predicate<T>()
+ {
+ public boolean apply(T t)
+ {
+ return identityMap.get(t) == t;
+ }
+ };
+ }
+
+}
[6/6] cassandra git commit: Extend Transactional API to sstable
lifecycle management
Posted by be...@apache.org.
Extend Transactional API to sstable lifecycle management
patch by benedict; reviewed by marcus for CASSANDRA-8568
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/e5a76bdb
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/e5a76bdb
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/e5a76bdb
Branch: refs/heads/cassandra-2.2
Commit: e5a76bdb5fc04ffa16b8becaa7877186226c3b32
Parents: 33d71b8
Author: Benedict Elliott Smith <be...@apache.org>
Authored: Thu Mar 12 10:23:35 2015 +0000
Committer: Benedict Elliott Smith <be...@apache.org>
Committed: Fri May 22 09:44:36 2015 +0100
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../apache/cassandra/db/ColumnFamilyStore.java | 249 ++++--
.../org/apache/cassandra/db/DataTracker.java | 793 -------------------
.../cassandra/db/HintedHandOffManager.java | 2 +-
src/java/org/apache/cassandra/db/Keyspace.java | 15 +-
src/java/org/apache/cassandra/db/Memtable.java | 22 +-
.../compaction/AbstractCompactionStrategy.java | 7 +-
.../db/compaction/AbstractCompactionTask.java | 19 +-
.../db/compaction/CompactionController.java | 4 +-
.../db/compaction/CompactionManager.java | 182 +++--
.../cassandra/db/compaction/CompactionTask.java | 54 +-
.../DateTieredCompactionStrategy.java | 17 +-
.../compaction/LeveledCompactionStrategy.java | 23 +-
.../db/compaction/LeveledCompactionTask.java | 11 +-
.../db/compaction/LeveledManifest.java | 11 +-
.../db/compaction/SSTableSplitter.java | 13 +-
.../cassandra/db/compaction/Scrubber.java | 21 +-
.../SizeTieredCompactionStrategy.java | 30 +-
.../cassandra/db/compaction/Upgrader.java | 15 +-
.../compaction/WrappingCompactionStrategy.java | 2 +-
.../writers/CompactionAwareWriter.java | 11 +-
.../writers/DefaultCompactionWriter.java | 11 +-
.../writers/MajorLeveledCompactionWriter.java | 11 +-
.../writers/MaxSSTableSizeWriter.java | 10 +-
.../SplittingSizeTieredCompactionWriter.java | 14 +-
.../AbstractSimplePerColumnSecondaryIndex.java | 4 +-
.../db/index/SecondaryIndexManager.java | 2 +-
.../apache/cassandra/db/lifecycle/Helpers.java | 241 ++++++
.../db/lifecycle/LifecycleTransaction.java | 511 ++++++++++++
.../db/lifecycle/SSTableIntervalTree.java | 40 +
.../apache/cassandra/db/lifecycle/Tracker.java | 468 +++++++++++
.../org/apache/cassandra/db/lifecycle/View.java | 252 ++++++
.../io/compress/CompressionMetadata.java | 2 +-
.../io/sstable/IndexSummaryManager.java | 106 ++-
.../io/sstable/SSTableDeletingTask.java | 27 +-
.../cassandra/io/sstable/SSTableRewriter.java | 295 ++-----
.../io/sstable/format/SSTableReader.java | 100 ++-
.../io/sstable/format/big/BigTableWriter.java | 6 +-
.../cassandra/io/util/SequentialWriter.java | 2 +-
.../cassandra/metrics/ColumnFamilyMetrics.java | 18 +-
.../cassandra/streaming/StreamSession.java | 7 +-
.../cassandra/tools/StandaloneScrubber.java | 12 +-
.../cassandra/tools/StandaloneSplitter.java | 7 +-
.../cassandra/tools/StandaloneUpgrader.java | 6 +-
.../cassandra/utils/concurrent/Blocker.java | 63 ++
.../utils/concurrent/Transactional.java | 31 +-
.../db/compaction/LongCompactionsTest.java | 10 +-
test/unit/org/apache/cassandra/MockSchema.java | 167 ++++
test/unit/org/apache/cassandra/Util.java | 27 +-
.../org/apache/cassandra/db/KeyCacheTest.java | 3 +-
.../unit/org/apache/cassandra/db/ScrubTest.java | 58 +-
.../db/compaction/AntiCompactionTest.java | 51 +-
.../compaction/CompactionAwareWriterTest.java | 45 +-
.../DateTieredCompactionStrategyTest.java | 6 +-
.../cassandra/db/lifecycle/HelpersTest.java | 158 ++++
.../db/lifecycle/LifecycleTransactionTest.java | 412 ++++++++++
.../cassandra/db/lifecycle/TrackerTest.java | 342 ++++++++
.../apache/cassandra/db/lifecycle/ViewTest.java | 202 +++++
.../io/sstable/IndexSummaryManagerTest.java | 123 ++-
.../cassandra/io/sstable/SSTableReaderTest.java | 11 +-
.../io/sstable/SSTableRewriterTest.java | 250 +++---
61 files changed, 3902 insertions(+), 1711 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 8b59309..ca87385 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
2.2
+ * Extend Transactional API to sstable lifecycle management (CASSANDRA-8568)
* (cqlsh) Add support for native protocol 4 (CASSANDRA-9399)
* Ensure that UDF and UDAs are keyspace-isolated (CASSANDRA-9409)
* Revert CASSANDRA-7807 (tracing completion client notifications) (CASSANDRA-9429)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 4452db2..cc9b26a 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -34,6 +34,10 @@ import com.google.common.base.Throwables;
import com.google.common.collect.*;
import com.google.common.util.concurrent.*;
+import org.apache.cassandra.db.lifecycle.SSTableIntervalTree;
+import org.apache.cassandra.db.lifecycle.View;
+import org.apache.cassandra.db.lifecycle.Tracker;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.io.FSWriteError;
import org.json.simple.*;
import org.slf4j.Logger;
@@ -80,6 +84,8 @@ import org.apache.cassandra.utils.memory.MemtableAllocator;
import com.clearspring.analytics.stream.Counter;
+import static org.apache.cassandra.utils.Throwables.maybeFail;
+
public class ColumnFamilyStore implements ColumnFamilyStoreMBean
{
private static final Logger logger = LoggerFactory.getLogger(ColumnFamilyStore.class);
@@ -149,12 +155,12 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
/**
* Memtables and SSTables on disk for this column family.
*
- * We synchronize on the DataTracker to ensure isolation when we want to make sure
+ * We synchronize on the Tracker to ensure isolation when we want to make sure
* that the memtable we're acting on doesn't change out from under us. I.e., flush
* syncronizes on it to make sure it can submit on both executors atomically,
* so anyone else who wants to make sure flush doesn't interfere should as well.
*/
- private final DataTracker data;
+ private final Tracker data;
/* The read order, used to track accesses to off-heap memtable storage */
public final OpOrder readOrdering = new OpOrder();
@@ -288,13 +294,27 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
}
}
- private ColumnFamilyStore(Keyspace keyspace,
+ public ColumnFamilyStore(Keyspace keyspace,
+ String columnFamilyName,
+ IPartitioner partitioner,
+ int generation,
+ CFMetaData metadata,
+ Directories directories,
+ boolean loadSSTables)
+ {
+ this(keyspace, columnFamilyName, partitioner, generation, metadata, directories, loadSSTables, true);
+ }
+
+
+ @VisibleForTesting
+ public ColumnFamilyStore(Keyspace keyspace,
String columnFamilyName,
IPartitioner partitioner,
int generation,
CFMetaData metadata,
Directories directories,
- boolean loadSSTables)
+ boolean loadSSTables,
+ boolean registerBookkeeping)
{
assert metadata != null : "null metadata for " + keyspace + ":" + columnFamilyName;
@@ -315,7 +335,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
logger.info("Initializing {}.{}", keyspace.getName(), name);
// scan for sstables corresponding to this cf and load them
- data = new DataTracker(this, loadSSTables);
+ data = new Tracker(this, loadSSTables);
if (data.loadsstables)
{
@@ -343,46 +363,59 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
indexManager.addIndexedColumn(info);
}
- // register the mbean
- String type = this.partitioner instanceof LocalPartitioner ? "IndexColumnFamilies" : "ColumnFamilies";
- mbeanName = "org.apache.cassandra.db:type=" + type + ",keyspace=" + this.keyspace.getName() + ",columnfamily=" + name;
- try
- {
- MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
- ObjectName nameObj = new ObjectName(mbeanName);
- mbs.registerMBean(this, nameObj);
- }
- catch (Exception e)
- {
- throw new RuntimeException(e);
- }
- logger.debug("retryPolicy for {} is {}", name, this.metadata.getSpeculativeRetry());
- latencyCalculator = ScheduledExecutors.optionalTasks.scheduleWithFixedDelay(new Runnable()
+ if (registerBookkeeping)
{
- public void run()
+ // register the mbean
+ String type = this.partitioner instanceof LocalPartitioner ? "IndexColumnFamilies" : "ColumnFamilies";
+ mbeanName = "org.apache.cassandra.db:type=" + type + ",keyspace=" + this.keyspace.getName() + ",columnfamily=" + name;
+ try
+ {
+ MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+ ObjectName nameObj = new ObjectName(mbeanName);
+ mbs.registerMBean(this, nameObj);
+ }
+ catch (Exception e)
{
- SpeculativeRetry retryPolicy = ColumnFamilyStore.this.metadata.getSpeculativeRetry();
- switch (retryPolicy.type)
+ throw new RuntimeException(e);
+ }
+ logger.debug("retryPolicy for {} is {}", name, this.metadata.getSpeculativeRetry());
+ latencyCalculator = ScheduledExecutors.optionalTasks.scheduleWithFixedDelay(new Runnable()
+ {
+ public void run()
{
- case PERCENTILE:
- // get percentile in nanos
- sampleLatencyNanos = (long) (metric.coordinatorReadLatency.getSnapshot().getValue(retryPolicy.value) * 1000d);
- break;
- case CUSTOM:
- // convert to nanos, since configuration is in millisecond
- sampleLatencyNanos = (long) (retryPolicy.value * 1000d * 1000d);
- break;
- default:
- sampleLatencyNanos = Long.MAX_VALUE;
- break;
+ SpeculativeRetry retryPolicy = ColumnFamilyStore.this.metadata.getSpeculativeRetry();
+ switch (retryPolicy.type)
+ {
+ case PERCENTILE:
+ // get percentile in nanos
+ sampleLatencyNanos = (long) (metric.coordinatorReadLatency.getSnapshot().getValue(retryPolicy.value) * 1000d);
+ break;
+ case CUSTOM:
+ // convert to nanos, since configuration is in millisecond
+ sampleLatencyNanos = (long) (retryPolicy.value * 1000d * 1000d);
+ break;
+ default:
+ sampleLatencyNanos = Long.MAX_VALUE;
+ break;
+ }
}
- }
- }, DatabaseDescriptor.getReadRpcTimeout(), DatabaseDescriptor.getReadRpcTimeout(), TimeUnit.MILLISECONDS);
+ }, DatabaseDescriptor.getReadRpcTimeout(), DatabaseDescriptor.getReadRpcTimeout(), TimeUnit.MILLISECONDS);
+ }
+ else
+ {
+ latencyCalculator = ScheduledExecutors.optionalTasks.schedule(Runnables.doNothing(), 0, TimeUnit.NANOSECONDS);
+ mbeanName = null;
+ }
}
/** call when dropping or renaming a CF. Performs mbean housekeeping and invalidates CFS to other operations */
public void invalidate()
{
+ invalidate(true);
+ }
+
+ public void invalidate(boolean expectMBean)
+ {
// disable and cancel in-progress compactions before invalidating
valid = false;
@@ -392,21 +425,24 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
}
catch (Exception e)
{
- JVMStabilityInspector.inspectThrowable(e);
- // this shouldn't block anything.
- logger.warn("Failed unregistering mbean: {}", mbeanName, e);
+ if (expectMBean)
+ {
+ JVMStabilityInspector.inspectThrowable(e);
+ // this shouldn't block anything.
+ logger.warn("Failed unregistering mbean: {}", mbeanName, e);
+ }
}
latencyCalculator.cancel(false);
SystemKeyspace.removeTruncationRecord(metadata.cfId);
- data.unreferenceSSTables();
+ data.dropSSTables();
indexManager.invalidate();
invalidateCaches();
}
/**
- * Removes every SSTable in the directory from the DataTracker's view.
+ * Removes every SSTable in the directory from the Tracker's view.
* @param directory the unreadable directory, possibly with SSTables in it, but not necessarily.
*/
void maybeRemoveUnreadableSSTables(File directory)
@@ -542,7 +578,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
}
/**
- * Replacing compacted sstables is atomic as far as observers of DataTracker are concerned, but not on the
+ * Replacing compacted sstables is atomic as far as observers of Tracker are concerned, but not on the
* filesystem: first the new sstables are renamed to "live" status (i.e., the tmp marker is removed), then
* their ancestors are removed.
*
@@ -826,7 +862,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
/*
* switchMemtable puts Memtable.getSortedContents on the writer executor. When the write is complete,
* we turn the writer into an SSTableReader and add it to ssTables where it is available for reads.
- * This method does not block except for synchronizing on DataTracker, but the Future it returns will
+ * This method does not block except for synchronizing on Tracker, but the Future it returns will
* not complete until the Memtable (and all prior Memtables) have been successfully flushed, and the CL
* marked clean up to the position owned by the Memtable.
*/
@@ -849,7 +885,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
// reclaiming includes that which we are GC-ing;
float onHeapRatio = 0, offHeapRatio = 0;
long onHeapTotal = 0, offHeapTotal = 0;
- Memtable memtable = getDataTracker().getView().getCurrentMemtable();
+ Memtable memtable = getTracker().getView().getCurrentMemtable();
onHeapRatio += memtable.getAllocator().onHeap().ownershipRatio();
offHeapRatio += memtable.getAllocator().offHeap().ownershipRatio();
onHeapTotal += memtable.getAllocator().onHeap().owns();
@@ -859,7 +895,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
{
if (index.getIndexCfs() != null)
{
- MemtableAllocator allocator = index.getIndexCfs().getDataTracker().getView().getCurrentMemtable().getAllocator();
+ MemtableAllocator allocator = index.getIndexCfs().getTracker().getView().getCurrentMemtable().getAllocator();
onHeapRatio += allocator.onHeap().ownershipRatio();
offHeapRatio += allocator.offHeap().ownershipRatio();
onHeapTotal += allocator.onHeap().owns();
@@ -984,7 +1020,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
}
/**
- * Should only be constructed/used from switchMemtable() or truncate(), with ownership of the DataTracker monitor.
+ * Should only be constructed/used from switchMemtable() or truncate(), with ownership of the Tracker monitor.
* In the constructor the current memtable(s) are swapped, and a barrier on outstanding writes is issued;
* when run by the flushWriter the barrier is waited on to ensure all outstanding writes have completed
* before all memtables are immediately written, and the CL is either immediately marked clean or, if
@@ -1117,7 +1153,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
// we take a reference to the current main memtable for the CF prior to snapping its ownership ratios
// to ensure we have some ordering guarantee for performing the switchMemtableIf(), i.e. we will only
// swap if the memtables we are measuring here haven't already been swapped by the time we try to swap them
- Memtable current = cfs.getDataTracker().getView().getCurrentMemtable();
+ Memtable current = cfs.getTracker().getView().getCurrentMemtable();
// find the total ownership ratio for the memtable and all SecondaryIndexes owned by this CF,
// both on- and off-heap, and select the largest of the two ratios to weight this CF
@@ -1129,7 +1165,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
{
if (index.getIndexCfs() != null)
{
- MemtableAllocator allocator = index.getIndexCfs().getDataTracker().getView().getCurrentMemtable().getAllocator();
+ MemtableAllocator allocator = index.getIndexCfs().getTracker().getView().getCurrentMemtable().getAllocator();
onHeap += allocator.onHeap().ownershipRatio();
offHeap += allocator.offHeap().ownershipRatio();
}
@@ -1278,7 +1314,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
if (!sstables.iterator().hasNext())
return ImmutableSet.of();
- DataTracker.SSTableIntervalTree tree = data.getView().intervalTree;
+ SSTableIntervalTree tree = data.getView().intervalTree;
Set<SSTableReader> results = null;
for (SSTableReader sstable : sstables)
@@ -1454,7 +1490,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
public void markObsolete(Collection<SSTableReader> sstables, OperationType compactionType)
{
assert !sstables.isEmpty();
- data.markObsolete(sstables, compactionType);
+ maybeFail(data.dropSSTables(Predicates.in(sstables), compactionType, null));
}
void replaceFlushed(Memtable memtable, SSTableReader sstable)
@@ -1473,7 +1509,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
/**
* Package protected for access from the CompactionManager.
*/
- public DataTracker getDataTracker()
+ public Tracker getTracker()
{
return data;
}
@@ -1485,7 +1521,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
public Set<SSTableReader> getUncompactingSSTables()
{
- return data.getUncompactingSSTables();
+ return data.getUncompacting();
}
public ColumnFamily getColumnFamily(DecoratedKey key,
@@ -1759,7 +1795,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
return repairedSSTables;
}
- public RefViewFragment selectAndReference(Function<DataTracker.View, List<SSTableReader>> filter)
+ public RefViewFragment selectAndReference(Function<View, List<SSTableReader>> filter)
{
while (true)
{
@@ -1770,9 +1806,9 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
}
}
- public ViewFragment select(Function<DataTracker.View, List<SSTableReader>> filter)
+ public ViewFragment select(Function<View, List<SSTableReader>> filter)
{
- DataTracker.View view = data.getView();
+ View view = data.getView();
List<SSTableReader> sstables = view.intervalTree.isEmpty()
? Collections.<SSTableReader>emptyList()
: filter.apply(view);
@@ -1784,12 +1820,12 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
* @return a ViewFragment containing the sstables and memtables that may need to be merged
* for the given @param key, according to the interval tree
*/
- public Function<DataTracker.View, List<SSTableReader>> viewFilter(final DecoratedKey key)
+ public Function<View, List<SSTableReader>> viewFilter(final DecoratedKey key)
{
assert !key.isMinimum();
- return new Function<DataTracker.View, List<SSTableReader>>()
+ return new Function<View, List<SSTableReader>>()
{
- public List<SSTableReader> apply(DataTracker.View view)
+ public List<SSTableReader> apply(View view)
{
return compactionStrategyWrapper.filterSSTablesForReads(view.intervalTree.search(key));
}
@@ -1800,17 +1836,43 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
* @return a ViewFragment containing the sstables and memtables that may need to be merged
* for rows within @param rowBounds, inclusive, according to the interval tree.
*/
- public Function<DataTracker.View, List<SSTableReader>> viewFilter(final AbstractBounds<RowPosition> rowBounds)
+ public Function<View, List<SSTableReader>> viewFilter(final AbstractBounds<RowPosition> rowBounds)
{
- return new Function<DataTracker.View, List<SSTableReader>>()
+ return new Function<View, List<SSTableReader>>()
{
- public List<SSTableReader> apply(DataTracker.View view)
+ public List<SSTableReader> apply(View view)
{
return compactionStrategyWrapper.filterSSTablesForReads(view.sstablesInBounds(rowBounds));
}
};
}
+ /**
+ * @return a ViewFragment containing the sstables and memtables that may need to be merged
+ * for rows for all of @param rowBoundsCollection, inclusive, according to the interval tree.
+ */
+ public Function<View, List<SSTableReader>> viewFilter(final Collection<AbstractBounds<RowPosition>> rowBoundsCollection, final boolean includeRepaired)
+ {
+ return new Function<View, List<SSTableReader>>()
+ {
+ public List<SSTableReader> apply(View view)
+ {
+ Set<SSTableReader> sstables = Sets.newHashSet();
+ for (AbstractBounds<RowPosition> rowBounds : rowBoundsCollection)
+ {
+ for (SSTableReader sstable : view.sstablesInBounds(rowBounds))
+ {
+ if (includeRepaired || !sstable.isRepaired())
+ sstables.add(sstable);
+ }
+ }
+
+ logger.debug("ViewFilter for {}/{} sstables", sstables.size(), getSSTables().size());
+ return ImmutableList.copyOf(sstables);
+ }
+ };
+ }
+
public List<String> getSSTablesForKey(String key)
{
DecoratedKey dk = partitioner.decorateKey(metadata.getKeyValidator().fromString(key));
@@ -2388,6 +2450,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
* thread safety. All we do is wipe the sstable containers clean, while leaving the actual
* data files present on disk. (This allows tests to easily call loadNewSSTables on them.)
*/
+ @VisibleForTesting
public void clearUnsafe()
{
for (final ColumnFamilyStore cfs : concatWithIndexes())
@@ -2396,7 +2459,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
{
public Void call()
{
- cfs.data.init();
+ cfs.data.reset();
return null;
}
}, true);
@@ -2489,7 +2552,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
// doublecheck that we finished, instead of timing out
for (ColumnFamilyStore cfs : selfWithIndexes)
{
- if (!cfs.getDataTracker().getCompacting().isEmpty())
+ if (!cfs.getTracker().getCompacting().isEmpty())
{
logger.warn("Unable to cancel in-progress compactions for {}. Perhaps there is an unusually large row in progress somewhere, or the system is simply overloaded.", metadata.cfName);
return null;
@@ -2515,19 +2578,19 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
}
}
- public Iterable<SSTableReader> markAllCompacting()
+ public LifecycleTransaction markAllCompacting(final OperationType operationType)
{
- Callable<Iterable<SSTableReader>> callable = new Callable<Iterable<SSTableReader>>()
+ Callable<LifecycleTransaction> callable = new Callable<LifecycleTransaction>()
{
- public Iterable<SSTableReader> call() throws Exception
+ public LifecycleTransaction call() throws Exception
{
assert data.getCompacting().isEmpty() : data.getCompacting();
Collection<SSTableReader> sstables = Lists.newArrayList(AbstractCompactionStrategy.filterSuspectSSTables(getSSTables()));
if (Iterables.isEmpty(sstables))
- return Collections.emptyList();
- boolean success = data.markCompacting(sstables);
- assert success : "something marked things compacting while compactions are disabled";
- return sstables;
+ return null;
+ LifecycleTransaction modifier = data.tryModify(sstables, operationType);
+ assert modifier != null: "something marked things compacting while compactions are disabled";
+ return modifier;
}
};
@@ -2634,12 +2697,23 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
public int getMeanColumns()
{
- return data.getMeanColumns();
+ long sum = 0;
+ long count = 0;
+ for (SSTableReader sstable : getSSTables())
+ {
+ long n = sstable.getEstimatedColumnCount().count();
+ sum += sstable.getEstimatedColumnCount().mean() * n;
+ count += n;
+ }
+ return count > 0 ? (int) (sum / count) : 0;
}
public long estimateKeys()
{
- return data.estimatedKeys();
+ long n = 0;
+ for (SSTableReader sstable : getSSTables())
+ n += sstable.estimatedKeys();
+ return n;
}
/** true if this CFS contains secondary index data */
@@ -2703,18 +2777,10 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
}
}
- /**
- * Returns the creation time of the oldest memtable not fully flushed yet.
- */
- public long oldestUnflushedMemtable()
- {
- return data.getView().getOldestMemtable().creationTime();
- }
-
public boolean isEmpty()
{
- DataTracker.View view = data.getView();
- return view.sstables.isEmpty() && view.getCurrentMemtable().getOperations() == 0 && view.getCurrentMemtable() == view.getOldestMemtable();
+ View view = data.getView();
+ return view.sstables.isEmpty() && view.getCurrentMemtable().getOperations() == 0 && view.liveMemtables.size() <= 1 && view.flushingMemtables.size() == 0;
}
private boolean isRowCacheEnabled()
@@ -2753,7 +2819,16 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
public double getDroppableTombstoneRatio()
{
- return getDataTracker().getDroppableTombstoneRatio();
+ double allDroppable = 0;
+ long allColumns = 0;
+ int localTime = (int)(System.currentTimeMillis()/1000);
+
+ for (SSTableReader sstable : getSSTables())
+ {
+ allDroppable += sstable.getDroppableTombstonesBefore(localTime - sstable.metadata.getGcGraceSeconds());
+ allColumns += sstable.getEstimatedColumnCount().mean() * sstable.getEstimatedColumnCount().count();
+ }
+ return allColumns > 0 ? allDroppable / allColumns : 0;
}
public long trueSnapshotsSize()
@@ -2770,9 +2845,9 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
// returns the "canonical" version of any current sstable, i.e. if an sstable is being replaced and is only partially
// visible to reads, this sstable will be returned as its original entirety, and its replacement will not be returned
// (even if it completely replaces it)
- public static final Function<DataTracker.View, List<SSTableReader>> CANONICAL_SSTABLES = new Function<DataTracker.View, List<SSTableReader>>()
+ public static final Function<View, List<SSTableReader>> CANONICAL_SSTABLES = new Function<View, List<SSTableReader>>()
{
- public List<SSTableReader> apply(DataTracker.View view)
+ public List<SSTableReader> apply(View view)
{
List<SSTableReader> sstables = new ArrayList<>();
for (SSTableReader sstable : view.compacting)
@@ -2785,9 +2860,9 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
}
};
- public static final Function<DataTracker.View, List<SSTableReader>> UNREPAIRED_SSTABLES = new Function<DataTracker.View, List<SSTableReader>>()
+ public static final Function<View, List<SSTableReader>> UNREPAIRED_SSTABLES = new Function<View, List<SSTableReader>>()
{
- public List<SSTableReader> apply(DataTracker.View view)
+ public List<SSTableReader> apply(View view)
{
List<SSTableReader> sstables = new ArrayList<>();
for (SSTableReader sstable : CANONICAL_SSTABLES.apply(view))
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/DataTracker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DataTracker.java b/src/java/org/apache/cassandra/db/DataTracker.java
deleted file mode 100644
index 36f22c5..0000000
--- a/src/java/org/apache/cassandra/db/DataTracker.java
+++ /dev/null
@@ -1,793 +0,0 @@
-/*
- * 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.db;
-
-import java.io.File;
-import java.util.*;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.atomic.AtomicReference;
-
-import com.google.common.base.Predicate;
-import com.google.common.base.Predicates;
-import com.google.common.collect.*;
-import org.apache.cassandra.io.sstable.format.SSTableReader;
-import org.apache.cassandra.db.commitlog.ReplayPosition;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.compaction.OperationType;
-import org.apache.cassandra.dht.AbstractBounds;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.metrics.StorageMetrics;
-import org.apache.cassandra.notifications.*;
-import org.apache.cassandra.utils.Interval;
-import org.apache.cassandra.utils.IntervalTree;
-import org.apache.cassandra.utils.concurrent.OpOrder;
-import org.apache.cassandra.utils.concurrent.Refs;
-
-public class DataTracker
-{
- private static final Logger logger = LoggerFactory.getLogger(DataTracker.class);
-
- public final Collection<INotificationConsumer> subscribers = new CopyOnWriteArrayList<>();
- public final ColumnFamilyStore cfstore;
- private final AtomicReference<View> view;
-
- // Indicates if it is safe to load the initial sstables (may not be true when running in
- //standalone processes meant to repair or upgrade sstables (e.g. standalone scrubber)
- public final boolean loadsstables;
-
- public DataTracker(ColumnFamilyStore cfstore, boolean loadsstables)
- {
- this.cfstore = cfstore;
- this.view = new AtomicReference<>();
- this.loadsstables = loadsstables;
- this.init();
- }
-
- // get the Memtable that the ordered writeOp should be directed to
- public Memtable getMemtableFor(OpOrder.Group opGroup, ReplayPosition replayPosition)
- {
- // since any new memtables appended to the list after we fetch it will be for operations started
- // after us, we can safely assume that we will always find the memtable that 'accepts' us;
- // if the barrier for any memtable is set whilst we are reading the list, it must accept us.
-
- // there may be multiple memtables in the list that would 'accept' us, however we only ever choose
- // the oldest such memtable, as accepts() only prevents us falling behind (i.e. ensures we don't
- // assign operations to a memtable that was retired/queued before we started)
- for (Memtable memtable : view.get().liveMemtables)
- {
- if (memtable.accepts(opGroup, replayPosition))
- return memtable;
- }
- throw new AssertionError(view.get().liveMemtables.toString());
- }
-
- public Set<SSTableReader> getSSTables()
- {
- return view.get().sstables;
- }
-
- public Set<SSTableReader> getUncompactingSSTables()
- {
- return view.get().nonCompactingSStables();
- }
-
- public Iterable<SSTableReader> getUncompactingSSTables(Iterable<SSTableReader> candidates)
- {
- final View v = view.get();
- return Iterables.filter(candidates, new Predicate<SSTableReader>()
- {
- public boolean apply(SSTableReader sstable)
- {
- return !v.compacting.contains(sstable);
- }
- });
- }
-
- public View getView()
- {
- return view.get();
- }
-
- /**
- * Switch the current memtable. This atomically appends a new memtable to the end of the list of active memtables,
- * returning the previously last memtable. It leaves the previous Memtable in the list of live memtables until
- * discarding(memtable) is called. These two methods must be synchronized/paired, i.e. m = switchMemtable
- * must be followed by discarding(m), they cannot be interleaved.
- *
- * @return the previously active memtable
- */
- public Memtable switchMemtable(boolean truncating)
- {
- Memtable newMemtable = new Memtable(cfstore);
- Memtable toFlushMemtable;
- View currentView, newView;
- do
- {
- currentView = view.get();
- toFlushMemtable = currentView.getCurrentMemtable();
- newView = currentView.switchMemtable(newMemtable);
- }
- while (!view.compareAndSet(currentView, newView));
-
- if (truncating)
- notifyRenewed(newMemtable);
-
- return toFlushMemtable;
- }
-
- public void markFlushing(Memtable memtable)
- {
- View currentView, newView;
- do
- {
- currentView = view.get();
- newView = currentView.markFlushing(memtable);
- }
- while (!view.compareAndSet(currentView, newView));
- }
-
- public void replaceFlushed(Memtable memtable, SSTableReader sstable)
- {
- // sstable may be null if we flushed batchlog and nothing needed to be retained
-
- if (!cfstore.isValid())
- {
- View currentView, newView;
- do
- {
- currentView = view.get();
- newView = currentView.replaceFlushed(memtable, sstable);
- if (sstable != null)
- newView = newView.replace(Arrays.asList(sstable), Collections.<SSTableReader>emptyList());
- }
- while (!view.compareAndSet(currentView, newView));
- return;
- }
-
- // back up before creating a new View (which makes the new one eligible for compaction)
- if (sstable != null)
- maybeIncrementallyBackup(sstable);
-
- View currentView, newView;
- do
- {
- currentView = view.get();
- newView = currentView.replaceFlushed(memtable, sstable);
- }
- while (!view.compareAndSet(currentView, newView));
-
- if (sstable != null)
- {
- addNewSSTablesSize(Arrays.asList(sstable));
- notifyAdded(sstable);
- }
- }
-
- public void maybeIncrementallyBackup(final SSTableReader sstable)
- {
- if (!DatabaseDescriptor.isIncrementalBackupsEnabled())
- return;
-
- File backupsDir = Directories.getBackupsDirectory(sstable.descriptor);
- sstable.createLinks(FileUtils.getCanonicalPath(backupsDir));
- }
-
- /**
- * @return true if we are able to mark the given @param sstables as compacted, before anyone else
- *
- * Note that we could acquire references on the marked sstables and release them in
- * unmarkCompacting, but since we will never call markObsolete on a sstable marked
- * as compacting (unless there is a serious bug), we can skip this.
- */
- public boolean markCompacting(Iterable<SSTableReader> sstables)
- {
- return markCompacting(sstables, false, false);
- }
-
- public boolean markCompacting(Iterable<SSTableReader> sstables, boolean newTables, boolean offline)
- {
- assert sstables != null && !Iterables.isEmpty(sstables);
- while (true)
- {
- final View currentView = view.get();
- if (Iterables.any(sstables, Predicates.in(currentView.compacting)))
- return false;
-
- Predicate<SSTableReader> live = new Predicate<SSTableReader>()
- {
- public boolean apply(SSTableReader sstable)
- {
- return currentView.sstablesMap.get(sstable) == sstable && !sstable.isMarkedCompacted();
- }
- };
- if (newTables)
- assert !Iterables.any(sstables, Predicates.in(currentView.sstables));
- else if (!offline && !Iterables.all(sstables, live))
- return false;
-
- View newView = currentView.markCompacting(sstables);
- if (view.compareAndSet(currentView, newView))
- return true;
- }
- }
-
- /**
- * Removes files from compacting status: this is different from 'markObsolete'
- * because it should be run regardless of whether a compaction succeeded.
- */
- public void unmarkCompacting(Iterable<SSTableReader> unmark)
- {
- View currentView, newView;
- do
- {
- currentView = view.get();
- newView = currentView.unmarkCompacting(unmark);
- }
- while (!view.compareAndSet(currentView, newView));
-
- if (!cfstore.isValid())
- {
- // when the CFS is invalidated, it will call unreferenceSSTables(). However, unreferenceSSTables only deals
- // with sstables that aren't currently being compacted. If there are ongoing compactions that finish or are
- // interrupted after the CFS is invalidated, those sstables need to be unreferenced as well, so we do that here.
- unreferenceSSTables();
- }
- }
-
- public void markObsolete(Collection<SSTableReader> sstables, OperationType compactionType)
- {
- removeSSTablesFromTracker(sstables);
- releaseReferences(sstables, false);
- notifySSTablesChanged(sstables, Collections.<SSTableReader>emptyList(), compactionType);
- }
-
- /**
- *
- * @param oldSSTables
- * @param allReplacements
- * @param compactionType
- */
- // note that this DOES NOT insert the replacement sstables, it only removes the old sstables and notifies any listeners
- // that they have been replaced by the provided sstables, which must have been performed by an earlier replaceReaders() call
- public void markCompactedSSTablesReplaced(Collection<SSTableReader> oldSSTables, Collection<SSTableReader> allReplacements, OperationType compactionType)
- {
- removeSSTablesFromTracker(oldSSTables);
- releaseReferences(oldSSTables, false);
- notifySSTablesChanged(oldSSTables, allReplacements, compactionType);
- addNewSSTablesSize(allReplacements);
- }
-
- public void addInitialSSTables(Collection<SSTableReader> sstables)
- {
- addSSTablesToTracker(sstables);
- // no notifications or backup necessary
- }
-
- public void addSSTables(Collection<SSTableReader> sstables)
- {
- addSSTablesToTracker(sstables);
- for (SSTableReader sstable : sstables)
- {
- maybeIncrementallyBackup(sstable);
- notifyAdded(sstable);
- }
- }
-
- /**
- * Replaces existing sstables with new instances, makes sure compaction strategies have the correct instance
- *
- * @param toReplace
- * @param replaceWith
- */
- public void replaceWithNewInstances(Collection<SSTableReader> toReplace, Collection<SSTableReader> replaceWith)
- {
- replaceReaders(toReplace, replaceWith, true);
- }
-
- /**
- * Adds the early opened files to the data tracker, but does not tell compaction strategies about it
- *
- * note that we dont track the live size of these sstables
- * @param toReplace
- * @param replaceWith
- */
- public void replaceEarlyOpenedFiles(Collection<SSTableReader> toReplace, Collection<SSTableReader> replaceWith)
- {
- for (SSTableReader s : toReplace)
- assert s.openReason == SSTableReader.OpenReason.EARLY;
- // note that we can replace an early opened file with a real one
- replaceReaders(toReplace, replaceWith, false);
- }
-
- /**
- * removes all sstables that are not busy compacting.
- */
- public void unreferenceSSTables()
- {
- Set<SSTableReader> notCompacting;
-
- View currentView, newView;
- do
- {
- currentView = view.get();
- if (!currentView.compacting.isEmpty())
- logger.error("Set of compacting sstables is non-empty when invalidating sstables {}", currentView.compacting);
- notCompacting = currentView.nonCompactingSStables();
- newView = currentView.replace(notCompacting, Collections.<SSTableReader>emptySet());
- }
- while (!view.compareAndSet(currentView, newView));
-
- if (notCompacting.isEmpty())
- {
- // notifySSTablesChanged -> LeveledManifest.promote doesn't like a no-op "promotion"
- return;
- }
- notifySSTablesChanged(notCompacting, Collections.<SSTableReader>emptySet(), OperationType.UNKNOWN);
- removeOldSSTablesSize(notCompacting);
- releaseReferences(notCompacting, true);
- }
-
- /**
- * Removes every SSTable in the directory from the DataTracker's view.
- * @param directory the unreadable directory, possibly with SSTables in it, but not necessarily.
- */
- void removeUnreadableSSTables(File directory)
- {
- View currentView, newView;
- Set<SSTableReader> remaining = new HashSet<>();
- do
- {
- currentView = view.get();
- for (SSTableReader r : currentView.nonCompactingSStables())
- if (!r.descriptor.directory.equals(directory))
- remaining.add(r);
-
- if (remaining.size() == currentView.nonCompactingSStables().size())
- return;
-
- newView = currentView.replace(currentView.sstables, remaining);
- }
- while (!view.compareAndSet(currentView, newView));
- for (SSTableReader sstable : currentView.sstables)
- if (!remaining.contains(sstable))
- sstable.selfRef().release();
- notifySSTablesChanged(remaining, Collections.<SSTableReader>emptySet(), OperationType.UNKNOWN);
- }
-
- /** (Re)initializes the tracker, purging all references. */
- void init()
- {
- view.set(new View(
- ImmutableList.of(new Memtable(cfstore)),
- ImmutableList.<Memtable>of(),
- Collections.<SSTableReader, SSTableReader>emptyMap(),
- Collections.<SSTableReader>emptySet(),
- Collections.<SSTableReader>emptySet(),
- SSTableIntervalTree.empty()));
- }
-
- /**
- * A special kind of replacement for SSTableReaders that were cloned with a new index summary sampling level (see
- * SSTableReader.cloneWithNewSummarySamplingLevel and CASSANDRA-5519). This does not mark the old reader
- * as compacted.
- * @param oldSSTables replaced readers
- * @param newSSTables replacement readers
- */
- private void replaceReaders(Collection<SSTableReader> oldSSTables, Collection<SSTableReader> newSSTables, boolean notify)
- {
- View currentView, newView;
- do
- {
- currentView = view.get();
- newView = currentView.replace(oldSSTables, newSSTables);
- }
- while (!view.compareAndSet(currentView, newView));
-
- if (!oldSSTables.isEmpty() && notify)
- notifySSTablesChanged(oldSSTables, newSSTables, OperationType.UNKNOWN);
-
- for (SSTableReader sstable : newSSTables)
- sstable.setTrackedBy(this);
-
- Refs.release(Refs.selfRefs(oldSSTables));
- }
-
- private void removeSSTablesFromTracker(Collection<SSTableReader> oldSSTables)
- {
- View currentView, newView;
- do
- {
- currentView = view.get();
- newView = currentView.replace(oldSSTables, Collections.<SSTableReader>emptyList());
- }
- while (!view.compareAndSet(currentView, newView));
- removeOldSSTablesSize(oldSSTables);
- }
-
- private void addSSTablesToTracker(Collection<SSTableReader> sstables)
- {
- View currentView, newView;
- do
- {
- currentView = view.get();
- newView = currentView.replace(Collections.<SSTableReader>emptyList(), sstables);
- }
- while (!view.compareAndSet(currentView, newView));
- addNewSSTablesSize(sstables);
- }
-
- private void addNewSSTablesSize(Iterable<SSTableReader> newSSTables)
- {
- for (SSTableReader sstable : newSSTables)
- {
- if (logger.isDebugEnabled())
- logger.debug(String.format("adding %s to list of files tracked for %s.%s",
- sstable.descriptor, cfstore.keyspace.getName(), cfstore.name));
- long size = sstable.bytesOnDisk();
- StorageMetrics.load.inc(size);
- cfstore.metric.liveDiskSpaceUsed.inc(size);
- cfstore.metric.totalDiskSpaceUsed.inc(size);
- sstable.setTrackedBy(this);
- }
- }
-
- private void removeOldSSTablesSize(Iterable<SSTableReader> oldSSTables)
- {
- for (SSTableReader sstable : oldSSTables)
- {
- if (logger.isDebugEnabled())
- logger.debug(String.format("removing %s from list of files tracked for %s.%s",
- sstable.descriptor, cfstore.keyspace.getName(), cfstore.name));
- long size = sstable.bytesOnDisk();
- StorageMetrics.load.dec(size);
- cfstore.metric.liveDiskSpaceUsed.dec(size);
- }
- }
-
- private void releaseReferences(Iterable<SSTableReader> oldSSTables, boolean tolerateCompacted)
- {
- for (SSTableReader sstable : oldSSTables)
- {
- boolean firstToCompact = sstable.markObsolete();
- assert tolerateCompacted || firstToCompact : sstable + " was already marked compacted";
- sstable.selfRef().release();
- }
- }
-
- public void spaceReclaimed(long size)
- {
- cfstore.metric.totalDiskSpaceUsed.dec(size);
- }
-
- public long estimatedKeys()
- {
- long n = 0;
- for (SSTableReader sstable : getSSTables())
- n += sstable.estimatedKeys();
- return n;
- }
-
- public int getMeanColumns()
- {
- long sum = 0;
- long count = 0;
- for (SSTableReader sstable : getSSTables())
- {
- long n = sstable.getEstimatedColumnCount().count();
- sum += sstable.getEstimatedColumnCount().mean() * n;
- count += n;
- }
- return count > 0 ? (int) (sum / count) : 0;
- }
-
- public double getDroppableTombstoneRatio()
- {
- double allDroppable = 0;
- long allColumns = 0;
- int localTime = (int)(System.currentTimeMillis()/1000);
-
- for (SSTableReader sstable : getSSTables())
- {
- allDroppable += sstable.getDroppableTombstonesBefore(localTime - sstable.metadata.getGcGraceSeconds());
- allColumns += sstable.getEstimatedColumnCount().mean() * sstable.getEstimatedColumnCount().count();
- }
- return allColumns > 0 ? allDroppable / allColumns : 0;
- }
-
- public void notifySSTablesChanged(Collection<SSTableReader> removed, Collection<SSTableReader> added, OperationType compactionType)
- {
- INotification notification = new SSTableListChangedNotification(added, removed, compactionType);
- for (INotificationConsumer subscriber : subscribers)
- subscriber.handleNotification(notification, this);
- }
-
- public void notifyAdded(SSTableReader added)
- {
- INotification notification = new SSTableAddedNotification(added);
- for (INotificationConsumer subscriber : subscribers)
- subscriber.handleNotification(notification, this);
- }
-
- public void notifySSTableRepairedStatusChanged(Collection<SSTableReader> repairStatusesChanged)
- {
- INotification notification = new SSTableRepairStatusChanged(repairStatusesChanged);
- for (INotificationConsumer subscriber : subscribers)
- subscriber.handleNotification(notification, this);
-
- }
-
- public void notifyDeleting(SSTableReader deleting)
- {
- INotification notification = new SSTableDeletingNotification(deleting);
- for (INotificationConsumer subscriber : subscribers)
- subscriber.handleNotification(notification, this);
- }
-
- public void notifyRenewed(Memtable renewed)
- {
- INotification notification = new MemtableRenewedNotification(renewed);
- for (INotificationConsumer subscriber : subscribers)
- subscriber.handleNotification(notification, this);
- }
-
- public void notifyTruncated(long truncatedAt)
- {
- INotification notification = new TruncationNotification(truncatedAt);
- for (INotificationConsumer subscriber : subscribers)
- subscriber.handleNotification(notification, this);
- }
-
- public void subscribe(INotificationConsumer consumer)
- {
- subscribers.add(consumer);
- }
-
- public void unsubscribe(INotificationConsumer consumer)
- {
- subscribers.remove(consumer);
- }
-
- public static SSTableIntervalTree buildIntervalTree(Iterable<SSTableReader> sstables)
- {
- return new SSTableIntervalTree(buildIntervals(sstables));
- }
-
- public static List<Interval<RowPosition, SSTableReader>> buildIntervals(Iterable<SSTableReader> sstables)
- {
- List<Interval<RowPosition, SSTableReader>> intervals = new ArrayList<>(Iterables.size(sstables));
- for (SSTableReader sstable : sstables)
- intervals.add(Interval.<RowPosition, SSTableReader>create(sstable.first, sstable.last, sstable));
- return intervals;
- }
-
- public Set<SSTableReader> getCompacting()
- {
- return getView().compacting;
- }
-
- public static class SSTableIntervalTree extends IntervalTree<RowPosition, SSTableReader, Interval<RowPosition, SSTableReader>>
- {
- private static final SSTableIntervalTree EMPTY = new SSTableIntervalTree(null);
-
- private SSTableIntervalTree(Collection<Interval<RowPosition, SSTableReader>> intervals)
- {
- super(intervals);
- }
-
- public static SSTableIntervalTree empty()
- {
- return EMPTY;
- }
- }
-
- /**
- * An immutable structure holding the current memtable, the memtables pending
- * flush, the sstables for a column family, and the sstables that are active
- * in compaction (a subset of the sstables).
- */
- public static class View
- {
- /**
- * ordinarily a list of size 1, but when preparing to flush will contain both the memtable we will flush
- * and the new replacement memtable, until all outstanding write operations on the old table complete.
- * The last item in the list is always the "current" memtable.
- */
- private final List<Memtable> liveMemtables;
- /**
- * contains all memtables that are no longer referenced for writing and are queued for / in the process of being
- * flushed. In chronologically ascending order.
- */
- private final List<Memtable> flushingMemtables;
- public final Set<SSTableReader> compacting;
- public final Set<SSTableReader> sstables;
- // we use a Map here so that we can easily perform identity checks as well as equality checks.
- // When marking compacting, we now indicate if we expect the sstables to be present (by default we do),
- // and we then check that not only are they all present in the live set, but that the exact instance present is
- // the one we made our decision to compact against.
- public final Map<SSTableReader, SSTableReader> sstablesMap;
-
- // all sstables that are still in the live set, but have been completely shadowed by a replacement sstable
- public final Set<SSTableReader> shadowed;
- public final SSTableIntervalTree intervalTree;
-
- View(List<Memtable> liveMemtables, List<Memtable> flushingMemtables, Map<SSTableReader, SSTableReader> sstables, Set<SSTableReader> compacting, Set<SSTableReader> shadowed, SSTableIntervalTree intervalTree)
- {
- this.shadowed = shadowed;
- assert liveMemtables != null;
- assert flushingMemtables != null;
- assert sstables != null;
- assert compacting != null;
- assert intervalTree != null;
-
- this.liveMemtables = liveMemtables;
- this.flushingMemtables = flushingMemtables;
-
- this.sstablesMap = sstables;
- this.sstables = sstablesMap.keySet();
- this.compacting = compacting;
- this.intervalTree = intervalTree;
- }
-
- public Memtable getOldestMemtable()
- {
- if (!flushingMemtables.isEmpty())
- return flushingMemtables.get(0);
- return liveMemtables.get(0);
- }
-
- public Memtable getCurrentMemtable()
- {
- return liveMemtables.get(liveMemtables.size() - 1);
- }
-
- public Iterable<Memtable> getMemtablesPendingFlush()
- {
- if (liveMemtables.size() == 1)
- return flushingMemtables;
- return Iterables.concat(liveMemtables.subList(0, 1), flushingMemtables);
- }
-
- /**
- * @return the active memtable and all the memtables that are pending flush.
- */
- public Iterable<Memtable> getAllMemtables()
- {
- return Iterables.concat(flushingMemtables, liveMemtables);
- }
-
- public Sets.SetView<SSTableReader> nonCompactingSStables()
- {
- return Sets.difference(ImmutableSet.copyOf(sstables), compacting);
- }
-
- View switchMemtable(Memtable newMemtable)
- {
- List<Memtable> newLiveMemtables = ImmutableList.<Memtable>builder().addAll(liveMemtables).add(newMemtable).build();
- return new View(newLiveMemtables, flushingMemtables, sstablesMap, compacting, shadowed, intervalTree);
- }
-
- View markFlushing(Memtable toFlushMemtable)
- {
- List<Memtable> live = liveMemtables, flushing = flushingMemtables;
-
- // since we can have multiple flushes queued, we may occasionally race and start a flush out of order,
- // so must locate it in the list to remove, rather than just removing from the beginning
- int i = live.indexOf(toFlushMemtable);
- assert i < live.size() - 1;
- List<Memtable> newLive = ImmutableList.<Memtable>builder()
- .addAll(live.subList(0, i))
- .addAll(live.subList(i + 1, live.size()))
- .build();
-
- // similarly, if we out-of-order markFlushing once, we may afterwards need to insert a memtable into the
- // flushing list in a position other than the end, though this will be rare
- i = flushing.size();
- while (i > 0 && flushing.get(i - 1).creationTime() > toFlushMemtable.creationTime())
- i--;
- List<Memtable> newFlushing = ImmutableList.<Memtable>builder()
- .addAll(flushing.subList(0, i))
- .add(toFlushMemtable)
- .addAll(flushing.subList(i, flushing.size()))
- .build();
-
- return new View(newLive, newFlushing, sstablesMap, compacting, shadowed, intervalTree);
- }
-
- View replaceFlushed(Memtable flushedMemtable, SSTableReader newSSTable)
- {
- int index = flushingMemtables.indexOf(flushedMemtable);
- List<Memtable> newQueuedMemtables = ImmutableList.<Memtable>builder()
- .addAll(flushingMemtables.subList(0, index))
- .addAll(flushingMemtables.subList(index + 1, flushingMemtables.size()))
- .build();
- Map<SSTableReader, SSTableReader> newSSTables = sstablesMap;
- SSTableIntervalTree intervalTree = this.intervalTree;
- if (newSSTable != null)
- {
- assert !sstables.contains(newSSTable);
- assert !shadowed.contains(newSSTable);
- newSSTables = ImmutableMap.<SSTableReader, SSTableReader>builder()
- .putAll(sstablesMap).put(newSSTable, newSSTable).build();
- intervalTree = buildIntervalTree(newSSTables.keySet());
- }
- return new View(liveMemtables, newQueuedMemtables, newSSTables, compacting, shadowed, intervalTree);
- }
-
- View replace(Collection<SSTableReader> oldSSTables, Iterable<SSTableReader> replacements)
- {
- ImmutableSet<SSTableReader> oldSet = ImmutableSet.copyOf(oldSSTables);
- int newSSTablesSize = shadowed.size() + sstables.size() - oldSSTables.size() + Iterables.size(replacements);
- assert newSSTablesSize >= Iterables.size(replacements) : String.format("Incoherent new size %d replacing %s by %s in %s", newSSTablesSize, oldSSTables, replacements, this);
- Map<SSTableReader, SSTableReader> newSSTables = new HashMap<>(newSSTablesSize);
- Set<SSTableReader> newShadowed = new HashSet<>(shadowed.size());
-
- for (SSTableReader sstable : sstables)
- if (!oldSet.contains(sstable))
- newSSTables.put(sstable, sstable);
-
- for (SSTableReader sstable : shadowed)
- if (!oldSet.contains(sstable))
- newShadowed.add(sstable);
-
- for (SSTableReader replacement : replacements)
- {
- if (replacement.openReason == SSTableReader.OpenReason.SHADOWED)
- newShadowed.add(replacement);
- else
- newSSTables.put(replacement, replacement);
- }
-
- assert newSSTables.size() + newShadowed.size() == newSSTablesSize :
- String.format("Expecting new size of %d, got %d while replacing %s by %s in %s",
- newSSTablesSize, newSSTables.size() + newShadowed.size(), oldSSTables, replacements, this);
- newShadowed = ImmutableSet.copyOf(newShadowed);
- newSSTables = ImmutableMap.copyOf(newSSTables);
- SSTableIntervalTree intervalTree = buildIntervalTree(newSSTables.keySet());
- return new View(liveMemtables, flushingMemtables, newSSTables, compacting, newShadowed, intervalTree);
- }
-
- View markCompacting(Iterable<SSTableReader> tomark)
- {
- Set<SSTableReader> compactingNew = ImmutableSet.<SSTableReader>builder().addAll(compacting).addAll(tomark).build();
- return new View(liveMemtables, flushingMemtables, sstablesMap, compactingNew, shadowed, intervalTree);
- }
-
- View unmarkCompacting(Iterable<SSTableReader> tounmark)
- {
- Set<SSTableReader> compactingNew = ImmutableSet.copyOf(Sets.difference(compacting, ImmutableSet.copyOf(tounmark)));
- return new View(liveMemtables, flushingMemtables, sstablesMap, compactingNew, shadowed, intervalTree);
- }
-
- @Override
- public String toString()
- {
- return String.format("View(pending_count=%d, sstables=%s, compacting=%s)", liveMemtables.size() + flushingMemtables.size() - 1, sstables, compacting);
- }
-
- public List<SSTableReader> sstablesInBounds(AbstractBounds<RowPosition> rowBounds)
- {
- if (intervalTree.isEmpty())
- return Collections.emptyList();
- RowPosition stopInTree = rowBounds.right.isMinimum() ? intervalTree.max() : rowBounds.right;
- return intervalTree.search(Interval.<RowPosition, SSTableReader>create(rowBounds.left, stopInTree));
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/HintedHandOffManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/HintedHandOffManager.java b/src/java/org/apache/cassandra/db/HintedHandOffManager.java
index 589958e..df8820b 100644
--- a/src/java/org/apache/cassandra/db/HintedHandOffManager.java
+++ b/src/java/org/apache/cassandra/db/HintedHandOffManager.java
@@ -250,7 +250,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
protected synchronized void compact()
{
ArrayList<Descriptor> descriptors = new ArrayList<>();
- for (SSTable sstable : hintStore.getDataTracker().getUncompactingSSTables())
+ for (SSTable sstable : hintStore.getTracker().getUncompacting())
descriptors.add(sstable.descriptor);
if (descriptors.isEmpty())
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/Keyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java
index f30bdaa..1d86784 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -25,7 +25,6 @@ import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.Future;
import com.google.common.base.Function;
-import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -147,7 +146,7 @@ public class Keyspace
}
/**
- * Removes every SSTable in the directory from the appropriate DataTracker's view.
+ * Removes every SSTable in the directory from the appropriate Tracker's view.
* @param directory the unreadable directory, possibly with SSTables in it, but not necessarily.
*/
public static void removeUnreadableSSTables(File directory)
@@ -276,6 +275,18 @@ public class Keyspace
}
}
+ private Keyspace(KSMetaData metadata)
+ {
+ this.metadata = metadata;
+ createReplicationStrategy(metadata);
+ this.metric = new KeyspaceMetrics(this);
+ }
+
+ public static Keyspace mockKS(KSMetaData metadata)
+ {
+ return new Keyspace(metadata);
+ }
+
public void createReplicationStrategy(KSMetaData ksm)
{
replicationStrategy = AbstractReplicationStrategy.createReplicationStrategy(ksm.name,
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/Memtable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Memtable.java b/src/java/org/apache/cassandra/db/Memtable.java
index eab64ae..55b0bfe 100644
--- a/src/java/org/apache/cassandra/db/Memtable.java
+++ b/src/java/org/apache/cassandra/db/Memtable.java
@@ -27,6 +27,9 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.io.sstable.Descriptor;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.sstable.format.SSTableWriter;
@@ -46,7 +49,7 @@ import org.apache.cassandra.utils.*;
import org.apache.cassandra.utils.concurrent.OpOrder;
import org.apache.cassandra.utils.memory.*;
-public class Memtable
+public class Memtable implements Comparable<Memtable>
{
private static final Logger logger = LoggerFactory.getLogger(Memtable.class);
@@ -64,6 +67,11 @@ public class Memtable
// the "first" ReplayPosition owned by this Memtable; this is inaccurate, and only used as a convenience to prevent CLSM flushing wantonly
private final ReplayPosition minReplayPosition = CommitLog.instance.getContext();
+ public int compareTo(Memtable that)
+ {
+ return this.minReplayPosition.compareTo(that.minReplayPosition);
+ }
+
public static final class LastReplayPosition extends ReplayPosition
{
public LastReplayPosition(ReplayPosition copy) {
@@ -92,6 +100,15 @@ public class Memtable
this.cfs.scheduleFlush();
}
+ // ONLY to be used for testing, to create a mock Memtable
+ @VisibleForTesting
+ public Memtable(CFMetaData metadata)
+ {
+ this.initialComparator = metadata.comparator;
+ this.cfs = null;
+ this.allocator = null;
+ }
+
public MemtableAllocator getAllocator()
{
return allocator;
@@ -107,7 +124,8 @@ public class Memtable
return currentOperations.get();
}
- void setDiscarding(OpOrder.Barrier writeBarrier, AtomicReference<ReplayPosition> lastReplayPosition)
+ @VisibleForTesting
+ public void setDiscarding(OpOrder.Barrier writeBarrier, AtomicReference<ReplayPosition> lastReplayPosition)
{
assert this.writeBarrier == null;
this.lastReplayPosition = lastReplayPosition;
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
index 29826b8..38107c0 100644
--- a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
@@ -30,6 +30,7 @@ import org.slf4j.LoggerFactory;
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.Memtable;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.dht.Range;
import org.apache.cassandra.dht.Token;
import org.apache.cassandra.exceptions.ConfigurationException;
@@ -175,9 +176,9 @@ public abstract class AbstractCompactionStrategy
*/
public abstract AbstractCompactionTask getUserDefinedTask(Collection<SSTableReader> sstables, final int gcBefore);
- public AbstractCompactionTask getCompactionTask(Collection<SSTableReader> sstables, final int gcBefore, long maxSSTableBytes)
+ public AbstractCompactionTask getCompactionTask(LifecycleTransaction txn, final int gcBefore, long maxSSTableBytes)
{
- return new CompactionTask(cfs, sstables, gcBefore, false);
+ return new CompactionTask(cfs, txn, gcBefore, false);
}
/**
@@ -231,7 +232,7 @@ public abstract class AbstractCompactionStrategy
*/
public void replaceFlushed(Memtable memtable, SSTableReader sstable)
{
- cfs.getDataTracker().replaceFlushed(memtable, sstable);
+ cfs.getTracker().replaceFlushed(memtable, sstable);
if (sstable != null)
CompactionManager.instance.submitBackground(cfs);
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/AbstractCompactionTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionTask.java b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionTask.java
index ac646ef..3bf224e 100644
--- a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionTask.java
@@ -24,27 +24,28 @@ import org.apache.cassandra.db.compaction.CompactionManager.CompactionExecutorSt
import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.utils.WrappedRunnable;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
public abstract class AbstractCompactionTask extends WrappedRunnable
{
protected final ColumnFamilyStore cfs;
- protected Set<SSTableReader> sstables;
+ protected LifecycleTransaction transaction;
protected boolean isUserDefined;
protected OperationType compactionType;
/**
* @param cfs
- * @param sstables must be marked compacting
+ * @param transaction the modifying managing the status of the sstables we're replacing
*/
- public AbstractCompactionTask(ColumnFamilyStore cfs, Set<SSTableReader> sstables)
+ public AbstractCompactionTask(ColumnFamilyStore cfs, LifecycleTransaction transaction)
{
this.cfs = cfs;
- this.sstables = sstables;
+ this.transaction = transaction;
this.isUserDefined = false;
this.compactionType = OperationType.COMPACTION;
// enforce contract that caller should mark sstables compacting
- Set<SSTableReader> compacting = cfs.getDataTracker().getCompacting();
- for (SSTableReader sstable : sstables)
+ Set<SSTableReader> compacting = transaction.tracker.getCompacting();
+ for (SSTableReader sstable : transaction.originals())
assert compacting.contains(sstable) : sstable.getFilename() + " is not correctly marked compacting";
}
@@ -59,10 +60,10 @@ public abstract class AbstractCompactionTask extends WrappedRunnable
}
finally
{
- cfs.getDataTracker().unmarkCompacting(sstables);
+ transaction.close();
}
}
- public abstract CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables);
+ public abstract CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables);
protected abstract int executeInternal(CompactionExecutorStatsCollector collector);
@@ -80,6 +81,6 @@ public abstract class AbstractCompactionTask extends WrappedRunnable
public String toString()
{
- return "CompactionTask(" + sstables + ")";
+ return "CompactionTask(" + transaction + ")";
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/CompactionController.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionController.java b/src/java/org/apache/cassandra/db/compaction/CompactionController.java
index a49a3ea..2292e01 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionController.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionController.java
@@ -24,6 +24,8 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.lifecycle.SSTableIntervalTree;
+import org.apache.cassandra.db.lifecycle.Tracker;
import org.apache.cassandra.db.DecoratedKey;
import org.apache.cassandra.db.RowPosition;
import org.apache.cassandra.utils.AlwaysPresentFilter;
@@ -31,7 +33,7 @@ import org.apache.cassandra.utils.AlwaysPresentFilter;
import org.apache.cassandra.utils.OverlapIterator;
import org.apache.cassandra.utils.concurrent.Refs;
-import static org.apache.cassandra.db.DataTracker.buildIntervals;
+import static org.apache.cassandra.db.lifecycle.SSTableIntervalTree.buildIntervals;
/**
* Manage compaction options.
[2/6] cassandra git commit: Extend Transactional API to sstable
lifecycle management
Posted by be...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/test/unit/org/apache/cassandra/db/lifecycle/HelpersTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/HelpersTest.java b/test/unit/org/apache/cassandra/db/lifecycle/HelpersTest.java
new file mode 100644
index 0000000..d53a830
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/lifecycle/HelpersTest.java
@@ -0,0 +1,158 @@
+/*
+* 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.db.lifecycle;
+
+import java.util.Map;
+import java.util.Set;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+
+import org.junit.Test;
+
+import junit.framework.Assert;
+import org.apache.cassandra.MockSchema;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.big.BigTableReader;
+import org.apache.cassandra.utils.concurrent.Refs;
+
+public class HelpersTest
+{
+
+ static Set<Integer> a = set(1, 2, 3);
+ static Set<Integer> b = set(4, 5, 6);
+ static Set<Integer> c = set(7, 8, 9);
+ static Set<Integer> abc = set(1, 2, 3, 4, 5, 6, 7, 8, 9);
+
+ // this also tests orIn
+ @Test
+ public void testFilterIn()
+ {
+ check(Helpers.filterIn(abc, a), a);
+ check(Helpers.filterIn(abc, a, c), set(1, 2, 3, 7, 8, 9));
+ check(Helpers.filterIn(a, c), set());
+ }
+
+ // this also tests notIn
+ @Test
+ public void testFilterOut()
+ {
+ check(Helpers.filterOut(abc, a), set(4, 5, 6, 7, 8, 9));
+ check(Helpers.filterOut(abc, b), set(1, 2, 3, 7, 8, 9));
+ check(Helpers.filterOut(a, a), set());
+ }
+
+ @Test
+ public void testConcatUniq()
+ {
+ check(Helpers.concatUniq(a, b, a, c, b, a), abc);
+ }
+
+ @Test
+ public void testIdentityMap()
+ {
+ Integer one = new Integer(1);
+ Integer two = new Integer(2);
+ Integer three = new Integer(3);
+ Map<Integer, Integer> identity = Helpers.identityMap(set(one, two, three));
+ Assert.assertEquals(3, identity.size());
+ Assert.assertSame(one, identity.get(1));
+ Assert.assertSame(two, identity.get(2));
+ Assert.assertSame(three, identity.get(3));
+ }
+
+ @Test
+ public void testReplace()
+ {
+ boolean failure;
+ failure = false;
+ try
+ {
+ Helpers.replace(abc, a, c);
+ }
+ catch (AssertionError e)
+ {
+ failure = true;
+ }
+ Assert.assertTrue(failure);
+
+ failure = false;
+ try
+ {
+ Helpers.replace(a, abc, c);
+ }
+ catch (AssertionError e)
+ {
+ failure = true;
+ }
+ Assert.assertTrue(failure);
+
+ failure = false;
+ try
+ {
+ Map<Integer, Integer> notIdentity = ImmutableMap.of(1, new Integer(1), 2, 2, 3, 3);
+ Helpers.replace(notIdentity, a, b);
+ }
+ catch (AssertionError e)
+ {
+ failure = true;
+ }
+ Assert.assertTrue(failure);
+
+ // check it actually works when correct values provided
+ check(Helpers.replace(a, a, b), b);
+ }
+
+ private static Set<Integer> set(Integer ... contents)
+ {
+ return ImmutableSet.copyOf(contents);
+ }
+
+ private static void check(Iterable<Integer> check, Set<Integer> expected)
+ {
+ Assert.assertEquals(expected, ImmutableSet.copyOf(check));
+ }
+
+ @Test
+ public void testSetupDeletionNotification()
+ {
+ Iterable<SSTableReader> readers = Lists.newArrayList(MockSchema.sstable(1), MockSchema.sstable(2));
+ Throwable accumulate = Helpers.setReplaced(readers, null);
+ Assert.assertNull(accumulate);
+ for (SSTableReader reader : readers)
+ Assert.assertTrue(reader.isReplaced());
+ accumulate = Helpers.setReplaced(readers, null);
+ Assert.assertNotNull(accumulate);
+ }
+
+ @Test
+ public void testMarkObsolete()
+ {
+ Iterable<SSTableReader> readers = Lists.newArrayList(MockSchema.sstable(1), MockSchema.sstable(2));
+ Throwable accumulate = Helpers.markObsolete(readers, null);
+ Assert.assertNull(accumulate);
+ for (SSTableReader reader : readers)
+ Assert.assertTrue(reader.isMarkedCompacted());
+ accumulate = Helpers.markObsolete(readers, null);
+ Assert.assertNotNull(accumulate);
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java b/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
new file mode 100644
index 0000000..3153ef1
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
@@ -0,0 +1,412 @@
+/*
+* 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.db.lifecycle;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import junit.framework.Assert;
+import org.apache.cassandra.MockSchema;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction.ReaderState;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction.ReaderState.Action;
+import org.apache.cassandra.io.sstable.SSTableDeletingTask;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.concurrent.AbstractTransactionalTest;
+import org.apache.cassandra.utils.concurrent.Transactional.AbstractTransactional.State;
+
+import static com.google.common.base.Predicates.in;
+import static com.google.common.collect.ImmutableList.copyOf;
+import static com.google.common.collect.ImmutableList.of;
+import static com.google.common.collect.Iterables.all;
+import static com.google.common.collect.Iterables.concat;
+import static com.google.common.collect.Iterables.size;
+import static org.apache.cassandra.db.lifecycle.Helpers.idIn;
+import static org.apache.cassandra.db.lifecycle.Helpers.orIn;
+import static org.apache.cassandra.db.lifecycle.Helpers.select;
+
+public class LifecycleTransactionTest extends AbstractTransactionalTest
+{
+ private boolean incrementalBackups;
+
+ @Before
+ public void disableIncrementalBackup()
+ {
+ incrementalBackups = DatabaseDescriptor.isIncrementalBackupsEnabled();
+ DatabaseDescriptor.setIncrementalBackupsEnabled(false);
+ }
+ @After
+ public void restoreIncrementalBackup()
+ {
+ DatabaseDescriptor.setIncrementalBackupsEnabled(incrementalBackups);
+ }
+
+ @Test
+ public void testUpdates() // (including obsoletion)
+ {
+ Tracker tracker = new Tracker(null, false);
+ SSTableReader[] readers = readersArray(0, 3);
+ SSTableReader[] readers2 = readersArray(0, 4);
+ SSTableReader[] readers3 = readersArray(0, 4);
+ tracker.addInitialSSTables(copyOf(readers));
+ LifecycleTransaction txn = tracker.tryModify(copyOf(readers), OperationType.UNKNOWN);
+
+ txn.update(readers2[0], true);
+ txn.obsolete(readers[1]);
+
+ Assert.assertTrue(txn.isObsolete(readers[1]));
+ Assert.assertFalse(txn.isObsolete(readers[0]));
+
+ testBadUpdate(txn, readers2[0], true); // same reader && instances
+ testBadUpdate(txn, readers2[1], true); // staged obsolete; cannot update
+ testBadUpdate(txn, readers3[0], true); // same reader, diff instances
+ testBadUpdate(txn, readers2[2], false); // incorrectly declared original status
+ testBadUpdate(txn, readers2[3], true); // incorrectly declared original status
+
+ testBadObsolete(txn, readers[1]); // staged obsolete; cannot obsolete again
+ testBadObsolete(txn, readers2[0]); // staged update; cannot obsolete
+
+ txn.update(readers2[3], false);
+
+ Assert.assertEquals(3, tracker.getView().compacting.size());
+ txn.checkpoint();
+ Assert.assertTrue(txn.isObsolete(readers[1]));
+ Assert.assertFalse(txn.isObsolete(readers[0]));
+ Assert.assertEquals(4, tracker.getView().compacting.size());
+ Assert.assertEquals(3, tracker.getView().sstables.size());
+ Assert.assertEquals(3, size(txn.current()));
+ Assert.assertTrue(all(of(readers2[0], readers[2], readers2[3]), idIn(tracker.getView().sstablesMap)));
+ Assert.assertTrue(all(txn.current(), idIn(tracker.getView().sstablesMap)));
+
+ testBadObsolete(txn, readers[1]); // logged obsolete; cannot obsolete again
+ testBadObsolete(txn, readers2[2]); // never seen instance; cannot obsolete
+ testBadObsolete(txn, readers2[3]); // non-original; cannot obsolete
+ testBadUpdate(txn, readers3[1], true); // logged obsolete; cannot update
+ testBadUpdate(txn, readers2[0], true); // same instance as logged update
+
+ txn.update(readers3[0], true); // same reader as logged update, different instance
+ txn.checkpoint();
+
+ Assert.assertEquals(4, tracker.getView().compacting.size());
+ Assert.assertEquals(3, tracker.getView().sstables.size());
+ Assert.assertEquals(3, size(txn.current()));
+ Assert.assertTrue(all(of(readers3[0], readers[2], readers2[3]), idIn(tracker.getView().sstablesMap)));
+ Assert.assertTrue(all(txn.current(), idIn(tracker.getView().sstablesMap)));
+
+ testBadObsolete(txn, readers2[0]); // not current version of sstable
+
+ txn.obsoleteOriginals();
+ txn.checkpoint();
+ Assert.assertEquals(1, tracker.getView().sstables.size());
+ txn.obsoleteOriginals(); // should be no-op
+ txn.checkpoint();
+ Assert.assertEquals(1, tracker.getView().sstables.size());
+ Assert.assertEquals(4, tracker.getView().compacting.size());
+ }
+
+ @Test
+ public void testCancellation()
+ {
+ Tracker tracker = new Tracker(null, false);
+ List<SSTableReader> readers = readers(0, 3);
+ tracker.addInitialSSTables(readers);
+ LifecycleTransaction txn = tracker.tryModify(readers, OperationType.UNKNOWN);
+
+ SSTableReader cancel = readers.get(0);
+ SSTableReader update = readers(1, 2).get(0);
+ SSTableReader fresh = readers(3, 4).get(0);
+ SSTableReader notPresent = readers(4, 5).get(0);
+
+ txn.cancel(cancel);
+ txn.update(update, true);
+ txn.update(fresh, false);
+
+ testBadCancel(txn, cancel);
+ testBadCancel(txn, update);
+ testBadCancel(txn, fresh);
+ testBadCancel(txn, notPresent);
+ Assert.assertEquals(2, txn.originals().size());
+ Assert.assertEquals(2, tracker.getView().compacting.size());
+ Assert.assertTrue(all(readers.subList(1, 3), idIn(tracker.getView().compacting)));
+
+ txn.checkpoint();
+
+ testBadCancel(txn, cancel);
+ testBadCancel(txn, update);
+ testBadCancel(txn, fresh);
+ testBadCancel(txn, notPresent);
+ Assert.assertEquals(2, txn.originals().size());
+ Assert.assertEquals(3, tracker.getView().compacting.size());
+ Assert.assertEquals(3, size(txn.current()));
+ Assert.assertTrue(all(concat(readers.subList(1, 3), of(fresh)), idIn(tracker.getView().compacting)));
+
+ txn.cancel(readers.get(2));
+ Assert.assertEquals(1, txn.originals().size());
+ Assert.assertEquals(2, tracker.getView().compacting.size());
+ Assert.assertEquals(2, size(txn.current()));
+ Assert.assertTrue(all(of(readers.get(1), fresh), idIn(tracker.getView().compacting)));
+ }
+
+ @Test
+ public void testSplit()
+ {
+ Tracker tracker = new Tracker(null, false);
+ List<SSTableReader> readers = readers(0, 3);
+ tracker.addInitialSSTables(readers);
+ LifecycleTransaction txn = tracker.tryModify(readers, OperationType.UNKNOWN);
+ LifecycleTransaction txn2 = txn.split(readers.subList(0, 1));
+ Assert.assertEquals(2, txn.originals().size());
+ Assert.assertTrue(all(readers.subList(1, 3), in(txn.originals())));
+ Assert.assertEquals(1, txn2.originals().size());
+ Assert.assertTrue(all(readers.subList(0, 1), in(txn2.originals())));
+ txn.update(readers(1, 2).get(0), true);
+ boolean failed = false;
+ try
+ {
+ txn.split(readers.subList(2, 3));
+ }
+ catch (Throwable t)
+ {
+ failed = true;
+ }
+ Assert.assertTrue(failed);
+ }
+
+ private static void testBadUpdate(LifecycleTransaction txn, SSTableReader update, boolean original)
+ {
+ boolean failed = false;
+ try
+ {
+ txn.update(update, original);
+ }
+ catch (Throwable t)
+ {
+ failed = true;
+ }
+ Assert.assertTrue(failed);
+ }
+
+ private static void testBadObsolete(LifecycleTransaction txn, SSTableReader update)
+ {
+ boolean failed = false;
+ try
+ {
+ txn.obsolete(update);
+ }
+ catch (Throwable t)
+ {
+ failed = true;
+ }
+ Assert.assertTrue(failed);
+ }
+
+ private static void testBadCancel(LifecycleTransaction txn, SSTableReader cancel)
+ {
+ boolean failed = false;
+ try
+ {
+ txn.cancel(cancel);
+ }
+ catch (Throwable t)
+ {
+ failed = true;
+ }
+ Assert.assertTrue(failed);
+ }
+
+ protected TestableTransaction newTest()
+ {
+ SSTableDeletingTask.waitForDeletions();
+ SSTableReader.resetTidying();
+ return new TxnTest();
+ }
+
+ private static final class TxnTest extends TestableTransaction
+ {
+ final List<SSTableReader> originals;
+ final List<SSTableReader> untouchedOriginals;
+ final List<SSTableReader> loggedUpdate;
+ final List<SSTableReader> loggedObsolete;
+ final List<SSTableReader> stagedObsolete;
+ final List<SSTableReader> loggedNew;
+ final List<SSTableReader> stagedNew;
+ final Tracker tracker;
+ final LifecycleTransaction txn;
+
+ private static Tracker tracker(List<SSTableReader> readers)
+ {
+ Tracker tracker = new Tracker(MockSchema.cfs, false);
+ tracker.addInitialSSTables(readers);
+ return tracker;
+ }
+
+ private TxnTest()
+ {
+ this(readers(0, 8));
+ }
+
+ private TxnTest(List<SSTableReader> readers)
+ {
+ this(tracker(readers), readers);
+ }
+
+ private TxnTest(Tracker tracker, List<SSTableReader> readers)
+ {
+ this(tracker, readers, tracker.tryModify(readers, OperationType.UNKNOWN));
+ }
+
+ private TxnTest(Tracker tracker, List<SSTableReader> readers, LifecycleTransaction txn)
+ {
+ super(txn);
+ this.tracker = tracker;
+ this.originals = readers;
+ this.txn = txn;
+ update(txn, loggedUpdate = readers(0, 2), true);
+ obsolete(txn, loggedObsolete = readers.subList(2, 4));
+ update(txn, loggedNew = readers(8, 10), false);
+ txn.checkpoint();
+ update(txn, stagedNew = readers(10, 12), false);
+ obsolete(txn, stagedObsolete = copyOf(concat(loggedUpdate, originals.subList(4, 6))));
+ untouchedOriginals = originals.subList(6, 8);
+ }
+
+ private ReaderState state(SSTableReader reader, State state)
+ {
+ SSTableReader original = select(reader, originals);
+ boolean isOriginal = original != null;
+
+ switch (state)
+ {
+ case ABORTED:
+ {
+ return new ReaderState(Action.NONE, Action.NONE, original, original, isOriginal);
+ }
+
+ case READY_TO_COMMIT:
+ {
+ ReaderState prev = state(reader, State.IN_PROGRESS);
+ Action logged;
+ SSTableReader visible;
+ if (prev.staged == Action.NONE)
+ {
+ logged = prev.logged;
+ visible = prev.currentlyVisible;
+ }
+ else
+ {
+ logged = prev.staged;
+ visible = prev.nextVisible;
+ }
+ return new ReaderState(logged, Action.NONE, visible, visible, isOriginal);
+ }
+
+ case IN_PROGRESS:
+ {
+ Action logged = Action.get(loggedUpdate.contains(reader) || loggedNew.contains(reader), loggedObsolete.contains(reader));
+ Action staged = Action.get(stagedNew.contains(reader), stagedObsolete.contains(reader));
+ SSTableReader currentlyVisible = ReaderState.visible(reader, in(loggedObsolete), loggedNew, loggedUpdate, originals);
+ SSTableReader nextVisible = ReaderState.visible(reader, orIn(stagedObsolete, loggedObsolete), stagedNew, loggedNew, loggedUpdate, originals);
+ return new ReaderState(logged, staged, currentlyVisible, nextVisible, isOriginal);
+ }
+ }
+ throw new IllegalStateException();
+ }
+
+ private List<Pair<SSTableReader, ReaderState>> states(State state)
+ {
+ List<Pair<SSTableReader, ReaderState>> result = new ArrayList<>();
+ for (SSTableReader reader : concat(originals, loggedNew, stagedNew))
+ result.add(Pair.create(reader, state(reader, state)));
+ return result;
+ }
+
+ protected void doAssert(State state)
+ {
+ for (Pair<SSTableReader, ReaderState> pair : states(state))
+ {
+ SSTableReader reader = pair.left;
+ ReaderState readerState = pair.right;
+
+ Assert.assertEquals(readerState, txn.state(reader));
+ Assert.assertEquals(readerState.currentlyVisible, tracker.getView().sstablesMap.get(reader));
+ if (readerState.currentlyVisible == null && readerState.nextVisible == null && !readerState.original)
+ Assert.assertTrue(reader.selfRef().globalCount() == 0);
+ }
+ }
+
+ protected void assertInProgress() throws Exception
+ {
+ doAssert(State.IN_PROGRESS);
+ }
+
+ protected void assertPrepared() throws Exception
+ {
+ doAssert(State.READY_TO_COMMIT);
+ }
+
+ protected void assertAborted() throws Exception
+ {
+ doAssert(State.ABORTED);
+ Assert.assertEquals(0, tracker.getView().compacting.size());
+ Assert.assertEquals(8, tracker.getView().sstables.size());
+ for (SSTableReader reader : concat(loggedNew, stagedNew))
+ Assert.assertTrue(reader.selfRef().globalCount() == 0);
+ }
+
+ protected void assertCommitted() throws Exception
+ {
+ doAssert(State.READY_TO_COMMIT);
+ Assert.assertEquals(0, tracker.getView().compacting.size());
+ Assert.assertEquals(6, tracker.getView().sstables.size());
+ for (SSTableReader reader : concat(loggedObsolete, stagedObsolete))
+ Assert.assertTrue(reader.selfRef().globalCount() == 0);
+ }
+ }
+
+ private static SSTableReader[] readersArray(int lb, int ub)
+ {
+ return readers(lb, ub).toArray(new SSTableReader[0]);
+ }
+
+ private static List<SSTableReader> readers(int lb, int ub)
+ {
+ List<SSTableReader> readers = new ArrayList<>();
+ for (int i = lb ; i < ub ; i++)
+ readers.add(MockSchema.sstable(i, i, true));
+ return copyOf(readers);
+ }
+
+ private static void update(LifecycleTransaction txn, Iterable<SSTableReader> readers, boolean originals)
+ {
+ for (SSTableReader reader : readers)
+ txn.update(reader, originals);
+ }
+
+ private static void obsolete(LifecycleTransaction txn, Iterable<SSTableReader> readers)
+ {
+ for (SSTableReader reader : readers)
+ txn.obsolete(reader);
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
new file mode 100644
index 0000000..1eef7b0
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
@@ -0,0 +1,342 @@
+/*
+* 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.db.lifecycle;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+import javax.annotation.Nullable;
+
+import com.google.common.base.Function;
+import com.google.common.base.Predicate;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import org.junit.Test;
+
+import junit.framework.Assert;
+import org.apache.cassandra.MockSchema;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Memtable;
+import org.apache.cassandra.db.commitlog.ReplayPosition;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.io.sstable.SSTableDeletingTask;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.notifications.*;
+import org.apache.cassandra.utils.concurrent.OpOrder;
+
+import static com.google.common.collect.ImmutableSet.copyOf;
+import static java.util.Collections.singleton;
+
+public class TrackerTest
+{
+
+ private static final class MockListener implements INotificationConsumer
+ {
+ final boolean throwException;
+ final List<INotification> received = new ArrayList<>();
+ final List<Object> senders = new ArrayList<>();
+
+ private MockListener(boolean throwException)
+ {
+ this.throwException = throwException;
+ }
+
+ public void handleNotification(INotification notification, Object sender)
+ {
+ if (throwException)
+ throw new RuntimeException();
+ received.add(notification);
+ senders.add(sender);
+ }
+ }
+
+ @Test
+ public void testTryModify()
+ {
+ Tracker tracker = new Tracker(MockSchema.cfs, false);
+ List<SSTableReader> readers = ImmutableList.of(MockSchema.sstable(0), MockSchema.sstable(1), MockSchema.sstable(2));
+ tracker.addInitialSSTables(copyOf(readers));
+ try (LifecycleTransaction txn = tracker.tryModify(readers.get(0), OperationType.COMPACTION);)
+ {
+ Assert.assertNotNull(txn);
+ Assert.assertNull(tracker.tryModify(readers.get(0), OperationType.COMPACTION));
+ Assert.assertEquals(1, txn.originals().size());
+ Assert.assertTrue(txn.originals().contains(readers.get(0)));
+ }
+ try (LifecycleTransaction txn = tracker.tryModify(Collections.<SSTableReader>emptyList(), OperationType.COMPACTION);)
+ {
+ Assert.assertNotNull(txn);
+ Assert.assertEquals(0, txn.originals().size());
+ }
+ }
+
+ @Test
+ public void testApply()
+ {
+ final Tracker tracker = new Tracker(null, false);
+ final View resultView = ViewTest.fakeView(0, 0);
+ final AtomicInteger count = new AtomicInteger();
+ tracker.apply(new Predicate<View>()
+ {
+ public boolean apply(View view)
+ {
+ // confound the CAS by swapping the view, and check we retry
+ if (count.incrementAndGet() < 3)
+ tracker.view.set(ViewTest.fakeView(0, 0));
+ return true;
+ }
+ }, new Function<View, View>()
+ {
+ @Nullable
+ public View apply(View view)
+ {
+ return resultView;
+ }
+ });
+ Assert.assertEquals(3, count.get());
+ Assert.assertEquals(resultView, tracker.getView());
+
+ count.set(0);
+ // check that if the predicate returns false, we stop immediately and return null
+ Assert.assertNull(tracker.apply(new Predicate<View>()
+ {
+ public boolean apply(View view)
+ {
+ count.incrementAndGet();
+ return false;
+ }
+ }, null));
+ Assert.assertEquals(1, count.get());
+ Assert.assertEquals(resultView, tracker.getView());
+ }
+
+ @Test
+ public void testAddInitialSSTables()
+ {
+ ColumnFamilyStore cfs = MockSchema.newCFS();
+ Tracker tracker = new Tracker(cfs, false);
+ List<SSTableReader> readers = ImmutableList.of(MockSchema.sstable(0, 17), MockSchema.sstable(1, 121), MockSchema.sstable(2, 9));
+ tracker.addInitialSSTables(copyOf(readers));
+
+ Assert.assertEquals(3, tracker.view.get().sstables.size());
+
+ for (SSTableReader reader : readers)
+ Assert.assertTrue(reader.isDeleteNotificationSetup());
+
+ Assert.assertEquals(17 + 121 + 9, cfs.metric.liveDiskSpaceUsed.getCount());
+ }
+
+ @Test
+ public void testAddSSTables()
+ {
+ boolean backups = DatabaseDescriptor.isIncrementalBackupsEnabled();
+ DatabaseDescriptor.setIncrementalBackupsEnabled(false);
+ Tracker tracker = new Tracker(MockSchema.cfs, false);
+ MockListener listener = new MockListener(false);
+ tracker.subscribe(listener);
+ List<SSTableReader> readers = ImmutableList.of(MockSchema.sstable(0, 17), MockSchema.sstable(1, 121), MockSchema.sstable(2, 9));
+ tracker.addSSTables(copyOf(readers));
+
+ Assert.assertEquals(3, tracker.view.get().sstables.size());
+
+ for (SSTableReader reader : readers)
+ Assert.assertTrue(reader.isDeleteNotificationSetup());
+
+ Assert.assertEquals(17 + 121 + 9, MockSchema.cfs.metric.liveDiskSpaceUsed.getCount());
+ Assert.assertEquals(3, listener.senders.size());
+ Assert.assertEquals(tracker, listener.senders.get(0));
+ Assert.assertTrue(listener.received.get(0) instanceof SSTableAddedNotification);
+ DatabaseDescriptor.setIncrementalBackupsEnabled(backups);
+ }
+
+ @Test
+ public void testDropSSTables()
+ {
+ testDropSSTables(false);
+ SSTableDeletingTask.waitForDeletions();
+ testDropSSTables(true);
+ SSTableDeletingTask.waitForDeletions();
+ }
+
+ private void testDropSSTables(boolean invalidate)
+ {
+ ColumnFamilyStore cfs = MockSchema.newCFS();
+ Tracker tracker = cfs.getTracker();
+ MockListener listener = new MockListener(false);
+ tracker.subscribe(listener);
+ final List<SSTableReader> readers = ImmutableList.of(MockSchema.sstable(0, 9, true), MockSchema.sstable(1, 15, true), MockSchema.sstable(2, 71, true));
+ tracker.addInitialSSTables(copyOf(readers));
+ try (LifecycleTransaction txn = tracker.tryModify(readers.get(0), OperationType.COMPACTION);)
+ {
+ SSTableDeletingTask.pauseDeletions(true);
+ if (invalidate)
+ cfs.invalidate(false);
+ else
+ tracker.dropSSTables();
+ Assert.assertEquals(95, cfs.metric.totalDiskSpaceUsed.getCount());
+ Assert.assertEquals(9, cfs.metric.liveDiskSpaceUsed.getCount());
+ Assert.assertEquals(1, tracker.getView().sstables.size());
+ SSTableDeletingTask.pauseDeletions(false);
+ }
+ if (!invalidate)
+ {
+ Assert.assertEquals(1, tracker.getView().sstables.size());
+ Assert.assertEquals(readers.get(0), Iterables.getFirst(tracker.getView().sstables, null));
+ Assert.assertEquals(1, readers.get(0).selfRef().globalCount());
+ Assert.assertFalse(readers.get(0).isMarkedCompacted());
+ for (SSTableReader reader : readers.subList(1, 3))
+ {
+ Assert.assertEquals(0, reader.selfRef().globalCount());
+ Assert.assertTrue(reader.isMarkedCompacted());
+ }
+ Assert.assertNull(tracker.dropSSTables(new Predicate<SSTableReader>()
+ {
+ public boolean apply(SSTableReader reader)
+ {
+ return reader != readers.get(0);
+ }
+ }, OperationType.UNKNOWN, null));
+
+ Assert.assertEquals(1, tracker.getView().sstables.size());
+ Assert.assertEquals(1, listener.received.size());
+ Assert.assertEquals(tracker, listener.senders.get(0));
+ Assert.assertEquals(2, ((SSTableListChangedNotification) listener.received.get(0)).removed.size());
+ Assert.assertEquals(0, ((SSTableListChangedNotification) listener.received.get(0)).added.size());
+ Assert.assertEquals(9, cfs.metric.liveDiskSpaceUsed.getCount());
+ readers.get(0).selfRef().release();
+ }
+ else
+ {
+ Assert.assertEquals(0, tracker.getView().sstables.size());
+ Assert.assertEquals(0, cfs.metric.liveDiskSpaceUsed.getCount());
+ for (SSTableReader reader : readers)
+ Assert.assertTrue(reader.isMarkedCompacted());
+ }
+ }
+
+ @Test
+ public void testMemtableReplacement()
+ {
+ boolean backups = DatabaseDescriptor.isIncrementalBackupsEnabled();
+ DatabaseDescriptor.setIncrementalBackupsEnabled(false);
+ ColumnFamilyStore cfs = MockSchema.newCFS();
+ MockListener listener = new MockListener(false);
+ Tracker tracker = cfs.getTracker();
+ tracker.subscribe(listener);
+
+ Memtable prev1 = tracker.switchMemtable(true);
+ OpOrder.Group write1 = cfs.keyspace.writeOrder.getCurrent();
+ OpOrder.Barrier barrier1 = cfs.keyspace.writeOrder.newBarrier();
+ prev1.setDiscarding(barrier1, new AtomicReference<ReplayPosition>());
+ barrier1.issue();
+ Memtable prev2 = tracker.switchMemtable(false);
+ OpOrder.Group write2 = cfs.keyspace.writeOrder.getCurrent();
+ OpOrder.Barrier barrier2 = cfs.keyspace.writeOrder.newBarrier();
+ prev2.setDiscarding(barrier2, new AtomicReference<ReplayPosition>());
+ barrier2.issue();
+ Memtable cur = tracker.getView().getCurrentMemtable();
+ OpOrder.Group writecur = cfs.keyspace.writeOrder.getCurrent();
+ Assert.assertEquals(prev1, tracker.getMemtableFor(write1, ReplayPosition.NONE));
+ Assert.assertEquals(prev2, tracker.getMemtableFor(write2, ReplayPosition.NONE));
+ Assert.assertEquals(cur, tracker.getMemtableFor(writecur, ReplayPosition.NONE));
+ Assert.assertEquals(1, listener.received.size());
+ Assert.assertTrue(listener.received.get(0) instanceof MemtableRenewedNotification);
+ listener.received.clear();
+
+ tracker.markFlushing(prev2);
+ Assert.assertEquals(1, tracker.getView().flushingMemtables.size());
+ Assert.assertTrue(tracker.getView().flushingMemtables.contains(prev2));
+
+ tracker.markFlushing(prev1);
+ Assert.assertTrue(tracker.getView().flushingMemtables.contains(prev1));
+ Assert.assertEquals(2, tracker.getView().flushingMemtables.size());
+
+ tracker.replaceFlushed(prev1, null);
+ Assert.assertEquals(1, tracker.getView().flushingMemtables.size());
+ Assert.assertTrue(tracker.getView().flushingMemtables.contains(prev2));
+
+ SSTableReader reader = MockSchema.sstable(0, 10, false, cfs);
+ tracker.replaceFlushed(prev2, reader);
+ Assert.assertEquals(1, tracker.getView().sstables.size());
+ Assert.assertEquals(1, listener.received.size());
+ Assert.assertEquals(reader, ((SSTableAddedNotification) listener.received.get(0)).added);
+ listener.received.clear();
+ Assert.assertTrue(reader.isDeleteNotificationSetup());
+ Assert.assertEquals(10, cfs.metric.liveDiskSpaceUsed.getCount());
+
+ // test invalidated CFS
+ cfs = MockSchema.newCFS();
+ tracker = cfs.getTracker();
+ listener = new MockListener(false);
+ tracker.subscribe(listener);
+ prev1 = tracker.switchMemtable(false);
+ tracker.markFlushing(prev1);
+ reader = MockSchema.sstable(0, 10, true, cfs);
+ cfs.invalidate(false);
+ tracker.replaceFlushed(prev1, reader);
+ Assert.assertEquals(0, tracker.getView().sstables.size());
+ Assert.assertEquals(0, tracker.getView().flushingMemtables.size());
+ Assert.assertEquals(0, cfs.metric.liveDiskSpaceUsed.getCount());
+ Assert.assertEquals(reader, ((SSTableAddedNotification) listener.received.get(0)).added);
+ Assert.assertEquals(1, ((SSTableListChangedNotification) listener.received.get(1)).removed.size());
+ DatabaseDescriptor.setIncrementalBackupsEnabled(backups);
+ }
+
+ @Test
+ public void testNotifications()
+ {
+ SSTableReader r1 = MockSchema.sstable(0), r2 = MockSchema.sstable(1);
+ Tracker tracker = new Tracker(null, false);
+ MockListener listener = new MockListener(false);
+ tracker.subscribe(listener);
+ tracker.notifyAdded(r1);
+ Assert.assertEquals(r1, ((SSTableAddedNotification) listener.received.get(0)).added);
+ listener.received.clear();
+ tracker.notifyDeleting(r1);
+ Assert.assertEquals(r1, ((SSTableDeletingNotification) listener.received.get(0)).deleting);
+ listener.received.clear();
+ Assert.assertNull(tracker.notifySSTablesChanged(singleton(r1), singleton(r2), OperationType.COMPACTION, null));
+ Assert.assertEquals(singleton(r1), ((SSTableListChangedNotification) listener.received.get(0)).removed);
+ Assert.assertEquals(singleton(r2), ((SSTableListChangedNotification) listener.received.get(0)).added);
+ listener.received.clear();
+ tracker.notifySSTableRepairedStatusChanged(singleton(r1));
+ Assert.assertEquals(singleton(r1), ((SSTableRepairStatusChanged) listener.received.get(0)).sstable);
+ listener.received.clear();
+ Memtable memtable = MockSchema.memtable();
+ tracker.notifyRenewed(memtable);
+ Assert.assertEquals(memtable, ((MemtableRenewedNotification) listener.received.get(0)).renewed);
+ listener.received.clear();
+ tracker.unsubscribe(listener);
+ MockListener failListener = new MockListener(true);
+ tracker.subscribe(failListener);
+ tracker.subscribe(listener);
+ Assert.assertNotNull(tracker.notifyAdded(r1, null));
+ Assert.assertEquals(r1, ((SSTableAddedNotification) listener.received.get(0)).added);
+ listener.received.clear();
+ Assert.assertNotNull(tracker.notifySSTablesChanged(singleton(r1), singleton(r2), OperationType.COMPACTION, null));
+ Assert.assertEquals(singleton(r1), ((SSTableListChangedNotification) listener.received.get(0)).removed);
+ Assert.assertEquals(singleton(r2), ((SSTableListChangedNotification) listener.received.get(0)).added);
+ listener.received.clear();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java b/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java
new file mode 100644
index 0000000..811e025
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java
@@ -0,0 +1,202 @@
+/*
+* 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.db.lifecycle;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import com.google.common.base.Function;
+import com.google.common.base.Predicates;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import org.junit.Test;
+
+import junit.framework.Assert;
+import org.apache.cassandra.MockSchema;
+import org.apache.cassandra.db.Memtable;
+import org.apache.cassandra.db.RowPosition;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+
+import static com.google.common.collect.ImmutableSet.copyOf;
+import static com.google.common.collect.ImmutableSet.of;
+import static com.google.common.collect.Iterables.concat;
+import static org.apache.cassandra.db.lifecycle.Helpers.emptySet;
+
+public class ViewTest
+{
+
+ @Test
+ public void testSSTablesInBounds()
+ {
+ View initialView = fakeView(0, 5);
+ for (int i = 0 ; i < 5 ; i++)
+ {
+ for (int j = i ; j < 5 ; j++)
+ {
+ RowPosition min = MockSchema.readerBounds(i);
+ RowPosition max = MockSchema.readerBounds(j);
+ for (boolean minInc : new boolean[] { true, false} )
+ {
+ for (boolean maxInc : new boolean[] { true, false} )
+ {
+ if (i == j && !(minInc && maxInc))
+ continue;
+ List<SSTableReader> r = initialView.sstablesInBounds(AbstractBounds.bounds(min, minInc, max, maxInc));
+ Assert.assertEquals(String.format("%d(%s) %d(%s)", i, minInc, j, maxInc), j - i + (minInc ? 0 : -1) + (maxInc ? 1 : 0), r.size());
+ }
+ }
+ }
+ }
+ }
+
+ @Test
+ public void testCompaction()
+ {
+ View initialView = fakeView(0, 5);
+ View cur = initialView;
+ List<SSTableReader> readers = ImmutableList.copyOf(initialView.sstables);
+ Assert.assertTrue(View.permitCompacting(readers).apply(cur));
+ // check we permit compacting duplicates in the predicate, so we don't spin infinitely if there is a screw up
+ Assert.assertTrue(View.permitCompacting(ImmutableList.copyOf(concat(readers, readers))).apply(cur));
+ // check we fail in the application in the presence of duplicates
+ testFailure(View.updateCompacting(emptySet(), concat(readers.subList(0, 1), readers.subList(0, 1))), cur);
+
+ // do lots of trivial checks that the compacting set and related methods behave properly for a simple update
+ cur = View.updateCompacting(emptySet(), readers.subList(0, 2)).apply(cur);
+ Assert.assertTrue(View.permitCompacting(readers.subList(2, 5)).apply(cur));
+ Assert.assertFalse(View.permitCompacting(readers.subList(0, 2)).apply(cur));
+ Assert.assertFalse(View.permitCompacting(readers.subList(0, 1)).apply(cur));
+ Assert.assertFalse(View.permitCompacting(readers.subList(1, 2)).apply(cur));
+ Assert.assertTrue(readers.subList(2, 5).containsAll(copyOf(cur.getUncompacting(readers))));
+ Assert.assertEquals(3, copyOf(cur.getUncompacting(readers)).size());
+ Assert.assertTrue(cur.nonCompactingSStables().containsAll(readers.subList(2, 5)));
+ Assert.assertEquals(3, cur.nonCompactingSStables().size());
+
+ // check marking already compacting readers fails with an exception
+ testFailure(View.updateCompacting(emptySet(), readers.subList(0, 1)), cur);
+ testFailure(View.updateCompacting(emptySet(), readers.subList(1, 2)), cur);
+ testFailure(View.updateCompacting(copyOf(readers.subList(0, 1)), readers.subList(1, 2)), cur);
+
+ // make equivalents of readers.subList(0, 3) that are different instances
+ SSTableReader r0 = MockSchema.sstable(0), r1 = MockSchema.sstable(1), r2 = MockSchema.sstable(2);
+ // attempt to mark compacting a version not in the live set
+ testFailure(View.updateCompacting(emptySet(), of(r2)), cur);
+ // update one compacting, one non-compacting, of the liveset to another instance of the same readers;
+ // confirm liveset changes but compacting does not
+ cur = View.updateLiveSet(copyOf(readers.subList(1, 3)), of(r1, r2)).apply(cur);
+ Assert.assertSame(readers.get(0), cur.sstablesMap.get(r0));
+ Assert.assertSame(r1, cur.sstablesMap.get(r1));
+ Assert.assertSame(r2, cur.sstablesMap.get(r2));
+ testFailure(View.updateCompacting(emptySet(), readers.subList(2, 3)), cur);
+ Assert.assertSame(readers.get(1), Iterables.getFirst(Iterables.filter(cur.compacting, Predicates.equalTo(r1)), null));
+
+ // unmark compacting, and check our methods are all correctly updated
+ cur = View.updateCompacting(copyOf(readers.subList(0, 1)), emptySet()).apply(cur);
+ Assert.assertTrue(View.permitCompacting(concat(readers.subList(0, 1), of(r2), readers.subList(3, 5))).apply(cur));
+ Assert.assertFalse(View.permitCompacting(readers.subList(1, 2)).apply(cur));
+ testFailure(View.updateCompacting(emptySet(), readers.subList(1, 2)), cur);
+ testFailure(View.updateCompacting(copyOf(readers.subList(0, 2)), emptySet()), cur);
+ Assert.assertTrue(copyOf(concat(readers.subList(0, 1), readers.subList(2, 5))).containsAll(copyOf(cur.getUncompacting(readers))));
+ Assert.assertEquals(4, copyOf(cur.getUncompacting(readers)).size());
+ Assert.assertTrue(cur.nonCompactingSStables().containsAll(readers.subList(2, 5)));
+ Assert.assertTrue(cur.nonCompactingSStables().containsAll(readers.subList(0, 1)));
+ Assert.assertEquals(4, cur.nonCompactingSStables().size());
+ }
+
+ private static void testFailure(Function<View, ?> function, View view)
+ {
+ boolean failed = true;
+ try
+ {
+ function.apply(view);
+ failed = false;
+ }
+ catch (Throwable t)
+ {
+ }
+ Assert.assertTrue(failed);
+ }
+
+ @Test
+ public void testFlushing()
+ {
+ View initialView = fakeView(1, 0);
+ View cur = initialView;
+ Memtable memtable1 = initialView.getCurrentMemtable();
+ Memtable memtable2 = MockSchema.memtable();
+
+ cur = View.switchMemtable(memtable2).apply(cur);
+ Assert.assertEquals(2, cur.liveMemtables.size());
+ Assert.assertEquals(memtable1, cur.liveMemtables.get(0));
+ Assert.assertEquals(memtable2, cur.getCurrentMemtable());
+
+ Memtable memtable3 = MockSchema.memtable();
+ cur = View.switchMemtable(memtable3).apply(cur);
+ Assert.assertEquals(3, cur.liveMemtables.size());
+ Assert.assertEquals(0, cur.flushingMemtables.size());
+ Assert.assertEquals(memtable1, cur.liveMemtables.get(0));
+ Assert.assertEquals(memtable2, cur.liveMemtables.get(1));
+ Assert.assertEquals(memtable3, cur.getCurrentMemtable());
+
+ testFailure(View.replaceFlushed(memtable2, null), cur);
+
+ cur = View.markFlushing(memtable2).apply(cur);
+ Assert.assertTrue(cur.flushingMemtables.contains(memtable2));
+ Assert.assertEquals(2, cur.liveMemtables.size());
+ Assert.assertEquals(1, cur.flushingMemtables.size());
+ Assert.assertEquals(memtable2, cur.flushingMemtables.get(0));
+ Assert.assertEquals(memtable1, cur.liveMemtables.get(0));
+ Assert.assertEquals(memtable3, cur.getCurrentMemtable());
+
+ cur = View.markFlushing(memtable1).apply(cur);
+ Assert.assertEquals(1, cur.liveMemtables.size());
+ Assert.assertEquals(2, cur.flushingMemtables.size());
+ Assert.assertEquals(memtable1, cur.flushingMemtables.get(0));
+ Assert.assertEquals(memtable2, cur.flushingMemtables.get(1));
+ Assert.assertEquals(memtable3, cur.getCurrentMemtable());
+
+ cur = View.replaceFlushed(memtable2, null).apply(cur);
+ Assert.assertEquals(1, cur.liveMemtables.size());
+ Assert.assertEquals(1, cur.flushingMemtables.size());
+ Assert.assertEquals(memtable1, cur.flushingMemtables.get(0));
+ Assert.assertEquals(memtable3, cur.getCurrentMemtable());
+
+ SSTableReader sstable = MockSchema.sstable(1);
+ cur = View.replaceFlushed(memtable1, sstable).apply(cur);
+ Assert.assertEquals(0, cur.flushingMemtables.size());
+ Assert.assertEquals(1, cur.liveMemtables.size());
+ Assert.assertEquals(memtable3, cur.getCurrentMemtable());
+ Assert.assertEquals(1, cur.sstables.size());
+ Assert.assertEquals(sstable, cur.sstablesMap.get(sstable));
+ }
+
+ static View fakeView(int memtableCount, int sstableCount)
+ {
+ List<Memtable> memtables = new ArrayList<>();
+ List<SSTableReader> sstables = new ArrayList<>();
+ for (int i = 0 ; i < memtableCount ; i++)
+ memtables.add(MockSchema.memtable());
+ for (int i = 0 ; i < sstableCount ; i++)
+ sstables.add(MockSchema.sstable(i));
+ return new View(ImmutableList.copyOf(memtables), Collections.<Memtable>emptyList(), Helpers.identityMap(sstables),
+ Collections.<SSTableReader>emptySet(), SSTableIntervalTree.build(sstables));
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
index eac6094..518d80e 100644
--- a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
@@ -41,13 +41,15 @@ import org.apache.cassandra.cache.CachingOptions;
import org.apache.cassandra.config.KSMetaData;
import org.apache.cassandra.db.*;
import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.compaction.OperationType;
import org.apache.cassandra.db.filter.QueryFilter;
import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.metrics.RestorableMeter;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.concurrent.OpOrder;
+import static com.google.common.collect.ImmutableMap.of;
+import static java.util.Arrays.asList;
import static org.apache.cassandra.io.sstable.Downsampling.BASE_SAMPLING_LEVEL;
import static org.apache.cassandra.io.sstable.IndexSummaryManager.DOWNSAMPLE_THESHOLD;
import static org.apache.cassandra.io.sstable.IndexSummaryManager.UPSAMPLE_THRESHOLD;
@@ -121,12 +123,15 @@ public class IndexSummaryManagerTest
return total;
}
- private static List<SSTableReader> resetSummaries(List<SSTableReader> sstables, long originalOffHeapSize) throws IOException
+ private static List<SSTableReader> resetSummaries(ColumnFamilyStore cfs, List<SSTableReader> sstables, long originalOffHeapSize) throws IOException
{
for (SSTableReader sstable : sstables)
sstable.overrideReadMeter(new RestorableMeter(100.0, 100.0));
- sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, originalOffHeapSize * sstables.size());
+ try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+ {
+ sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), originalOffHeapSize * sstables.size());
+ }
for (SSTableReader sstable : sstables)
assertEquals(BASE_SAMPLING_LEVEL, sstable.getIndexSummarySamplingLevel());
@@ -230,7 +235,11 @@ public class IndexSummaryManagerTest
cfs.metadata.minIndexInterval(originalMinIndexInterval / 2);
SSTableReader sstable = cfs.getSSTables().iterator().next();
long summarySpace = sstable.getIndexSummaryOffHeapSize();
- IndexSummaryManager.redistributeSummaries(Collections.EMPTY_LIST, Arrays.asList(sstable), summarySpace);
+ try (LifecycleTransaction txn = cfs.getTracker().tryModify(asList(sstable), OperationType.UNKNOWN))
+ {
+ redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), summarySpace);
+ }
+
sstable = cfs.getSSTables().iterator().next();
assertEquals(originalMinIndexInterval, sstable.getEffectiveIndexInterval(), 0.001);
assertEquals(numRows / originalMinIndexInterval, sstable.getIndexSummarySize());
@@ -238,7 +247,10 @@ public class IndexSummaryManagerTest
// keep the min_index_interval the same, but now give the summary enough space to grow by 50%
double previousInterval = sstable.getEffectiveIndexInterval();
int previousSize = sstable.getIndexSummarySize();
- IndexSummaryManager.redistributeSummaries(Collections.EMPTY_LIST, Arrays.asList(sstable), (long) Math.ceil(summarySpace * 1.5));
+ try (LifecycleTransaction txn = cfs.getTracker().tryModify(asList(sstable), OperationType.UNKNOWN))
+ {
+ redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (long) Math.ceil(summarySpace * 1.5));
+ }
sstable = cfs.getSSTables().iterator().next();
assertEquals(previousSize * 1.5, (double) sstable.getIndexSummarySize(), 1);
assertEquals(previousInterval * (1.0 / 1.5), sstable.getEffectiveIndexInterval(), 0.001);
@@ -246,7 +258,10 @@ public class IndexSummaryManagerTest
// return min_index_interval to it's original value (double it), but only give the summary enough space
// to have an effective index interval of twice the new min
cfs.metadata.minIndexInterval(originalMinIndexInterval);
- IndexSummaryManager.redistributeSummaries(Collections.EMPTY_LIST, Arrays.asList(sstable), (long) Math.ceil(summarySpace / 2.0));
+ try (LifecycleTransaction txn = cfs.getTracker().tryModify(asList(sstable), OperationType.UNKNOWN))
+ {
+ redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (long) Math.ceil(summarySpace / 2.0));
+ }
sstable = cfs.getSSTables().iterator().next();
assertEquals(originalMinIndexInterval * 2, sstable.getEffectiveIndexInterval(), 0.001);
assertEquals(numRows / (originalMinIndexInterval * 2), sstable.getIndexSummarySize());
@@ -256,7 +271,10 @@ public class IndexSummaryManagerTest
// result in an effective interval above the new max)
cfs.metadata.minIndexInterval(originalMinIndexInterval * 4);
cfs.metadata.maxIndexInterval(originalMinIndexInterval * 4);
- IndexSummaryManager.redistributeSummaries(Collections.EMPTY_LIST, Arrays.asList(sstable), 10);
+ try (LifecycleTransaction txn = cfs.getTracker().tryModify(asList(sstable), OperationType.UNKNOWN))
+ {
+ redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), 10);
+ }
sstable = cfs.getSSTables().iterator().next();
assertEquals(cfs.metadata.getMinIndexInterval(), sstable.getEffectiveIndexInterval(), 0.001);
}
@@ -276,14 +294,20 @@ public class IndexSummaryManagerTest
for (SSTableReader sstable : sstables)
sstable.overrideReadMeter(new RestorableMeter(100.0, 100.0));
- IndexSummaryManager.redistributeSummaries(Collections.EMPTY_LIST, sstables, 1);
+ try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+ {
+ redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), 10);
+ }
sstables = new ArrayList<>(cfs.getSSTables());
for (SSTableReader sstable : sstables)
assertEquals(cfs.metadata.getMaxIndexInterval(), sstable.getEffectiveIndexInterval(), 0.01);
// halve the max_index_interval
cfs.metadata.maxIndexInterval(cfs.metadata.getMaxIndexInterval() / 2);
- IndexSummaryManager.redistributeSummaries(Collections.EMPTY_LIST, sstables, 1);
+ try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+ {
+ redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), 1);
+ }
sstables = new ArrayList<>(cfs.getSSTables());
for (SSTableReader sstable : sstables)
{
@@ -293,7 +317,10 @@ public class IndexSummaryManagerTest
// return max_index_interval to its original value
cfs.metadata.maxIndexInterval(cfs.metadata.getMaxIndexInterval() * 2);
- IndexSummaryManager.redistributeSummaries(Collections.EMPTY_LIST, sstables, 1);
+ try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+ {
+ redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), 1);
+ }
for (SSTableReader sstable : cfs.getSSTables())
{
assertEquals(cfs.metadata.getMaxIndexInterval(), sstable.getEffectiveIndexInterval(), 0.01);
@@ -321,7 +348,10 @@ public class IndexSummaryManagerTest
long singleSummaryOffHeapSpace = sstables.get(0).getIndexSummaryOffHeapSize();
// there should be enough space to not downsample anything
- sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, (singleSummaryOffHeapSpace * numSSTables));
+ try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+ {
+ sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * numSSTables));
+ }
for (SSTableReader sstable : sstables)
assertEquals(BASE_SAMPLING_LEVEL, sstable.getIndexSummarySamplingLevel());
assertEquals(singleSummaryOffHeapSpace * numSSTables, totalOffHeapSize(sstables));
@@ -329,26 +359,38 @@ public class IndexSummaryManagerTest
// everything should get cut in half
assert sstables.size() == 4;
- sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, (singleSummaryOffHeapSpace * (numSSTables / 2)));
+ try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+ {
+ sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * (numSSTables / 2)));
+ }
for (SSTableReader sstable : sstables)
assertEquals(BASE_SAMPLING_LEVEL / 2, sstable.getIndexSummarySamplingLevel());
validateData(cfs, numRows);
// everything should get cut to a quarter
- sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, (singleSummaryOffHeapSpace * (numSSTables / 4)));
+ try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+ {
+ sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * (numSSTables / 4)));
+ }
for (SSTableReader sstable : sstables)
assertEquals(BASE_SAMPLING_LEVEL / 4, sstable.getIndexSummarySamplingLevel());
validateData(cfs, numRows);
// upsample back up to half
- sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables,(singleSummaryOffHeapSpace * (numSSTables / 2) + 4));
+ try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+ {
+ sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * (numSSTables / 2) + 4));
+ }
assert sstables.size() == 4;
for (SSTableReader sstable : sstables)
assertEquals(BASE_SAMPLING_LEVEL / 2, sstable.getIndexSummarySamplingLevel());
validateData(cfs, numRows);
// upsample back up to the original index summary
- sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, (singleSummaryOffHeapSpace * numSSTables));
+ try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+ {
+ sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * numSSTables));
+ }
for (SSTableReader sstable : sstables)
assertEquals(BASE_SAMPLING_LEVEL, sstable.getIndexSummarySamplingLevel());
validateData(cfs, numRows);
@@ -357,7 +399,10 @@ public class IndexSummaryManagerTest
// so the two cold sstables should get downsampled to be half of their original size
sstables.get(0).overrideReadMeter(new RestorableMeter(50.0, 50.0));
sstables.get(1).overrideReadMeter(new RestorableMeter(50.0, 50.0));
- sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, (singleSummaryOffHeapSpace * 3));
+ try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+ {
+ sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * 3));
+ }
Collections.sort(sstables, hotnessComparator);
assertEquals(BASE_SAMPLING_LEVEL / 2, sstables.get(0).getIndexSummarySamplingLevel());
assertEquals(BASE_SAMPLING_LEVEL / 2, sstables.get(1).getIndexSummarySamplingLevel());
@@ -370,7 +415,10 @@ public class IndexSummaryManagerTest
double higherRate = 50.0 * (UPSAMPLE_THRESHOLD - (UPSAMPLE_THRESHOLD * 0.10));
sstables.get(0).overrideReadMeter(new RestorableMeter(lowerRate, lowerRate));
sstables.get(1).overrideReadMeter(new RestorableMeter(higherRate, higherRate));
- sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, (singleSummaryOffHeapSpace * 3));
+ try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+ {
+ sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * 3));
+ }
Collections.sort(sstables, hotnessComparator);
assertEquals(BASE_SAMPLING_LEVEL / 2, sstables.get(0).getIndexSummarySamplingLevel());
assertEquals(BASE_SAMPLING_LEVEL / 2, sstables.get(1).getIndexSummarySamplingLevel());
@@ -379,13 +427,16 @@ public class IndexSummaryManagerTest
validateData(cfs, numRows);
// reset, and then this time, leave enough space for one of the cold sstables to not get downsampled
- sstables = resetSummaries(sstables, singleSummaryOffHeapSpace);
+ sstables = resetSummaries(cfs, sstables, singleSummaryOffHeapSpace);
sstables.get(0).overrideReadMeter(new RestorableMeter(1.0, 1.0));
sstables.get(1).overrideReadMeter(new RestorableMeter(2.0, 2.0));
sstables.get(2).overrideReadMeter(new RestorableMeter(1000.0, 1000.0));
sstables.get(3).overrideReadMeter(new RestorableMeter(1000.0, 1000.0));
- sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, (singleSummaryOffHeapSpace * 3) + 50);
+ try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+ {
+ sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * 3) + 50);
+ }
Collections.sort(sstables, hotnessComparator);
if (sstables.get(0).getIndexSummarySamplingLevel() == minSamplingLevel)
@@ -406,7 +457,10 @@ public class IndexSummaryManagerTest
sstables.get(1).overrideReadMeter(new RestorableMeter(0.0, 0.0));
sstables.get(2).overrideReadMeter(new RestorableMeter(92, 92));
sstables.get(3).overrideReadMeter(new RestorableMeter(128.0, 128.0));
- sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, (long) (singleSummaryOffHeapSpace + (singleSummaryOffHeapSpace * (92.0 / BASE_SAMPLING_LEVEL))));
+ try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+ {
+ sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (long) (singleSummaryOffHeapSpace + (singleSummaryOffHeapSpace * (92.0 / BASE_SAMPLING_LEVEL))));
+ }
Collections.sort(sstables, hotnessComparator);
assertEquals(1, sstables.get(0).getIndexSummarySize()); // at the min sampling level
assertEquals(1, sstables.get(0).getIndexSummarySize()); // at the min sampling level
@@ -416,7 +470,10 @@ public class IndexSummaryManagerTest
validateData(cfs, numRows);
// Don't leave enough space for even the minimal index summaries
- sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, 10);
+ try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+ {
+ sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), 10);
+ }
for (SSTableReader sstable : sstables)
assertEquals(1, sstable.getIndexSummarySize()); // at the min sampling level
validateData(cfs, numRows);
@@ -449,19 +506,19 @@ public class IndexSummaryManagerTest
SSTableReader original = sstables.get(0);
SSTableReader sstable = original;
- for (int samplingLevel = 1; samplingLevel < BASE_SAMPLING_LEVEL; samplingLevel++)
+ try (LifecycleTransaction txn = cfs.getTracker().tryModify(asList(sstable), OperationType.UNKNOWN))
{
- SSTableReader prev = sstable;
- sstable = sstable.cloneWithNewSummarySamplingLevel(cfs, samplingLevel);
- assertEquals(samplingLevel, sstable.getIndexSummarySamplingLevel());
- int expectedSize = (numRows * samplingLevel) / (sstable.metadata.getMinIndexInterval() * BASE_SAMPLING_LEVEL);
- assertEquals(expectedSize, sstable.getIndexSummarySize(), 1);
- if (prev != original)
- prev.selfRef().release();
+ for (int samplingLevel = 1; samplingLevel < BASE_SAMPLING_LEVEL; samplingLevel++)
+ {
+ sstable = sstable.cloneWithNewSummarySamplingLevel(cfs, samplingLevel);
+ assertEquals(samplingLevel, sstable.getIndexSummarySamplingLevel());
+ int expectedSize = (numRows * samplingLevel) / (sstable.metadata.getMinIndexInterval() * BASE_SAMPLING_LEVEL);
+ assertEquals(expectedSize, sstable.getIndexSummarySize(), 1);
+ txn.update(sstable, true);
+ txn.checkpoint();
+ }
+ txn.finish();
}
-
- // don't leave replaced SSTRs around to break other tests
- cfs.getDataTracker().replaceWithNewInstances(Collections.singleton(original), Collections.singleton(sstable));
}
@Test
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/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 82cb8d5..682d999 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
@@ -59,7 +59,9 @@ import org.apache.cassandra.db.Row;
import org.apache.cassandra.db.RowPosition;
import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.compaction.OperationType;
import org.apache.cassandra.db.composites.Composites;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.dht.LocalPartitioner;
import org.apache.cassandra.dht.LocalPartitioner.LocalToken;
import org.apache.cassandra.dht.Range;
@@ -498,8 +500,13 @@ public class SSTableReaderTest
}));
}
- SSTableReader replacement = sstable.cloneWithNewSummarySamplingLevel(store, 1);
- store.getDataTracker().replaceWithNewInstances(Arrays.asList(sstable), Arrays.asList(replacement));
+ SSTableReader replacement;
+ try (LifecycleTransaction txn = store.getTracker().tryModify(Arrays.asList(sstable), OperationType.UNKNOWN))
+ {
+ replacement = sstable.cloneWithNewSummarySamplingLevel(store, 1);
+ txn.update(replacement, true);
+ txn.finish();
+ }
for (Future future : futures)
future.get();