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/07/24 18:32:10 UTC
[1/5] cassandra git commit: Introduce safer durable sstable
membership management (and simplify cleanup of compaction leftovers)
Repository: cassandra
Updated Branches:
refs/heads/trunk e338d2fa8 -> b09e60f72
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
index 60cac2b..860f1d1 100644
--- a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
@@ -100,7 +100,7 @@ public class LegacySSTableTest
protected Descriptor getDescriptor(String ver)
{
File directory = new File(LEGACY_SSTABLE_ROOT + File.separator + ver + File.separator + KSNAME);
- return new Descriptor(ver, directory, KSNAME, CFNAME, 0, Descriptor.Type.FINAL, SSTableFormat.Type.LEGACY);
+ return new Descriptor(ver, directory, KSNAME, CFNAME, 0, SSTableFormat.Type.LEGACY);
}
/**
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
index 5a7c074..782f7fd 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
@@ -21,6 +21,8 @@ import java.io.File;
import java.util.List;
import com.google.common.io.Files;
+import org.junit.After;
+import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -33,6 +35,7 @@ import org.apache.cassandra.db.partitions.*;
import org.apache.cassandra.db.marshal.AsciiType;
import org.apache.cassandra.dht.Range;
import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.util.FileUtils;
import org.apache.cassandra.schema.KeyspaceParams;
import org.apache.cassandra.service.StorageService;
import org.apache.cassandra.utils.ByteBufferUtil;
@@ -40,11 +43,15 @@ import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.OutputHandler;
import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
public class SSTableLoaderTest
{
public static final String KEYSPACE1 = "SSTableLoaderTest";
- public static final String CF_STANDARD = "Standard1";
+ public static final String CF_STANDARD1 = "Standard1";
+ public static final String CF_STANDARD2 = "Standard2";
+
+ private File tmpdir;
@BeforeClass
public static void defineSchema() throws Exception
@@ -52,57 +59,66 @@ public class SSTableLoaderTest
SchemaLoader.prepareServer();
SchemaLoader.createKeyspace(KEYSPACE1,
KeyspaceParams.simple(1),
- SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD));
- setup();
+ SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
+ SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2));
+
+ StorageService.instance.initServer();
}
- public static void setup() throws Exception
+ @Before
+ public void setup() throws Exception
{
- StorageService.instance.initServer();
+ tmpdir = Files.createTempDir();
+ }
+
+ @After
+ public void cleanup()
+ {
+ FileUtils.deleteRecursive(tmpdir);
+ }
+
+ private static final class TestClient extends SSTableLoader.Client
+ {
+ private String keyspace;
+
+ public void init(String keyspace)
+ {
+ this.keyspace = keyspace;
+ for (Range<Token> range : StorageService.instance.getLocalRanges(KEYSPACE1))
+ addRangeForEndpoint(range, FBUtilities.getBroadcastAddress());
+ setPartitioner(StorageService.getPartitioner());
+ }
+
+ public CFMetaData getTableMetadata(String tableName)
+ {
+ return Schema.instance.getCFMetaData(keyspace, tableName);
+ }
}
@Test
public void testLoadingSSTable() throws Exception
{
- File tempdir = Files.createTempDir();
- File dataDir = new File(tempdir.getAbsolutePath() + File.separator + KEYSPACE1 + File.separator + CF_STANDARD);
+ File dataDir = new File(tmpdir.getAbsolutePath() + File.separator + KEYSPACE1 + File.separator + CF_STANDARD1);
assert dataDir.mkdirs();
- CFMetaData cfmeta = Schema.instance.getCFMetaData(KEYSPACE1, CF_STANDARD);
+ CFMetaData cfmeta = Schema.instance.getCFMetaData(KEYSPACE1, CF_STANDARD1);
String schema = "CREATE TABLE %s.%s (key ascii, name ascii, val ascii, val1 ascii, PRIMARY KEY (key, name))";
String query = "INSERT INTO %s.%s (key, name, val) VALUES (?, ?, ?)";
- ;
+
try (CQLSSTableWriter writer = CQLSSTableWriter.builder()
.inDirectory(dataDir)
.withPartitioner(StorageService.getPartitioner())
- .forTable(String.format(schema, KEYSPACE1, CF_STANDARD))
- .using(String.format(query, KEYSPACE1, CF_STANDARD))
+ .forTable(String.format(schema, KEYSPACE1, CF_STANDARD1))
+ .using(String.format(query, KEYSPACE1, CF_STANDARD1))
.build())
{
writer.addRow("key1", "col1", "100");
}
- SSTableLoader loader = new SSTableLoader(dataDir, new SSTableLoader.Client()
- {
- private String keyspace;
-
- public void init(String keyspace)
- {
- this.keyspace = keyspace;
- for (Range<Token> range : StorageService.instance.getLocalRanges(KEYSPACE1))
- addRangeForEndpoint(range, FBUtilities.getBroadcastAddress());
- setPartitioner(StorageService.getPartitioner());
- }
-
- public CFMetaData getTableMetadata(String tableName)
- {
- return Schema.instance.getCFMetaData(keyspace, tableName);
- }
- }, new OutputHandler.SystemOutput(false, false));
-
+ SSTableLoader loader = new SSTableLoader(dataDir, new TestClient(), new OutputHandler.SystemOutput(false, false));
loader.stream().get();
- List<FilteredPartition> partitions = Util.getAll(Util.cmd(Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD)).build());
+ List<FilteredPartition> partitions = Util.getAll(Util.cmd(Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1)).build());
assertEquals(1, partitions.size());
assertEquals("key1", AsciiType.instance.getString(partitions.get(0).partitionKey().getKey()));
@@ -110,4 +126,51 @@ public class SSTableLoaderTest
.getCell(cfmeta.getColumnDefinition(ByteBufferUtil.bytes("val")))
.value());
}
+
+ @Test
+ public void testLoadingIncompleteSSTable() throws Exception
+ {
+ File dataDir = new File(tmpdir.getAbsolutePath() + File.separator + KEYSPACE1 + File.separator + CF_STANDARD2);
+ assert dataDir.mkdirs();
+
+ //make sure we have no tables...
+ assertTrue(dataDir.listFiles().length == 0);
+
+ String schema = "CREATE TABLE %s.%s (key ascii, name ascii, val ascii, val1 ascii, PRIMARY KEY (key, name))";
+ String query = "INSERT INTO %s.%s (key, name, val) VALUES (?, ?, ?)";
+
+ CQLSSTableWriter writer = CQLSSTableWriter.builder()
+ .inDirectory(dataDir)
+ .withPartitioner(StorageService.getPartitioner())
+ .forTable(String.format(schema, KEYSPACE1, CF_STANDARD2))
+ .using(String.format(query, KEYSPACE1, CF_STANDARD2))
+ .withBufferSizeInMB(1)
+ .build();
+
+ for (int i = 0; i < 1000; i++) // make sure to write more than 1 MB
+ {
+ for (int j = 0; j < 100; j++)
+ writer.addRow(String.format("key%d", i), String.format("col%d", j), "100");
+ }
+
+ //make sure we have some tables...
+ assertTrue(dataDir.listFiles().length > 0);
+
+ //writer is still open so loader should not load anything
+ SSTableLoader loader = new SSTableLoader(dataDir, new TestClient(), new OutputHandler.SystemOutput(false, false));
+ loader.stream().get();
+
+ List<FilteredPartition> partitions = Util.getAll(Util.cmd(Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD2)).build());
+
+ assertTrue(partitions.size() > 0 && partitions.size() < 1000);
+
+ // now we complete the write and the second loader should load the last sstable as well
+ writer.close();
+
+ loader = new SSTableLoader(dataDir, new TestClient(), new OutputHandler.SystemOutput(false, false));
+ loader.stream().get();
+
+ partitions = Util.getAll(Util.cmd(Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD2)).build());
+ assertEquals(1000, partitions.size());
+ }
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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 579f981..0e533c2 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
@@ -44,6 +44,7 @@ import org.apache.cassandra.db.Keyspace;
import org.apache.cassandra.db.RowUpdateBuilder;
import org.apache.cassandra.db.SerializationHeader;
import org.apache.cassandra.db.rows.EncodingStats;
+import org.apache.cassandra.db.lifecycle.TransactionLogs;
import org.apache.cassandra.db.rows.UnfilteredRowIterator;
import org.apache.cassandra.db.compaction.AbstractCompactionStrategy;
import org.apache.cassandra.db.compaction.CompactionController;
@@ -106,7 +107,7 @@ public class SSTableRewriterTest extends SchemaLoader
Keyspace keyspace = Keyspace.open(KEYSPACE);
ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF);
store.truncateBlocking();
- SSTableDeletingTask.waitForDeletions();
+ TransactionLogs.waitForDeletions();
}
@Test
@@ -135,16 +136,16 @@ public class SSTableRewriterTest extends SchemaLoader
CompactionController controller = new CompactionController(cfs, sstables, cfs.gcBefore(nowInSec));
CompactionIterator ci = new CompactionIterator(OperationType.COMPACTION, scanners.scanners, controller, nowInSec, UUIDGen.getTimeUUID()))
{
- writer.switchWriter(getWriter(cfs, sstables.iterator().next().descriptor.directory));
+ writer.switchWriter(getWriter(cfs, sstables.iterator().next().descriptor.directory, txn));
while(ci.hasNext())
{
writer.append(ci.next());
}
writer.finish();
}
- SSTableDeletingTask.waitForDeletions();
+ TransactionLogs.waitForDeletions();
validateCFS(cfs);
- int filecounts = assertFileCounts(sstables.iterator().next().descriptor.directory.list(), 0, 0);
+ int filecounts = assertFileCounts(sstables.iterator().next().descriptor.directory.list());
assertEquals(1, filecounts);
truncate(cfs);
}
@@ -167,16 +168,16 @@ public class SSTableRewriterTest extends SchemaLoader
CompactionController controller = new CompactionController(cfs, sstables, cfs.gcBefore(nowInSec));
CompactionIterator ci = new CompactionIterator(OperationType.COMPACTION, scanners.scanners, controller, nowInSec, UUIDGen.getTimeUUID()))
{
- writer.switchWriter(getWriter(cfs, sstables.iterator().next().descriptor.directory));
+ writer.switchWriter(getWriter(cfs, sstables.iterator().next().descriptor.directory, txn));
while (ci.hasNext())
{
writer.append(ci.next());
}
writer.finish();
}
- SSTableDeletingTask.waitForDeletions();
+ TransactionLogs.waitForDeletions();
validateCFS(cfs);
- int filecounts = assertFileCounts(sstables.iterator().next().descriptor.directory.list(), 0, 0);
+ int filecounts = assertFileCounts(sstables.iterator().next().descriptor.directory.list());
assertEquals(1, filecounts);
}
@@ -200,7 +201,7 @@ public class SSTableRewriterTest extends SchemaLoader
CompactionController controller = new CompactionController(cfs, sstables, cfs.gcBefore(nowInSec));
CompactionIterator ci = new CompactionIterator(OperationType.COMPACTION, scanners.scanners, controller, nowInSec, UUIDGen.getTimeUUID()))
{
- writer.switchWriter(getWriter(cfs, sstables.iterator().next().descriptor.directory));
+ writer.switchWriter(getWriter(cfs, sstables.iterator().next().descriptor.directory, txn));
while (ci.hasNext())
{
UnfilteredRowIterator row = ci.next();
@@ -229,9 +230,9 @@ public class SSTableRewriterTest extends SchemaLoader
assertTrue(checked);
writer.finish();
}
- SSTableDeletingTask.waitForDeletions();
+ TransactionLogs.waitForDeletions();
validateCFS(cfs);
- int filecounts = assertFileCounts(sstables.iterator().next().descriptor.directory.list(), 0, 0);
+ int filecounts = assertFileCounts(sstables.iterator().next().descriptor.directory.list());
assertEquals(1, filecounts);
truncate(cfs);
}
@@ -244,7 +245,8 @@ public class SSTableRewriterTest extends SchemaLoader
truncate(cfs);
File dir = cfs.directories.getDirectoryForNewSSTables();
- try (SSTableWriter writer = getWriter(cfs, dir))
+ LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.WRITE, cfs.metadata);
+ try (SSTableWriter writer = getWriter(cfs, dir, txn))
{
for (int i = 0; i < 10000; i++)
{
@@ -256,7 +258,7 @@ public class SSTableRewriterTest extends SchemaLoader
SSTableReader s = writer.setMaxDataAge(1000).openEarly();
assert s != null;
- assertFileCounts(dir.list(), 2, 2);
+ assertFileCounts(dir.list());
for (int i = 10000; i < 20000; i++)
{
UpdateBuilder builder = UpdateBuilder.create(cfs.metadata, random(i, 10)).withTimestamp(1);
@@ -266,20 +268,20 @@ public class SSTableRewriterTest extends SchemaLoader
}
SSTableReader s2 = writer.setMaxDataAge(1000).openEarly();
assertTrue(s.last.compareTo(s2.last) < 0);
- assertFileCounts(dir.list(), 2, 2);
- s.markObsolete(cfs.getTracker());
+ assertFileCounts(dir.list());
s.selfRef().release();
s2.selfRef().release();
// These checks don't work on Windows because the writer has the channel still
// open till .abort() is called (via the builder)
if (!FBUtilities.isWindows())
{
- SSTableDeletingTask.waitForDeletions();
- assertFileCounts(dir.list(), 0, 2);
+ TransactionLogs.waitForDeletions();
+ assertFileCounts(dir.list());
}
writer.abort();
- SSTableDeletingTask.waitForDeletions();
- int datafiles = assertFileCounts(dir.list(), 0, 0);
+ txn.abort();
+ TransactionLogs.waitForDeletions();
+ int datafiles = assertFileCounts(dir.list());
assertEquals(datafiles, 0);
validateCFS(cfs);
}
@@ -306,14 +308,14 @@ public class SSTableRewriterTest extends SchemaLoader
SSTableRewriter rewriter = new SSTableRewriter(cfs, txn, 1000, false, 10000000);
CompactionIterator ci = new CompactionIterator(OperationType.COMPACTION, Collections.singletonList(scanner), controller, FBUtilities.nowInSeconds(), UUIDGen.getTimeUUID()))
{
- rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
+ rewriter.switchWriter(getWriter(cfs, s.descriptor.directory, txn));
while(ci.hasNext())
{
rewriter.append(ci.next());
if (rewriter.currentWriter().getOnDiskFilePointer() > 25000000)
{
- rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
+ rewriter.switchWriter(getWriter(cfs, s.descriptor.directory, txn));
files++;
assertEquals(cfs.getSSTables().size(), files); // we have one original file plus the ones we have switched out.
assertEquals(s.bytesOnDisk(), cfs.metric.liveDiskSpaceUsed.getCount());
@@ -323,6 +325,9 @@ public class SSTableRewriterTest extends SchemaLoader
}
sstables = rewriter.finish();
}
+
+ TransactionLogs.waitForDeletions();
+
long sum = 0;
for (SSTableReader x : cfs.getSSTables())
sum += x.bytesOnDisk();
@@ -330,11 +335,11 @@ public class SSTableRewriterTest extends SchemaLoader
assertEquals(startStorageMetricsLoad - sBytesOnDisk + sum, StorageMetrics.load.getCount());
assertEquals(files, sstables.size());
assertEquals(files, cfs.getSSTables().size());
- SSTableDeletingTask.waitForDeletions();
+ TransactionLogs.waitForDeletions();
// tmplink and tmp files should be gone:
assertEquals(sum, cfs.metric.totalDiskSpaceUsed.getCount());
- assertFileCounts(s.descriptor.directory.list(), 0, 0);
+ assertFileCounts(s.descriptor.directory.list());
validateCFS(cfs);
}
@@ -358,14 +363,14 @@ public class SSTableRewriterTest extends SchemaLoader
SSTableRewriter rewriter = new SSTableRewriter(cfs, txn, 1000, false, 10000000);
CompactionIterator ci = new CompactionIterator(OperationType.COMPACTION, Collections.singletonList(scanner), controller, FBUtilities.nowInSeconds(), UUIDGen.getTimeUUID()))
{
- rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
+ rewriter.switchWriter(getWriter(cfs, s.descriptor.directory, txn));
while(ci.hasNext())
{
rewriter.append(ci.next());
if (rewriter.currentWriter().getOnDiskFilePointer() > 25000000)
{
- rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
+ rewriter.switchWriter(getWriter(cfs, s.descriptor.directory, txn));
files++;
assertEquals(cfs.getSSTables().size(), files); // we have one original file plus the ones we have switched out.
}
@@ -375,9 +380,9 @@ public class SSTableRewriterTest extends SchemaLoader
assertEquals(files, sstables.size());
assertEquals(files, cfs.getSSTables().size());
- SSTableDeletingTask.waitForDeletions();
+ TransactionLogs.waitForDeletions();
- assertFileCounts(s.descriptor.directory.list(), 0, 0);
+ assertFileCounts(s.descriptor.directory.list());
validateCFS(cfs);
}
@@ -387,7 +392,12 @@ public class SSTableRewriterTest extends SchemaLoader
{
testNumberOfFiles_abort(new RewriterTest()
{
- public void run(ISSTableScanner scanner, CompactionController controller, SSTableReader sstable, ColumnFamilyStore cfs, SSTableRewriter rewriter)
+ public void run(ISSTableScanner scanner,
+ CompactionController controller,
+ SSTableReader sstable,
+ ColumnFamilyStore cfs,
+ SSTableRewriter rewriter,
+ LifecycleTransaction txn)
{
try (CompactionIterator ci = new CompactionIterator(OperationType.COMPACTION, Collections.singletonList(scanner), controller, FBUtilities.nowInSeconds(), UUIDGen.getTimeUUID()))
{
@@ -397,7 +407,7 @@ public class SSTableRewriterTest extends SchemaLoader
rewriter.append(ci.next());
if (rewriter.currentWriter().getFilePointer() > 25000000)
{
- rewriter.switchWriter(getWriter(cfs, sstable.descriptor.directory));
+ rewriter.switchWriter(getWriter(cfs, sstable.descriptor.directory, txn));
files++;
assertEquals(cfs.getSSTables().size(), files); // we have one original file plus the ones we have switched out.
}
@@ -413,7 +423,12 @@ public class SSTableRewriterTest extends SchemaLoader
{
testNumberOfFiles_abort(new RewriterTest()
{
- public void run(ISSTableScanner scanner, CompactionController controller, SSTableReader sstable, ColumnFamilyStore cfs, SSTableRewriter rewriter)
+ public void run(ISSTableScanner scanner,
+ CompactionController controller,
+ SSTableReader sstable,
+ ColumnFamilyStore cfs,
+ SSTableRewriter rewriter,
+ LifecycleTransaction txn)
{
try (CompactionIterator ci = new CompactionIterator(OperationType.COMPACTION, Collections.singletonList(scanner), controller, FBUtilities.nowInSeconds(), UUIDGen.getTimeUUID()))
{
@@ -423,7 +438,7 @@ public class SSTableRewriterTest extends SchemaLoader
rewriter.append(ci.next());
if (rewriter.currentWriter().getFilePointer() > 25000000)
{
- rewriter.switchWriter(getWriter(cfs, sstable.descriptor.directory));
+ rewriter.switchWriter(getWriter(cfs, sstable.descriptor.directory, txn));
files++;
assertEquals(cfs.getSSTables().size(), files); // we have one original file plus the ones we have switched out.
}
@@ -444,7 +459,12 @@ public class SSTableRewriterTest extends SchemaLoader
{
testNumberOfFiles_abort(new RewriterTest()
{
- public void run(ISSTableScanner scanner, CompactionController controller, SSTableReader sstable, ColumnFamilyStore cfs, SSTableRewriter rewriter)
+ public void run(ISSTableScanner scanner,
+ CompactionController controller,
+ SSTableReader sstable,
+ ColumnFamilyStore cfs,
+ SSTableRewriter rewriter,
+ LifecycleTransaction txn)
{
try(CompactionIterator ci = new CompactionIterator(OperationType.COMPACTION, Collections.singletonList(scanner), controller, FBUtilities.nowInSeconds(), UUIDGen.getTimeUUID()))
{
@@ -454,7 +474,7 @@ public class SSTableRewriterTest extends SchemaLoader
rewriter.append(ci.next());
if (files == 1 && rewriter.currentWriter().getFilePointer() > 10000000)
{
- rewriter.switchWriter(getWriter(cfs, sstable.descriptor.directory));
+ rewriter.switchWriter(getWriter(cfs, sstable.descriptor.directory, txn));
files++;
assertEquals(cfs.getSSTables().size(), files); // we have one original file plus the ones we have switched out.
}
@@ -467,7 +487,12 @@ public class SSTableRewriterTest extends SchemaLoader
private static interface RewriterTest
{
- public void run(ISSTableScanner scanner, CompactionController controller, SSTableReader sstable, ColumnFamilyStore cfs, SSTableRewriter rewriter);
+ public void run(ISSTableScanner scanner,
+ CompactionController controller,
+ SSTableReader sstable,
+ ColumnFamilyStore cfs,
+ SSTableRewriter rewriter,
+ LifecycleTransaction txn);
}
private void testNumberOfFiles_abort(RewriterTest test) throws Exception
@@ -483,21 +508,20 @@ public class SSTableRewriterTest extends SchemaLoader
DecoratedKey origLast = s.last;
long startSize = cfs.metric.liveDiskSpaceUsed.getCount();
Set<SSTableReader> compacting = Sets.newHashSet(s);
-
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, 10000000))
{
- rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
- test.run(scanner, controller, s, cfs, rewriter);
+ rewriter.switchWriter(getWriter(cfs, s.descriptor.directory, txn));
+ test.run(scanner, controller, s, cfs, rewriter, txn);
}
- SSTableDeletingTask.waitForDeletions();
+ TransactionLogs.waitForDeletions();
assertEquals(startSize, cfs.metric.liveDiskSpaceUsed.getCount());
assertEquals(1, cfs.getSSTables().size());
- assertFileCounts(s.descriptor.directory.list(), 0, 0);
+ assertFileCounts(s.descriptor.directory.list());
assertEquals(cfs.getSSTables().iterator().next().first, origFirst);
assertEquals(cfs.getSSTables().iterator().next().last, origLast);
validateCFS(cfs);
@@ -522,13 +546,13 @@ public class SSTableRewriterTest extends SchemaLoader
SSTableRewriter rewriter = new SSTableRewriter(cfs, txn, 1000, false, 10000000);
CompactionIterator ci = new CompactionIterator(OperationType.COMPACTION, Collections.singletonList(scanner), controller, FBUtilities.nowInSeconds(), UUIDGen.getTimeUUID()))
{
- rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
+ rewriter.switchWriter(getWriter(cfs, s.descriptor.directory, txn));
while(ci.hasNext())
{
rewriter.append(ci.next());
if (rewriter.currentWriter().getFilePointer() > 2500000)
{
- rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
+ rewriter.switchWriter(getWriter(cfs, s.descriptor.directory, txn));
files++;
assertEquals(cfs.getSSTables().size(), files); // we have one original file plus the ones we have switched out.
}
@@ -541,10 +565,10 @@ public class SSTableRewriterTest extends SchemaLoader
}
}
- SSTableDeletingTask.waitForDeletions();
+ TransactionLogs.waitForDeletions();
assertEquals(files - 1, cfs.getSSTables().size()); // we never wrote anything to the last file
- assertFileCounts(s.descriptor.directory.list(), 0, 0);
+ assertFileCounts(s.descriptor.directory.list());
validateCFS(cfs);
}
@@ -568,13 +592,13 @@ public class SSTableRewriterTest extends SchemaLoader
SSTableRewriter rewriter = new SSTableRewriter(cfs, txn, 1000, false, 10000000);
CompactionIterator ci = new CompactionIterator(OperationType.COMPACTION, Collections.singletonList(scanner), controller, FBUtilities.nowInSeconds(), UUIDGen.getTimeUUID()))
{
- rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
+ rewriter.switchWriter(getWriter(cfs, s.descriptor.directory, txn));
while(ci.hasNext())
{
rewriter.append(ci.next());
if (rewriter.currentWriter().getOnDiskFilePointer() > 25000000)
{
- rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
+ rewriter.switchWriter(getWriter(cfs, s.descriptor.directory, txn));
files++;
assertEquals(cfs.getSSTables().size(), files); // we have one original file plus the ones we have switched out.
}
@@ -583,8 +607,8 @@ public class SSTableRewriterTest extends SchemaLoader
sstables = rewriter.finish();
}
- SSTableDeletingTask.waitForDeletions();
- assertFileCounts(s.descriptor.directory.list(), 0, 0);
+ TransactionLogs.waitForDeletions();
+ assertFileCounts(s.descriptor.directory.list());
validateCFS(cfs);
}
@@ -608,14 +632,14 @@ public class SSTableRewriterTest extends SchemaLoader
SSTableRewriter rewriter = new SSTableRewriter(cfs, txn, 1000, false, 1000000);
CompactionIterator ci = new CompactionIterator(OperationType.COMPACTION, Collections.singletonList(scanner), controller, FBUtilities.nowInSeconds(), UUIDGen.getTimeUUID()))
{
- rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
+ rewriter.switchWriter(getWriter(cfs, s.descriptor.directory, txn));
while(ci.hasNext())
{
rewriter.append(ci.next());
if (rewriter.currentWriter().getOnDiskFilePointer() > 2500000)
{
assertEquals(files, cfs.getSSTables().size()); // all files are now opened early
- rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
+ rewriter.switchWriter(getWriter(cfs, s.descriptor.directory, txn));
files++;
}
}
@@ -624,8 +648,8 @@ public class SSTableRewriterTest extends SchemaLoader
}
assertEquals(files, sstables.size());
assertEquals(files, cfs.getSSTables().size());
- SSTableDeletingTask.waitForDeletions();
- assertFileCounts(s.descriptor.directory.list(), 0, 0);
+ TransactionLogs.waitForDeletions();
+ assertFileCounts(s.descriptor.directory.list());
validateCFS(cfs);
}
@@ -643,10 +667,10 @@ public class SSTableRewriterTest extends SchemaLoader
SSTableSplitter splitter = new SSTableSplitter(cfs, txn, 10);
splitter.split();
- assertFileCounts(s.descriptor.directory.list(), 0, 0);
+ assertFileCounts(s.descriptor.directory.list());
s.selfRef().release();
- SSTableDeletingTask.waitForDeletions();
+ TransactionLogs.waitForDeletions();
for (File f : s.descriptor.directory.listFiles())
{
@@ -697,13 +721,13 @@ public class SSTableRewriterTest extends SchemaLoader
CompactionIterator ci = new CompactionIterator(OperationType.COMPACTION, Collections.singletonList(scanner), controller, FBUtilities.nowInSeconds(), UUIDGen.getTimeUUID())
)
{
- rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
+ rewriter.switchWriter(getWriter(cfs, s.descriptor.directory, txn));
while (ci.hasNext())
{
rewriter.append(ci.next());
if (rewriter.currentWriter().getOnDiskFilePointer() > 25000000)
{
- rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
+ rewriter.switchWriter(getWriter(cfs, s.descriptor.directory, txn));
}
}
try
@@ -722,9 +746,9 @@ public class SSTableRewriterTest extends SchemaLoader
s.selfRef().release();
}
- SSTableDeletingTask.waitForDeletions();
+ TransactionLogs.waitForDeletions();
- int filecount = assertFileCounts(s.descriptor.directory.list(), 0, 0);
+ int filecount = assertFileCounts(s.descriptor.directory.list());
assertEquals(filecount, 1);
if (!offline)
{
@@ -737,7 +761,7 @@ public class SSTableRewriterTest extends SchemaLoader
assertEquals(0, cfs.getSSTables().size());
cfs.truncateBlocking();
}
- filecount = assertFileCounts(s.descriptor.directory.list(), 0, 0);
+ filecount = assertFileCounts(s.descriptor.directory.list());
if (offline)
{
// the file is not added to the CFS, therefore not truncated away above
@@ -746,7 +770,7 @@ public class SSTableRewriterTest extends SchemaLoader
{
FileUtils.deleteRecursive(f);
}
- filecount = assertFileCounts(s.descriptor.directory.list(), 0, 0);
+ filecount = assertFileCounts(s.descriptor.directory.list());
}
assertEquals(0, filecount);
@@ -787,13 +811,13 @@ public class SSTableRewriterTest extends SchemaLoader
CompactionIterator ci = new CompactionIterator(OperationType.COMPACTION, Collections.singletonList(scanner), controller, FBUtilities.nowInSeconds(), UUIDGen.getTimeUUID())
)
{
- rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
+ rewriter.switchWriter(getWriter(cfs, s.descriptor.directory, txn));
while (ci.hasNext())
{
rewriter.append(ci.next());
if (keyCount % 10 == 0)
{
- rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
+ rewriter.switchWriter(getWriter(cfs, s.descriptor.directory, txn));
}
keyCount++;
validateKeys(keyspace);
@@ -801,7 +825,7 @@ public class SSTableRewriterTest extends SchemaLoader
rewriter.finish();
}
validateKeys(keyspace);
- SSTableDeletingTask.waitForDeletions();
+ TransactionLogs.waitForDeletions();
validateCFS(cfs);
truncate(cfs);
}
@@ -825,7 +849,7 @@ public class SSTableRewriterTest extends SchemaLoader
CompactionIterator ci = new CompactionIterator(OperationType.COMPACTION, Collections.singletonList(scanner), controller, FBUtilities.nowInSeconds(), UUIDGen.getTimeUUID())
)
{
- writer.switchWriter(getWriter(cfs, sstables.iterator().next().descriptor.directory));
+ writer.switchWriter(getWriter(cfs, sstables.iterator().next().descriptor.directory, txn));
while (ci.hasNext())
{
writer.append(ci.next());
@@ -870,8 +894,8 @@ public class SSTableRewriterTest extends SchemaLoader
CompactionIterator ci = new CompactionIterator(OperationType.COMPACTION, scanners.scanners, controller, nowInSec, UUIDGen.getTimeUUID())
)
{
- writer.switchWriter(getWriter(cfs, sstables.iterator().next().descriptor.directory));
- writer2.switchWriter(getWriter(cfs, sstables.iterator().next().descriptor.directory));
+ writer.switchWriter(getWriter(cfs, sstables.iterator().next().descriptor.directory, txn));
+ writer2.switchWriter(getWriter(cfs, sstables.iterator().next().descriptor.directory, txn));
while (ci.hasNext())
{
if (writer.currentWriter().getFilePointer() < 15000000)
@@ -886,7 +910,6 @@ public class SSTableRewriterTest extends SchemaLoader
validateCFS(cfs);
}
-
private void validateKeys(Keyspace ks)
{
for (int i = 0; i < 100; i++)
@@ -900,8 +923,8 @@ public class SSTableRewriterTest extends SchemaLoader
public static void truncate(ColumnFamilyStore cfs)
{
cfs.truncateBlocking();
- SSTableDeletingTask.waitForDeletions();
- Uninterruptibles.sleepUninterruptibly(10L,TimeUnit.MILLISECONDS);
+ TransactionLogs.waitForDeletions();
+ Uninterruptibles.sleepUninterruptibly(10L, TimeUnit.MILLISECONDS);
assertEquals(0, cfs.metric.liveDiskSpaceUsed.getCount());
assertEquals(0, cfs.metric.totalDiskSpaceUsed.getCount());
validateCFS(cfs);
@@ -919,9 +942,9 @@ public class SSTableRewriterTest extends SchemaLoader
for (int f = 0 ; f < fileCount ; f++)
{
File dir = cfs.directories.getDirectoryForNewSSTables();
- String filename = cfs.getTempSSTablePath(dir);
+ String filename = cfs.getSSTablePath(dir);
- try (SSTableWriter writer = SSTableWriter.create(filename, 0, 0, new SerializationHeader(cfs.metadata, cfs.metadata.partitionColumns(), EncodingStats.NO_STATS)))
+ try (SSTableTxnWriter writer = SSTableTxnWriter.create(filename, 0, 0, new SerializationHeader(cfs.metadata, cfs.metadata.partitionColumns(), EncodingStats.NO_STATS)))
{
int end = f == fileCount - 1 ? partitionCount : ((f + 1) * partitionCount) / fileCount;
for ( ; i < end ; i++)
@@ -965,7 +988,7 @@ public class SSTableRewriterTest extends SchemaLoader
assertTrue(cfs.getTracker().getCompacting().isEmpty());
}
- public static int assertFileCounts(String [] files, int expectedtmplinkCount, int expectedtmpCount)
+ public static int assertFileCounts(String [] files)
{
int tmplinkcount = 0;
int tmpcount = 0;
@@ -981,15 +1004,15 @@ public class SSTableRewriterTest extends SchemaLoader
else if (f.contains("Data"))
datacount++;
}
- assertEquals(expectedtmplinkCount, tmplinkcount);
- assertEquals(expectedtmpCount, tmpcount);
+ assertEquals(0, tmplinkcount);
+ assertEquals(0, tmpcount);
return datacount;
}
- public static SSTableWriter getWriter(ColumnFamilyStore cfs, File directory)
+ public static SSTableWriter getWriter(ColumnFamilyStore cfs, File directory, LifecycleTransaction txn)
{
- String filename = cfs.getTempSSTablePath(directory);
- return SSTableWriter.create(filename, 0, 0, new SerializationHeader(cfs.metadata, cfs.metadata.partitionColumns(), EncodingStats.NO_STATS));
+ String filename = cfs.getSSTablePath(directory);
+ return SSTableWriter.create(filename, 0, 0, new SerializationHeader(cfs.metadata, cfs.metadata.partitionColumns(), EncodingStats.NO_STATS), txn);
}
public static ByteBuffer random(int i, int size)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java b/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
index 2c8377f..6de5bb9 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
@@ -28,10 +28,7 @@ import org.apache.cassandra.db.*;
import org.apache.cassandra.db.rows.*;
import org.apache.cassandra.db.partitions.*;
import org.apache.cassandra.io.sstable.format.SSTableReader;
-import org.apache.cassandra.io.sstable.format.SSTableWriter;
import org.apache.cassandra.service.ActiveRepairService;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.Util;
import static org.junit.Assert.assertEquals;
@@ -78,7 +75,7 @@ public class SSTableUtils
File cfDir = new File(tempdir, keyspaceName + File.separator + cfname);
cfDir.mkdirs();
cfDir.deleteOnExit();
- File datafile = new File(new Descriptor(cfDir, keyspaceName, cfname, generation, Descriptor.Type.FINAL).filenameFor("Data.db"));
+ File datafile = new File(new Descriptor(cfDir, keyspaceName, cfname, generation).filenameFor("Data.db"));
if (!datafile.createNewFile())
throw new IOException("unable to create file " + datafile);
datafile.deleteOnExit();
@@ -185,7 +182,7 @@ public class SSTableUtils
return write(sorted.size(), new Appender()
{
@Override
- public boolean append(SSTableWriter writer) throws IOException
+ public boolean append(SSTableTxnWriter writer) throws IOException
{
if (!iter.hasNext())
return false;
@@ -208,7 +205,7 @@ public class SSTableUtils
{
File datafile = (dest == null) ? tempSSTableFile(ksname, cfname, generation) : new File(dest.filenameFor(Component.DATA));
SerializationHeader header = SerializationHeader.make(Schema.instance.getCFMetaData(ksname, cfname), Collections.EMPTY_LIST);
- SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(datafile.getAbsolutePath()), expectedSize, ActiveRepairService.UNREPAIRED_SSTABLE, 0, header);
+ SSTableTxnWriter writer = SSTableTxnWriter.create(datafile.getAbsolutePath(), expectedSize, ActiveRepairService.UNREPAIRED_SSTABLE, 0, header);
while (appender.append(writer)) { /* pass */ }
SSTableReader reader = writer.finish(true);
// mark all components for removal
@@ -222,6 +219,6 @@ public class SSTableUtils
public static abstract class Appender
{
/** Called with an open writer until it returns false. */
- public abstract boolean append(SSTableWriter writer) throws IOException;
+ public abstract boolean append(SSTableTxnWriter writer) throws IOException;
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java b/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
index 7051bd3..c763932 100644
--- a/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
@@ -74,7 +74,7 @@ public class MetadataSerializerTest
serializer.serialize(originalMetadata, out);
}
- Descriptor desc = new Descriptor( statsFile.getParentFile(), "", "", 0, Descriptor.Type.FINAL);
+ Descriptor desc = new Descriptor( statsFile.getParentFile(), "", "", 0);
try (RandomAccessReader in = RandomAccessReader.open(statsFile))
{
Map<MetadataType, MetadataComponent> deserialized = serializer.deserialize(desc, in, EnumSet.allOf(MetadataType.class));
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/unit/org/apache/cassandra/schema/DefsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/schema/DefsTest.java b/test/unit/org/apache/cassandra/schema/DefsTest.java
index b567bb5..2bfd6ae 100644
--- a/test/unit/org/apache/cassandra/schema/DefsTest.java
+++ b/test/unit/org/apache/cassandra/schema/DefsTest.java
@@ -39,12 +39,12 @@ import org.apache.cassandra.cql3.QueryProcessor;
import org.apache.cassandra.cql3.UntypedResultSet;
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.lifecycle.TransactionLogs;
import org.apache.cassandra.db.marshal.BytesType;
import org.apache.cassandra.db.marshal.UTF8Type;
import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.io.sstable.Component;
import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.SSTableDeletingTask;
import org.apache.cassandra.locator.OldNetworkTopologyStrategy;
import org.apache.cassandra.service.MigrationManager;
import org.apache.cassandra.utils.ByteBufferUtil;
@@ -536,7 +536,7 @@ public class DefsTest
// check
assertTrue(cfs.indexManager.getIndexes().isEmpty());
- SSTableDeletingTask.waitForDeletions();
+ TransactionLogs.waitForDeletions();
assertFalse(new File(desc.filenameFor(Component.DATA)).exists());
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java b/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
index 6227a1f..875c306 100644
--- a/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
+++ b/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
@@ -87,7 +87,7 @@ public class StreamTransferTaskTest
f.get();
// when timeout runs on second file, task should be completed
- f = task.scheduleTimeout(1, 1, TimeUnit.MILLISECONDS);
+ f = task.scheduleTimeout(1, 10, TimeUnit.MILLISECONDS);
task.complete(1);
try
{
@@ -97,6 +97,7 @@ public class StreamTransferTaskTest
catch (CancellationException ex)
{
}
+
assertEquals(StreamSession.State.WAIT_COMPLETE, session.state());
// when all streaming are done, time out task should not be scheduled.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/unit/org/apache/cassandra/utils/concurrent/AbstractTransactionalTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/concurrent/AbstractTransactionalTest.java b/test/unit/org/apache/cassandra/utils/concurrent/AbstractTransactionalTest.java
index 4e160c2..f0c850d 100644
--- a/test/unit/org/apache/cassandra/utils/concurrent/AbstractTransactionalTest.java
+++ b/test/unit/org/apache/cassandra/utils/concurrent/AbstractTransactionalTest.java
@@ -87,9 +87,29 @@ public abstract class AbstractTransactionalTest
txn = newTest();
Throwable t = new RuntimeException();
txn.testing.prepareToCommit();
- Assert.assertEquals(t, txn.testing.commit(t));
- Assert.assertEquals(t, txn.testing.abort(t));
- Assert.assertTrue(t.getSuppressed()[0] instanceof IllegalStateException);
+
+ if (txn.commitCanThrow())
+ {
+ try
+ {
+ txn.testing.commit(t);
+ }
+ catch (Throwable tt)
+ {
+ Assert.assertEquals(t, tt);
+ }
+
+ Assert.assertEquals(t, txn.testing.abort(t));
+ Assert.assertEquals(0, t.getSuppressed().length);
+ }
+ else
+ {
+ Assert.assertEquals(t, txn.testing.commit(t));
+ Assert.assertEquals(t, txn.testing.abort(t));
+ Assert.assertTrue(t.getSuppressed()[0] instanceof IllegalStateException);
+ }
+
+
}
@Test
@@ -132,5 +152,10 @@ public abstract class AbstractTransactionalTest
protected abstract void assertPrepared() throws Exception;
protected abstract void assertAborted() throws Exception;
protected abstract void assertCommitted() throws Exception;
+
+ protected boolean commitCanThrow()
+ {
+ return false;
+ }
}
}
[2/5] cassandra git commit: Introduce safer durable sstable
membership management (and simplify cleanup of compaction leftovers)
Posted by be...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java b/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java
index fcec40d..b49055d 100644
--- a/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java
+++ b/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java
@@ -30,8 +30,6 @@ import org.junit.BeforeClass;
import org.junit.Test;
import org.apache.cassandra.SchemaLoader;
import org.apache.cassandra.config.Config;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.db.Keyspace;
import org.apache.cassandra.service.StorageService;
public class CQLSSTableWriterLongTest
@@ -39,8 +37,7 @@ public class CQLSSTableWriterLongTest
@BeforeClass
public static void setup() throws Exception
{
- SchemaLoader.cleanupAndLeaveDirs();
- Keyspace.setInitialized();
+ SchemaLoader.prepareServer();
StorageService.instance.initServer();
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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
index 104e88f..d9c7e8b 100644
--- a/test/unit/org/apache/cassandra/MockSchema.java
+++ b/test/unit/org/apache/cassandra/MockSchema.java
@@ -60,7 +60,7 @@ public class MockSchema
private static final AtomicInteger id = new AtomicInteger();
public static final Keyspace ks = Keyspace.mockKS(KeyspaceMetadata.create("mockks", KeyspaceParams.simpleTransient(1)));
- private static final IndexSummary indexSummary;
+ public static final IndexSummary indexSummary;
private static final SegmentedFile segmentedFile = new BufferedSegmentedFile(new ChannelProxy(temp("mocksegmentedfile")), 0);
public static Memtable memtable(ColumnFamilyStore cfs)
@@ -88,8 +88,7 @@ public class MockSchema
Descriptor descriptor = new Descriptor(cfs.directories.getDirectoryForNewSSTables(),
cfs.keyspace.getName(),
cfs.getColumnFamilyName(),
- generation,
- Descriptor.Type.FINAL);
+ generation);
Set<Component> components = ImmutableSet.of(Component.DATA, Component.PRIMARY_INDEX, Component.FILTER, Component.TOC);
for (Component component : components)
{
@@ -132,8 +131,13 @@ public class MockSchema
public static ColumnFamilyStore newCFS()
{
+ return newCFS(ks.getName());
+ }
+
+ public static ColumnFamilyStore newCFS(String ksname)
+ {
String cfname = "mockcf" + (id.incrementAndGet());
- CFMetaData metadata = newCFMetaData(ks.getName(), cfname);
+ CFMetaData metadata = newCFMetaData(ksname, cfname);
return new ColumnFamilyStore(ks, cfname, Murmur3Partitioner.instance, 0, metadata, new Directories(metadata), false, false);
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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 654b8c6..c828de9 100644
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@ -53,12 +53,9 @@ import org.apache.cassandra.dht.Token;
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.service.StorageService;
-import org.apache.cassandra.utils.AlwaysPresentFilter;
import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.utils.CounterId;
import org.apache.cassandra.utils.FBUtilities;
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java
index 06b5ceb..ddc41c7 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -282,7 +282,7 @@ public abstract class CQLTester
schemaChange(String.format("DROP TYPE IF EXISTS %s.%s", KEYSPACE, typesToDrop.get(i)));
// Dropping doesn't delete the sstables. It's not a huge deal but it's cleaner to cleanup after us
- // Thas said, we shouldn't delete blindly before the SSTableDeletingTask for the table we drop
+ // Thas said, we shouldn't delete blindly before the TransactionLogs.SSTableTidier for the table we drop
// have run or they will be unhappy. Since those taks are scheduled on StorageService.tasks and that's
// mono-threaded, just push a task on the queue to find when it's empty. No perfect but good enough.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
index 9da4876..2a15bdf 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
@@ -49,7 +49,7 @@ import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.Pair;
import org.apache.cassandra.utils.WrappedRunnable;
-
+import static junit.framework.Assert.assertNotNull;
@RunWith(OrderedJUnit4ClassRunner.class)
public class ColumnFamilyStoreTest
{
@@ -349,8 +349,8 @@ public class ColumnFamilyStoreTest
for (int version = 1; version <= 2; ++version)
{
- Descriptor existing = new Descriptor(cfs.directories.getDirectoryForNewSSTables(), KEYSPACE2, CF_STANDARD1, version, Descriptor.Type.FINAL);
- Descriptor desc = new Descriptor(Directories.getBackupsDirectory(existing), KEYSPACE2, CF_STANDARD1, version, Descriptor.Type.FINAL);
+ Descriptor existing = new Descriptor(cfs.directories.getDirectoryForNewSSTables(), KEYSPACE2, CF_STANDARD1, version);
+ Descriptor desc = new Descriptor(Directories.getBackupsDirectory(existing), KEYSPACE2, CF_STANDARD1, version);
for (Component c : new Component[]{ Component.DATA, Component.PRIMARY_INDEX, Component.FILTER, Component.STATS })
assertTrue("Cannot find backed-up file:" + desc.filenameFor(c), new File(desc.filenameFor(c)).exists());
}
@@ -389,126 +389,10 @@ public class ColumnFamilyStoreTest
// assertEquals(ByteBufferUtil.bytes("B"), cfSliced.getColumn(CellNames.compositeDense(superColName, ByteBufferUtil.bytes("b"))).value());
// }
- // TODO: fix once SSTableSimpleWriter's back in
-// @Test
-// public void testRemoveUnfinishedCompactionLeftovers() throws Throwable
-// {
-// String ks = KEYSPACE1;
-// String cf = CF_STANDARD3; // should be empty
-//
-// final CFMetaData cfmeta = Schema.instance.getCFMetaData(ks, cf);
-// Directories dir = new Directories(cfmeta);
-// ByteBuffer key = bytes("key");
-//
-// // 1st sstable
-// SSTableSimpleWriter writer = new SSTableSimpleWriter(dir.getDirectoryForNewSSTables(), cfmeta, StorageService.getPartitioner());
-// writer.newRow(key);
-// writer.addColumn(bytes("col"), bytes("val"), 1);
-// writer.close();
-//
-// Map<Descriptor, Set<Component>> sstables = dir.sstableLister().list();
-// assertEquals(1, sstables.size());
-//
-// Map.Entry<Descriptor, Set<Component>> sstableToOpen = sstables.entrySet().iterator().next();
-// final SSTableReader sstable1 = SSTableReader.open(sstableToOpen.getKey());
-//
-// // simulate incomplete compaction
-// writer = new SSTableSimpleWriter(dir.getDirectoryForNewSSTables(),
-// cfmeta, StorageService.getPartitioner())
-// {
-// protected SSTableWriter getWriter()
-// {
-// MetadataCollector collector = new MetadataCollector(cfmeta.comparator);
-// collector.addAncestor(sstable1.descriptor.generation); // add ancestor from previously written sstable
-// return SSTableWriter.create(createDescriptor(directory, metadata.ksName, metadata.cfName, DatabaseDescriptor.getSSTableFormat()),
-// 0L,
-// ActiveRepairService.UNREPAIRED_SSTABLE,
-// metadata,
-// DatabaseDescriptor.getPartitioner(),
-// collector);
-// }
-// };
-// writer.newRow(key);
-// writer.addColumn(bytes("col"), bytes("val"), 1);
-// writer.close();
-//
-// // should have 2 sstables now
-// sstables = dir.sstableLister().list();
-// assertEquals(2, sstables.size());
-//
-// SSTableReader sstable2 = SSTableReader.open(sstable1.descriptor);
-// UUID compactionTaskID = SystemKeyspace.startCompaction(
-// Keyspace.open(ks).getColumnFamilyStore(cf),
-// Collections.singleton(sstable2));
-//
-// Map<Integer, UUID> unfinishedCompaction = new HashMap<>();
-// unfinishedCompaction.put(sstable1.descriptor.generation, compactionTaskID);
-// ColumnFamilyStore.removeUnfinishedCompactionLeftovers(cfmeta, unfinishedCompaction);
-//
-// // 2nd sstable should be removed (only 1st sstable exists in set of size 1)
-// sstables = dir.sstableLister().list();
-// assertEquals(1, sstables.size());
-// assertTrue(sstables.containsKey(sstable1.descriptor));
-//
-// Map<Pair<String, String>, Map<Integer, UUID>> unfinished = SystemKeyspace.getUnfinishedCompactions();
-// assertTrue(unfinished.isEmpty());
-// sstable1.selfRef().release();
-// sstable2.selfRef().release();
-// }
// TODO: Fix once SSTableSimpleWriter's back in
// @see <a href="https://issues.apache.org/jira/browse/CASSANDRA-6086">CASSANDRA-6086</a>
-// @Test
-// public void testFailedToRemoveUnfinishedCompactionLeftovers() throws Throwable
-// {
-// final String ks = KEYSPACE1;
-// final String cf = CF_STANDARD4; // should be empty
-//
-// final CFMetaData cfmeta = Schema.instance.getCFMetaData(ks, cf);
-// Directories dir = new Directories(cfmeta);
-// ByteBuffer key = bytes("key");
-//
-// // Write SSTable generation 3 that has ancestors 1 and 2
-// final Set<Integer> ancestors = Sets.newHashSet(1, 2);
-// SSTableSimpleWriter writer = new SSTableSimpleWriter(dir.getDirectoryForNewSSTables(),
-// cfmeta, StorageService.getPartitioner())
-// {
-// protected SSTableWriter getWriter()
-// {
-// MetadataCollector collector = new MetadataCollector(cfmeta.comparator);
-// for (int ancestor : ancestors)
-// collector.addAncestor(ancestor);
-// String file = new Descriptor(directory, ks, cf, 3, Descriptor.Type.TEMP).filenameFor(Component.DATA);
-// return SSTableWriter.create(Descriptor.fromFilename(file),
-// 0L,
-// ActiveRepairService.UNREPAIRED_SSTABLE,
-// metadata,
-// StorageService.getPartitioner(),
-// collector);
-// }
-// };
-// writer.newRow(key);
-// writer.addColumn(bytes("col"), bytes("val"), 1);
-// writer.close();
-//
-// Map<Descriptor, Set<Component>> sstables = dir.sstableLister().list();
-// assert sstables.size() == 1;
-//
-// Map.Entry<Descriptor, Set<Component>> sstableToOpen = sstables.entrySet().iterator().next();
-// final SSTableReader sstable1 = SSTableReader.open(sstableToOpen.getKey());
-//
-// // simulate we don't have generation in compaction_history
-// Map<Integer, UUID> unfinishedCompactions = new HashMap<>();
-// UUID compactionTaskID = UUID.randomUUID();
-// for (Integer ancestor : ancestors)
-// unfinishedCompactions.put(ancestor, compactionTaskID);
-// ColumnFamilyStore.removeUnfinishedCompactionLeftovers(cfmeta, unfinishedCompactions);
-//
-// // SSTable should not be deleted
-// sstables = dir.sstableLister().list();
-// assert sstables.size() == 1;
-// assert sstables.containsKey(sstable1.descriptor);
-// }
+
// TODO: Fix once SSTableSimpleWriter's back in
// @Test
@@ -625,4 +509,30 @@ public class ColumnFamilyStoreTest
}
assertEquals(count, found);
}
+
+ @Test
+ public void testScrubDataDirectories() throws Throwable
+ {
+ ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1);
+
+ ColumnFamilyStore.scrubDataDirectories(cfs.metadata);
+
+ new RowUpdateBuilder(cfs.metadata, 2, "key").clustering("name").add("val", "2").build().applyUnsafe();
+ cfs.forceBlockingFlush();
+
+ // Nuke the metadata and reload that sstable
+ Collection<SSTableReader> ssTables = cfs.getSSTables();
+ assertEquals(1, ssTables.size());
+ SSTableReader ssTable = ssTables.iterator().next();
+
+ String dataFileName = ssTable.descriptor.filenameFor(Component.DATA);
+ String tmpDataFileName = ssTable.descriptor.tmpFilenameFor(Component.DATA);
+ new File(dataFileName).renameTo(new File(tmpDataFileName));
+
+ ColumnFamilyStore.scrubDataDirectories(cfs.metadata);
+
+ List<File> ssTableFiles = new Directories(cfs.metadata).sstableLister().listFiles();
+ assertNotNull(ssTableFiles);
+ assertEquals(0, ssTableFiles.size());
+ }
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/unit/org/apache/cassandra/db/DirectoriesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/DirectoriesTest.java b/test/unit/org/apache/cassandra/db/DirectoriesTest.java
index 6f3ccc9..14db2d1 100644
--- a/test/unit/org/apache/cassandra/db/DirectoriesTest.java
+++ b/test/unit/org/apache/cassandra/db/DirectoriesTest.java
@@ -95,22 +95,22 @@ public class DirectoriesTest
File dir = cfDir(cfm);
dir.mkdirs();
- createFakeSSTable(dir, cfm.cfName, 1, false, fs);
- createFakeSSTable(dir, cfm.cfName, 2, true, fs);
+ createFakeSSTable(dir, cfm.cfName, 1, fs);
+ createFakeSSTable(dir, cfm.cfName, 2, fs);
File backupDir = new File(dir, Directories.BACKUPS_SUBDIR);
backupDir.mkdir();
- createFakeSSTable(backupDir, cfm.cfName, 1, false, fs);
+ createFakeSSTable(backupDir, cfm.cfName, 1, fs);
File snapshotDir = new File(dir, Directories.SNAPSHOT_SUBDIR + File.separator + "42");
snapshotDir.mkdirs();
- createFakeSSTable(snapshotDir, cfm.cfName, 1, false, fs);
+ createFakeSSTable(snapshotDir, cfm.cfName, 1, fs);
}
}
- private static void createFakeSSTable(File dir, String cf, int gen, boolean temp, List<File> addTo) throws IOException
+ private static void createFakeSSTable(File dir, String cf, int gen, List<File> addTo) throws IOException
{
- Descriptor desc = new Descriptor(dir, KS, cf, gen, temp ? Descriptor.Type.TEMP : Descriptor.Type.FINAL);
+ Descriptor desc = new Descriptor(dir, KS, cf, gen);
for (Component c : new Component[]{ Component.DATA, Component.PRIMARY_INDEX, Component.FILTER })
{
File f = new File(desc.filenameFor(c));
@@ -145,7 +145,7 @@ public class DirectoriesTest
Directories directories = new Directories(cfm);
assertEquals(cfDir(cfm), directories.getDirectoryForNewSSTables());
- Descriptor desc = new Descriptor(cfDir(cfm), KS, cfm.cfName, 1, Descriptor.Type.FINAL);
+ Descriptor desc = new Descriptor(cfDir(cfm), KS, cfm.cfName, 1);
File snapshotDir = new File(cfDir(cfm), File.separator + Directories.SNAPSHOT_SUBDIR + File.separator + "42");
assertEquals(snapshotDir, Directories.getSnapshotDirectory(desc, "42"));
@@ -173,8 +173,8 @@ public class DirectoriesTest
{
assertEquals(cfDir(INDEX_CFM), dir);
}
- Descriptor parentDesc = new Descriptor(parentDirectories.getDirectoryForNewSSTables(), KS, PARENT_CFM.cfName, 0, Descriptor.Type.FINAL);
- Descriptor indexDesc = new Descriptor(indexDirectories.getDirectoryForNewSSTables(), KS, INDEX_CFM.cfName, 0, Descriptor.Type.FINAL);
+ Descriptor parentDesc = new Descriptor(parentDirectories.getDirectoryForNewSSTables(), KS, PARENT_CFM.cfName, 0);
+ Descriptor indexDesc = new Descriptor(indexDirectories.getDirectoryForNewSSTables(), KS, INDEX_CFM.cfName, 0);
// snapshot dir should be created under its parent's
File parentSnapshotDirectory = Directories.getSnapshotDirectory(parentDesc, "test");
@@ -191,9 +191,9 @@ public class DirectoriesTest
indexDirectories.snapshotCreationTime("test"));
// check true snapshot size
- Descriptor parentSnapshot = new Descriptor(parentSnapshotDirectory, KS, PARENT_CFM.cfName, 0, Descriptor.Type.FINAL);
+ Descriptor parentSnapshot = new Descriptor(parentSnapshotDirectory, KS, PARENT_CFM.cfName, 0);
createFile(parentSnapshot.filenameFor(Component.DATA), 30);
- Descriptor indexSnapshot = new Descriptor(indexSnapshotDirectory, KS, INDEX_CFM.cfName, 0, Descriptor.Type.FINAL);
+ Descriptor indexSnapshot = new Descriptor(indexSnapshotDirectory, KS, INDEX_CFM.cfName, 0);
createFile(indexSnapshot.filenameFor(Component.DATA), 40);
assertEquals(30, parentDirectories.trueSnapshotsSize());
@@ -311,7 +311,7 @@ public class DirectoriesTest
final String n = Long.toString(System.nanoTime());
Callable<File> directoryGetter = new Callable<File>() {
public File call() throws Exception {
- Descriptor desc = new Descriptor(cfDir(cfm), KS, cfm.cfName, 1, Descriptor.Type.FINAL);
+ Descriptor desc = new Descriptor(cfDir(cfm), KS, cfm.cfName, 1);
return Directories.getSnapshotDirectory(desc, n);
}
};
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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 47bfa0c..181b4e0 100644
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@ -65,6 +65,7 @@ import org.apache.cassandra.OrderedJUnit4ClassRunner;
import org.apache.cassandra.SchemaLoader;
import org.apache.cassandra.service.StorageService;
import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.OutputHandler;
import static org.junit.Assert.*;
import static org.junit.Assume.assumeTrue;
@@ -315,7 +316,7 @@ public class ScrubTest
* Code used to generate an outOfOrder sstable. The test for out-of-order key in BigTableWriter must also be commented out.
* The test also assumes an ordered partitioner.
List<String> keys = Arrays.asList("t", "a", "b", "z", "c", "y", "d");
- String filename = cfs.getTempSSTablePath(new File(System.getProperty("corrupt-sstable-root")));
+ SSTableWriter writer = new SSTableWriter(cfs.getTempSSTablePath(new File(System.getProperty("corrupt-sstable-root"))),
SSTableWriter writer = SSTableWriter.create(cfs.metadata,
Descriptor.fromFilename(filename),
keys.size(),
@@ -341,7 +342,7 @@ public class ScrubTest
File rootDir = new File(root);
assert rootDir.isDirectory();
- Descriptor desc = new Descriptor("la", rootDir, KEYSPACE, columnFamily, 1, Descriptor.Type.FINAL, SSTableFormat.Type.BIG);
+ Descriptor desc = new Descriptor("la", rootDir, KEYSPACE, columnFamily, 1, SSTableFormat.Type.BIG);
CFMetaData metadata = Schema.instance.getCFMetaData(desc.ksname, desc.cfname);
try
@@ -366,7 +367,7 @@ public class ScrubTest
sstable.last = sstable.first;
try (LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.SCRUB, sstable);
- Scrubber scrubber = new Scrubber(cfs, txn, false, true, true);)
+ Scrubber scrubber = new Scrubber(cfs, txn, false, new OutputHandler.LogOutput(), true, true, true))
{
scrubber.scrub();
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java b/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
index 9ea7f93..d58985c 100644
--- a/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
+++ b/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
@@ -17,11 +17,15 @@
*/
package org.apache.cassandra.db;
+import java.io.File;
import java.io.IOException;
import java.net.InetAddress;
import java.net.UnknownHostException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
import java.util.*;
+import org.apache.commons.io.FileUtils;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -30,6 +34,7 @@ import org.apache.cassandra.cql3.QueryProcessor;
import org.apache.cassandra.cql3.UntypedResultSet;
import org.apache.cassandra.dht.ByteOrderedPartitioner.BytesToken;
import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.Descriptor;
import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.CassandraVersion;
@@ -39,6 +44,8 @@ import static org.junit.Assert.assertTrue;
public class SystemKeyspaceTest
{
+ public static final String MIGRATION_SSTABLES_ROOT = "migration-sstable-root";
+
@BeforeClass
public static void prepSnapshotTracker()
{
@@ -145,6 +152,47 @@ public class SystemKeyspaceTest
Keyspace.clearSnapshot(null, SystemKeyspace.NAME);
}
+ @Test
+ public void testMigrateDataDirs() throws IOException
+ {
+ Path migrationSSTableRoot = Paths.get(System.getProperty(MIGRATION_SSTABLES_ROOT), "2.2");
+ Path dataDir = Paths.get(DatabaseDescriptor.getAllDataFileLocations()[0]);
+
+ FileUtils.copyDirectory(migrationSSTableRoot.toFile(), dataDir.toFile());
+
+ assertEquals(5, numLegacyFiles()); // see test data
+
+ SystemKeyspace.migrateDataDirs();
+
+ assertEquals(0, numLegacyFiles());
+ }
+
+ private static int numLegacyFiles()
+ {
+ int ret = 0;
+ Iterable<String> dirs = Arrays.asList(DatabaseDescriptor.getAllDataFileLocations());
+ for (String dataDir : dirs)
+ {
+ File dir = new File(dataDir);
+ for (File ksdir : dir.listFiles((d, n) -> d.isDirectory()))
+ {
+ for (File cfdir : ksdir.listFiles((d, n) -> d.isDirectory()))
+ {
+ if (Descriptor.isLegacyFile(cfdir.getName()))
+ {
+ ret++;
+ }
+ else
+ {
+ File[] legacyFiles = cfdir.listFiles((d, n) -> Descriptor.isLegacyFile(n));
+ ret += legacyFiles.length;
+ }
+ }
+ }
+ }
+ return ret;
+ }
+
private String getOlderVersionString()
{
String version = FBUtilities.getReleaseVersionString();
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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 b18e67b..9de33db 100644
--- a/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
@@ -156,9 +156,9 @@ public class AntiCompactionTest
private SSTableReader writeFile(ColumnFamilyStore cfs, int count)
{
File dir = cfs.directories.getDirectoryForNewSSTables();
- String filename = cfs.getTempSSTablePath(dir);
+ String filename = cfs.getSSTablePath(dir);
- try (SSTableWriter writer = SSTableWriter.create(filename, 0, 0, new SerializationHeader(cfm, cfm.partitionColumns(), EncodingStats.NO_STATS)))
+ try (SSTableTxnWriter writer = SSTableTxnWriter.create(filename, 0, 0, new SerializationHeader(cfm, cfm.partitionColumns(), EncodingStats.NO_STATS)))
{
for (int i = 0; i < count; i++)
{
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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 0bcd603..30f14fc 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionAwareWriterTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionAwareWriterTest.java
@@ -78,7 +78,7 @@ public class CompactionAwareWriterTest extends CQLTester
populate(rowCount);
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);
+ CompactionAwareWriter writer = new DefaultCompactionWriter(cfs, txn, txn.originals(), false);
int rows = compact(cfs, txn, writer);
assertEquals(1, cfs.getSSTables().size());
assertEquals(rowCount, rows);
@@ -97,7 +97,7 @@ public class CompactionAwareWriterTest extends CQLTester
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, txn, txn.originals(), sstableSize, 0, false, OperationType.COMPACTION);
+ CompactionAwareWriter writer = new MaxSSTableSizeWriter(cfs, txn, txn.originals(), sstableSize, 0, false);
int rows = compact(cfs, txn, writer);
assertEquals(10, cfs.getSSTables().size());
assertEquals(rowCount, rows);
@@ -114,7 +114,7 @@ public class CompactionAwareWriterTest extends CQLTester
populate(rowCount);
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);
+ CompactionAwareWriter writer = new SplittingSizeTieredCompactionWriter(cfs, txn, txn.originals(), 0);
int rows = compact(cfs, txn, writer);
long expectedSize = beforeSize / 2;
List<SSTableReader> sortedSSTables = new ArrayList<>(cfs.getSSTables());
@@ -150,7 +150,7 @@ public class CompactionAwareWriterTest extends CQLTester
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, txn, txn.originals(), sstableSize, false, OperationType.COMPACTION);
+ CompactionAwareWriter writer = new MajorLeveledCompactionWriter(cfs, txn, txn.originals(), sstableSize, false);
int rows = compact(cfs, txn, writer);
assertEquals(targetSSTableCount, cfs.getSSTables().size());
int [] levelCounts = new int[5];
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
index 2fa8488..7beb405 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
@@ -21,6 +21,11 @@ package org.apache.cassandra.db.compaction;
import java.util.*;
import java.util.concurrent.TimeUnit;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
import org.apache.cassandra.OrderedJUnit4ClassRunner;
import org.apache.cassandra.SchemaLoader;
import org.apache.cassandra.Util;
@@ -34,11 +39,6 @@ import org.apache.cassandra.schema.KeyspaceParams;
import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.utils.FBUtilities;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-
import static org.junit.Assert.*;
@RunWith(OrderedJUnit4ClassRunner.class)
@@ -394,21 +394,25 @@ public class CompactionsTest
cf.addColumn(Util.column("a", "a", 3));
cf.deletionInfo().add(new RangeTombstone(Util.cellname("0"), Util.cellname("b"), 2, (int) (System.currentTimeMillis()/1000)),cfmeta.comparator);
- SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(dir.getDirectoryForNewSSTables())), 0, 0, 0);
-
-
- writer.append(Util.dk("0"), cf);
- writer.append(Util.dk("1"), cf);
- writer.append(Util.dk("3"), cf);
+ Descriptor desc = Descriptor.fromFilename(cfs.getSSTablePath(dir.getDirectoryForNewSSTables()));
+ try(SSTableTxnWriter writer = SSTableTxnWriter.create(desc, 0, 0, 0))
+ {
+ writer.append(Util.dk("0"), cf);
+ writer.append(Util.dk("1"), cf);
+ writer.append(Util.dk("3"), cf);
- cfs.addSSTable(writer.closeAndOpenReader());
- writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(dir.getDirectoryForNewSSTables())), 0, 0, 0);
+ cfs.addSSTable(writer.closeAndOpenReader());
+ }
- writer.append(Util.dk("0"), cf);
- writer.append(Util.dk("1"), cf);
- writer.append(Util.dk("2"), cf);
- writer.append(Util.dk("3"), cf);
- cfs.addSSTable(writer.closeAndOpenReader());
+ desc = Descriptor.fromFilename(cfs.getSSTablePath(dir.getDirectoryForNewSSTables()));
+ try (SSTableTxnWriter writer = SSTableTxnWriter.create(desc, 0, 0, 0))
+ {
+ writer.append(Util.dk("0"), cf);
+ writer.append(Util.dk("1"), cf);
+ writer.append(Util.dk("2"), cf);
+ writer.append(Util.dk("3"), cf);
+ cfs.addSSTable(writer.closeAndOpenReader());
+ }
Collection<SSTableReader> toCompact = cfs.getSSTables();
assert toCompact.size() == 2;
@@ -439,35 +443,6 @@ public class CompactionsTest
assertEquals(keys, k);
}
- @Test
- public void testCompactionLog() throws Exception
- {
- SystemKeyspace.discardCompactionsInProgress();
-
- String cf = "Standard4";
- ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(cf);
- SchemaLoader.insertData(KEYSPACE1, cf, 0, 1);
- cfs.forceBlockingFlush();
-
- Collection<SSTableReader> sstables = cfs.getSSTables();
- assertFalse(sstables.isEmpty());
- Set<Integer> generations = Sets.newHashSet(Iterables.transform(sstables, new Function<SSTableReader, Integer>()
- {
- public Integer apply(SSTableReader sstable)
- {
- return sstable.descriptor.generation;
- }
- }));
- UUID taskId = SystemKeyspace.startCompaction(cfs, sstables);
- Map<Pair<String, String>, Map<Integer, UUID>> compactionLogs = SystemKeyspace.getUnfinishedCompactions();
- Set<Integer> unfinishedCompactions = compactionLogs.get(Pair.create(KEYSPACE1, cf)).keySet();
- assertTrue(unfinishedCompactions.containsAll(generations));
-
- SystemKeyspace.finishCompaction(taskId);
- compactionLogs = SystemKeyspace.getUnfinishedCompactions();
- assertFalse(compactionLogs.containsKey(Pair.create(KEYSPACE1, cf)));
- }
-
private void testDontPurgeAccidentaly(String k, String cfname) throws InterruptedException
{
// This test catches the regression of CASSANDRA-2786
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
index ab84555..a869a7f 100644
--- a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
@@ -190,6 +190,9 @@ public class LeveledCompactionStrategyTest
// L0 is the lowest priority, so when that's done, we know everything is done
while (strategy.getSSTableCountPerLevel()[0] > 1)
Thread.sleep(100);
+
+ // in AbstractCompationStrategy.replaceSSTables() first we remove and then we add sstables so wait a little bit longer
+ Thread.sleep(10);
}
@Test
@@ -211,7 +214,7 @@ public class LeveledCompactionStrategyTest
}
waitForLeveling(cfs);
- LeveledCompactionStrategy strategy = (LeveledCompactionStrategy) ( cfs.getCompactionStrategyManager()).getStrategies().get(1);
+ LeveledCompactionStrategy strategy = (LeveledCompactionStrategy) (cfs.getCompactionStrategyManager()).getStrategies().get(1);
assert strategy.getLevelSize(1) > 0;
// get LeveledScanner for level 1 sstables
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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
index 18bce10..e9c903e 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/HelpersTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/HelpersTest.java
@@ -18,6 +18,8 @@
*/
package org.apache.cassandra.db.lifecycle;
+import java.util.ArrayList;
+import java.util.List;
import java.util.Map;
import java.util.Set;
@@ -32,11 +34,16 @@ import junit.framework.Assert;
import org.apache.cassandra.MockSchema;
import org.apache.cassandra.Util;
import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.compaction.OperationType;
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.Pair;
import org.apache.cassandra.utils.concurrent.Refs;
+import static junit.framework.Assert.assertEquals;
+import static junit.framework.Assert.assertNotNull;
+
public class HelpersTest
{
@@ -150,19 +157,29 @@ public class HelpersTest
for (SSTableReader reader : readers)
Assert.assertTrue(reader.isReplaced());
accumulate = Helpers.setReplaced(readers, null);
- Assert.assertNotNull(accumulate);
+ assertNotNull(accumulate);
}
@Test
public void testMarkObsolete()
{
ColumnFamilyStore cfs = MockSchema.newCFS();
+ TransactionLogs txnLogs = new TransactionLogs(OperationType.UNKNOWN, cfs.metadata);
Iterable<SSTableReader> readers = Lists.newArrayList(MockSchema.sstable(1, cfs), MockSchema.sstable(2, cfs));
- Throwable accumulate = Helpers.markObsolete(null, readers, null);
+
+ List<TransactionLogs.Obsoletion> obsoletions = new ArrayList<>();
+ Assert.assertNull(Helpers.prepareForObsoletion(readers, txnLogs, obsoletions, null));
+ assertNotNull(obsoletions);
+ assertEquals(2, obsoletions.size());
+
+ Throwable accumulate = Helpers.markObsolete(obsoletions, null);
Assert.assertNull(accumulate);
for (SSTableReader reader : readers)
Assert.assertTrue(reader.isMarkedCompacted());
- accumulate = Helpers.markObsolete(null, readers, null);
- Assert.assertNotNull(accumulate);
+
+ accumulate = Helpers.markObsolete(obsoletions, null);
+ assertNotNull(accumulate);
+
+ txnLogs.finish();
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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
index d6af447..a876891 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
@@ -33,7 +33,6 @@ import org.apache.cassandra.db.ColumnFamilyStore;
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;
@@ -249,7 +248,7 @@ public class LifecycleTransactionTest extends AbstractTransactionalTest
protected TestableTransaction newTest()
{
- SSTableDeletingTask.waitForDeletions();
+ TransactionLogs.waitForDeletions();
SSTableReader.resetTidying();
return new TxnTest();
}
@@ -399,6 +398,12 @@ public class LifecycleTransactionTest extends AbstractTransactionalTest
for (SSTableReader reader : concat(loggedObsolete, stagedObsolete))
Assert.assertTrue(reader.selfRef().globalCount() == 0);
}
+
+ @Override
+ protected boolean commitCanThrow()
+ {
+ return true;
+ }
}
private static SSTableReader[] readersArray(int lb, int ub, ColumnFamilyStore cfs)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/unit/org/apache/cassandra/db/lifecycle/RealTransactionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/RealTransactionsTest.java b/test/unit/org/apache/cassandra/db/lifecycle/RealTransactionsTest.java
new file mode 100644
index 0000000..5291baa
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/lifecycle/RealTransactionsTest.java
@@ -0,0 +1,228 @@
+/*
+ * 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.io.IOException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.lang3.StringUtils;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import junit.framework.Assert;
+import org.apache.cassandra.MockSchema;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.DatabaseDescriptor;
+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.SerializationHeader;
+import org.apache.cassandra.db.compaction.AbstractCompactionStrategy;
+import org.apache.cassandra.db.compaction.CompactionController;
+import org.apache.cassandra.db.compaction.CompactionIterator;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.io.sstable.CQLSSTableWriter;
+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.schema.KeyspaceParams;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests to simulate real transactions such as compactions and flushing
+ * using SSTableRewriter, ColumnFamilyStore, LifecycleTransaction, TransactionLogs, etc
+ */
+public class RealTransactionsTest extends SchemaLoader
+{
+ private static final String KEYSPACE = "TransactionLogsTest";
+ private static final String REWRITE_FINISHED_CF = "RewriteFinished";
+ private static final String REWRITE_ABORTED_CF = "RewriteAborted";
+ private static final String FLUSH_CF = "Flush";
+
+ @BeforeClass
+ public static void setUp()
+ {
+ MockSchema.cleanup();
+
+ SchemaLoader.prepareServer();
+ SchemaLoader.createKeyspace(KEYSPACE,
+ KeyspaceParams.simple(1),
+ SchemaLoader.standardCFMD(KEYSPACE, REWRITE_FINISHED_CF),
+ SchemaLoader.standardCFMD(KEYSPACE, REWRITE_ABORTED_CF),
+ SchemaLoader.standardCFMD(KEYSPACE, FLUSH_CF));
+ }
+
+ @Test
+ public void testRewriteFinished() throws IOException
+ {
+ Keyspace keyspace = Keyspace.open(KEYSPACE);
+ ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(REWRITE_FINISHED_CF);
+
+ SSTableReader oldSSTable = getSSTable(cfs, 1);
+ LifecycleTransaction txn = cfs.getTracker().tryModify(oldSSTable, OperationType.COMPACTION);
+ SSTableReader newSSTable = replaceSSTable(cfs, txn, false);
+ TransactionLogs.waitForDeletions();
+
+ assertFiles(txn.logs().getDataFolder(), new HashSet<>(newSSTable.getAllFilePaths()));
+ assertFiles(txn.logs().getLogsFolder(), Collections.<String>emptySet());
+ }
+
+ @Test
+ public void testRewriteAborted() throws IOException
+ {
+ Keyspace keyspace = Keyspace.open(KEYSPACE);
+ ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(REWRITE_ABORTED_CF);
+
+ SSTableReader oldSSTable = getSSTable(cfs, 1);
+ LifecycleTransaction txn = cfs.getTracker().tryModify(oldSSTable, OperationType.COMPACTION);
+
+ replaceSSTable(cfs, txn, true);
+ TransactionLogs.waitForDeletions();
+
+ assertFiles(txn.logs().getDataFolder(), new HashSet<>(oldSSTable.getAllFilePaths()));
+ assertFiles(txn.logs().getLogsFolder(), Collections.<String>emptySet());
+ }
+
+ @Test
+ public void testFlush() throws IOException
+ {
+ Keyspace keyspace = Keyspace.open(KEYSPACE);
+ ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(FLUSH_CF);
+
+ SSTableReader ssTableReader = getSSTable(cfs, 100);
+
+ String dataFolder = cfs.getSSTables().iterator().next().descriptor.directory.getPath();
+ String transactionLogsFolder = StringUtils.join(dataFolder, File.separator, Directories.TRANSACTIONS_SUBDIR);
+
+ assertTrue(new File(transactionLogsFolder).exists());
+ assertFiles(transactionLogsFolder, Collections.<String>emptySet());
+
+ assertFiles(dataFolder, new HashSet<>(ssTableReader.getAllFilePaths()));
+ }
+
+ private SSTableReader getSSTable(ColumnFamilyStore cfs, int numPartitions) throws IOException
+ {
+ createSSTable(cfs, numPartitions);
+
+ Set<SSTableReader> sstables = new HashSet<>(cfs.getSSTables());
+ assertEquals(1, sstables.size());
+ return sstables.iterator().next();
+ }
+
+ private void createSSTable(ColumnFamilyStore cfs, int numPartitions) throws IOException
+ {
+ cfs.truncateBlocking();
+
+ String schema = "CREATE TABLE %s.%s (key ascii, name ascii, val ascii, val1 ascii, PRIMARY KEY (key, name))";
+ String query = "INSERT INTO %s.%s (key, name, val) VALUES (?, ?, ?)";
+
+ try (CQLSSTableWriter writer = CQLSSTableWriter.builder()
+ .withPartitioner(StorageService.getPartitioner())
+ .inDirectory(cfs.directories.getDirectoryForNewSSTables())
+ .forTable(String.format(schema, cfs.keyspace.getName(), cfs.name))
+ .using(String.format(query, cfs.keyspace.getName(), cfs.name))
+ .build())
+ {
+ for (int j = 0; j < numPartitions; j ++)
+ writer.addRow(String.format("key%d", j), "col1", "0");
+ }
+
+ cfs.loadNewSSTables();
+ }
+
+ private SSTableReader replaceSSTable(ColumnFamilyStore cfs, LifecycleTransaction txn, boolean fail)
+ {
+ List<SSTableReader> newsstables = null;
+ int nowInSec = FBUtilities.nowInSeconds();
+ try (CompactionController controller = new CompactionController(cfs, txn.originals(), cfs.gcBefore(FBUtilities.nowInSeconds())))
+ {
+ try (SSTableRewriter rewriter = new SSTableRewriter(cfs, txn, 1000, false);
+ AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategyManager().getScanners(txn.originals());
+ CompactionIterator ci = new CompactionIterator(txn.opType(), scanners.scanners, controller, nowInSec, txn.opId())
+ )
+ {
+ long lastCheckObsoletion = System.nanoTime();
+ File directory = txn.originals().iterator().next().descriptor.directory;
+ Descriptor desc = Descriptor.fromFilename(cfs.getSSTablePath(directory));
+ CFMetaData metadata = Schema.instance.getCFMetaData(desc);
+ rewriter.switchWriter(SSTableWriter.create(metadata,
+ desc,
+ 0,
+ 0,
+ 0,
+ DatabaseDescriptor.getPartitioner(),
+ SerializationHeader.make(cfs.metadata, txn.originals()),
+ txn));
+ while (ci.hasNext())
+ {
+ rewriter.append(ci.next());
+
+ if (System.nanoTime() - lastCheckObsoletion > TimeUnit.MINUTES.toNanos(1L))
+ {
+ controller.maybeRefreshOverlaps();
+ lastCheckObsoletion = System.nanoTime();
+ }
+ }
+
+ if (!fail)
+ newsstables = rewriter.finish();
+ else
+ rewriter.abort();
+ }
+ }
+
+ assertTrue(fail || newsstables != null);
+
+ if (newsstables != null)
+ {
+ Assert.assertEquals(1, newsstables.size());
+ return newsstables.iterator().next();
+ }
+
+ return null;
+ }
+
+ private void assertFiles(String dirPath, Set<String> expectedFiles)
+ {
+ File dir = new File(dirPath);
+ for (File file : dir.listFiles())
+ {
+ if (file.isDirectory())
+ continue;
+
+ String filePath = file.getPath();
+ assertTrue(filePath, expectedFiles.contains(filePath));
+ expectedFiles.remove(filePath);
+ }
+
+ assertTrue(expectedFiles.isEmpty());
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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
index dbd5287..89924a5 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
@@ -40,7 +40,6 @@ 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;
@@ -152,6 +151,9 @@ public class TrackerTest
Assert.assertEquals(3, tracker.view.get().sstables.size());
+ for (SSTableReader reader : readers)
+ Assert.assertTrue(reader.isKeyCacheSetup());
+
Assert.assertEquals(17 + 121 + 9, cfs.metric.liveDiskSpaceUsed.getCount());
}
@@ -171,6 +173,9 @@ public class TrackerTest
Assert.assertEquals(3, tracker.view.get().sstables.size());
+ for (SSTableReader reader : readers)
+ Assert.assertTrue(reader.isKeyCacheSetup());
+
Assert.assertEquals(17 + 121 + 9, cfs.metric.liveDiskSpaceUsed.getCount());
Assert.assertEquals(3, listener.senders.size());
Assert.assertEquals(tracker, listener.senders.get(0));
@@ -182,9 +187,9 @@ public class TrackerTest
public void testDropSSTables()
{
testDropSSTables(false);
- SSTableDeletingTask.waitForDeletions();
+ TransactionLogs.waitForDeletions();
testDropSSTables(true);
- SSTableDeletingTask.waitForDeletions();
+ TransactionLogs.waitForDeletions();
}
private void testDropSSTables(boolean invalidate)
@@ -200,7 +205,7 @@ public class TrackerTest
try
{
- SSTableDeletingTask.pauseDeletions(true);
+ TransactionLogs.pauseDeletions(true);
try (LifecycleTransaction txn = tracker.tryModify(readers.get(0), OperationType.COMPACTION))
{
if (invalidate)
@@ -222,19 +227,19 @@ public class TrackerTest
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.assertNull(tracker.dropSSTables(reader -> reader != readers.get(0), OperationType.UNKNOWN, null));
+
Assert.assertEquals(1, tracker.getView().sstables.size());
Assert.assertEquals(3, 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.assertTrue(listener.received.get(0) instanceof SSTableDeletingNotification);
+ Assert.assertTrue(listener.received.get(1) instanceof SSTableDeletingNotification);
+ Assert.assertTrue(listener.received.get(2) instanceof SSTableListChangedNotification);
+ Assert.assertEquals(readers.get(1), ((SSTableDeletingNotification) listener.received.get(0)).deleting);
+ Assert.assertEquals(readers.get(2), ((SSTableDeletingNotification)listener.received.get(1)).deleting);
+ Assert.assertEquals(2, ((SSTableListChangedNotification) listener.received.get(2)).removed.size());
+ Assert.assertEquals(0, ((SSTableListChangedNotification) listener.received.get(2)).added.size());
Assert.assertEquals(9, cfs.metric.liveDiskSpaceUsed.getCount());
readers.get(0).selfRef().release();
}
@@ -248,7 +253,7 @@ public class TrackerTest
}
finally
{
- SSTableDeletingTask.pauseDeletions(false);
+ TransactionLogs.pauseDeletions(false);
}
}
@@ -299,6 +304,7 @@ public class TrackerTest
Assert.assertEquals(1, listener.received.size());
Assert.assertEquals(reader, ((SSTableAddedNotification) listener.received.get(0)).added);
listener.received.clear();
+ Assert.assertTrue(reader.isKeyCacheSetup());
Assert.assertEquals(10, cfs.metric.liveDiskSpaceUsed.getCount());
// test invalidated CFS
@@ -314,8 +320,10 @@ public class TrackerTest
Assert.assertEquals(0, tracker.getView().sstables.size());
Assert.assertEquals(0, tracker.getView().flushingMemtables.size());
Assert.assertEquals(0, cfs.metric.liveDiskSpaceUsed.getCount());
+ Assert.assertEquals(3, listener.received.size());
Assert.assertEquals(reader, ((SSTableAddedNotification) listener.received.get(0)).added);
- Assert.assertEquals(1, ((SSTableListChangedNotification) listener.received.get(1)).removed.size());
+ Assert.assertTrue(listener.received.get(1) instanceof SSTableDeletingNotification);
+ Assert.assertEquals(1, ((SSTableListChangedNotification) listener.received.get(2)).removed.size());
DatabaseDescriptor.setIncrementalBackupsEnabled(backups);
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/unit/org/apache/cassandra/db/lifecycle/TransactionLogsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/TransactionLogsTest.java b/test/unit/org/apache/cassandra/db/lifecycle/TransactionLogsTest.java
new file mode 100644
index 0000000..3150087
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/lifecycle/TransactionLogsTest.java
@@ -0,0 +1,558 @@
+/*
+ * 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.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.util.*;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Sets;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static junit.framework.Assert.assertNotNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import junit.framework.Assert;
+import org.apache.cassandra.MockSchema;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Directories;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.db.compaction.*;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.io.sstable.*;
+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.SegmentedFile;
+import org.apache.cassandra.utils.AlwaysPresentFilter;
+import org.apache.cassandra.utils.concurrent.AbstractTransactionalTest;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+public class TransactionLogsTest extends AbstractTransactionalTest
+{
+ private static final String KEYSPACE = "TransactionLogsTest";
+
+ @BeforeClass
+ public static void setUp()
+ {
+ MockSchema.cleanup();
+ }
+
+ protected AbstractTransactionalTest.TestableTransaction newTest() throws Exception
+ {
+ TransactionLogs.waitForDeletions();
+ SSTableReader.resetTidying();
+ return new TxnTest();
+ }
+
+ private static final class TxnTest extends TestableTransaction
+ {
+ private final static class Transaction extends Transactional.AbstractTransactional implements Transactional
+ {
+ final ColumnFamilyStore cfs;
+ final TransactionLogs txnLogs;
+ final SSTableReader sstableOld;
+ final SSTableReader sstableNew;
+ final TransactionLogs.SSTableTidier tidier;
+
+ public Transaction(ColumnFamilyStore cfs, TransactionLogs txnLogs) throws IOException
+ {
+ this.cfs = cfs;
+ this.txnLogs = txnLogs;
+ this.sstableOld = sstable(cfs, 0, 128);
+ this.sstableNew = sstable(cfs, 1, 128);
+
+ assertNotNull(txnLogs);
+ assertNotNull(txnLogs.getId());
+ Assert.assertEquals(OperationType.COMPACTION, txnLogs.getType());
+
+ txnLogs.trackNew(sstableNew);
+ tidier = txnLogs.obsoleted(sstableOld);
+ assertNotNull(tidier);
+ }
+
+ protected Throwable doCommit(Throwable accumulate)
+ {
+ sstableOld.markObsolete(tidier);
+ sstableOld.selfRef().release();
+ TransactionLogs.waitForDeletions();
+
+ Throwable ret = txnLogs.commit(accumulate);
+
+ sstableNew.selfRef().release();
+ return ret;
+ }
+
+ protected Throwable doAbort(Throwable accumulate)
+ {
+ tidier.abort();
+ TransactionLogs.waitForDeletions();
+
+ Throwable ret = txnLogs.abort(accumulate);
+
+ sstableNew.selfRef().release();
+ sstableOld.selfRef().release();
+ return ret;
+ }
+
+ protected void doPrepare()
+ {
+ txnLogs.prepareToCommit();
+ }
+
+ protected void assertInProgress() throws Exception
+ {
+ assertFiles(txnLogs.getDataFolder(), Sets.newHashSet(Iterables.concat(sstableNew.getAllFilePaths(),
+ sstableOld.getAllFilePaths())));
+ assertFiles(txnLogs.getLogsFolder(), Sets.newHashSet(txnLogs.getData().oldLog().file.getPath(),
+ txnLogs.getData().newLog().file.getPath()));
+ assertEquals(2, TransactionLogs.getLogFiles(cfs.metadata).size());
+ }
+
+ protected void assertPrepared() throws Exception
+ {
+ }
+
+ protected void assertAborted() throws Exception
+ {
+ assertFiles(txnLogs.getDataFolder(), new HashSet<>(sstableOld.getAllFilePaths()));
+ assertFiles(txnLogs.getLogsFolder(), Collections.<String>emptySet());
+ assertEquals(0, TransactionLogs.getLogFiles(cfs.metadata).size());
+ }
+
+ protected void assertCommitted() throws Exception
+ {
+ assertFiles(txnLogs.getDataFolder(), new HashSet<>(sstableNew.getAllFilePaths()));
+ assertFiles(txnLogs.getLogsFolder(), Collections.<String>emptySet());
+ assertEquals(0, TransactionLogs.getLogFiles(cfs.metadata).size());
+ }
+ }
+
+ final Transaction txn;
+
+ private TxnTest() throws IOException
+ {
+ this(MockSchema.newCFS(KEYSPACE));
+ }
+
+ private TxnTest(ColumnFamilyStore cfs) throws IOException
+ {
+ this(cfs, new TransactionLogs(OperationType.COMPACTION, cfs.metadata));
+ }
+
+ private TxnTest(ColumnFamilyStore cfs, TransactionLogs txnLogs) throws IOException
+ {
+ this(new Transaction(cfs, txnLogs));
+ }
+
+ private TxnTest(Transaction txn)
+ {
+ super(txn);
+ this.txn = txn;
+ }
+
+ protected void assertInProgress() throws Exception
+ {
+ txn.assertInProgress();
+ }
+
+ protected void assertPrepared() throws Exception
+ {
+ txn.assertPrepared();
+ }
+
+ protected void assertAborted() throws Exception
+ {
+ txn.assertAborted();
+ }
+
+ protected void assertCommitted() throws Exception
+ {
+ txn.assertCommitted();
+ }
+ }
+
+ @Test
+ public void testUntrack() throws Throwable
+ {
+ ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+ SSTableReader sstableNew = sstable(cfs, 1, 128);
+
+ // complete a transaction without keep the new files since they were untracked
+ TransactionLogs transactionLogs = new TransactionLogs(OperationType.COMPACTION, cfs.metadata);
+ assertNotNull(transactionLogs);
+
+ transactionLogs.trackNew(sstableNew);
+ transactionLogs.untrackNew(sstableNew);
+
+ transactionLogs.finish();
+
+ assertFiles(transactionLogs.getDataFolder(), Collections.<String>emptySet());
+ assertFiles(transactionLogs.getLogsFolder(), Collections.<String>emptySet());
+ assertEquals(0, TransactionLogs.getLogFiles(cfs.metadata).size());
+
+ sstableNew.selfRef().release();
+ }
+
+ @Test
+ public void testCommitSameDesc() throws Throwable
+ {
+ ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+ SSTableReader sstableOld1 = sstable(cfs, 0, 128);
+ SSTableReader sstableOld2 = sstable(cfs, 0, 256);
+ SSTableReader sstableNew = sstable(cfs, 1, 128);
+
+ TransactionLogs transactionLogs = new TransactionLogs(OperationType.COMPACTION, cfs.metadata);
+ assertNotNull(transactionLogs);
+
+ transactionLogs.trackNew(sstableNew);
+
+ sstableOld1.setReplaced();
+
+ TransactionLogs.SSTableTidier tidier = transactionLogs.obsoleted(sstableOld2);
+ assertNotNull(tidier);
+
+ transactionLogs.finish();
+
+ sstableOld2.markObsolete(tidier);
+
+ sstableOld1.selfRef().release();
+ sstableOld2.selfRef().release();
+
+ TransactionLogs.waitForDeletions();
+
+ assertFiles(transactionLogs.getDataFolder(), new HashSet<>(sstableNew.getAllFilePaths()));
+ assertFiles(transactionLogs.getLogsFolder(), Collections.<String>emptySet());
+ assertEquals(0, TransactionLogs.getLogFiles(cfs.metadata).size());
+
+ sstableNew.selfRef().release();
+ }
+
+ @Test
+ public void testCommitOnlyNew() throws Throwable
+ {
+ ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+ SSTableReader sstable = sstable(cfs, 0, 128);
+
+ TransactionLogs transactionLogs = new TransactionLogs(OperationType.COMPACTION, cfs.metadata);
+ assertNotNull(transactionLogs);
+
+ transactionLogs.trackNew(sstable);
+ transactionLogs.finish();
+
+ assertFiles(transactionLogs.getDataFolder(), new HashSet<>(sstable.getAllFilePaths()));
+ assertFiles(transactionLogs.getLogsFolder(), Collections.<String>emptySet());
+ assertEquals(0, TransactionLogs.getLogFiles(cfs.metadata).size());
+
+ sstable.selfRef().release();
+ }
+
+ @Test
+ public void testCommitOnlyOld() throws Throwable
+ {
+ ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+ SSTableReader sstable = sstable(cfs, 0, 128);
+
+ TransactionLogs transactionLogs = new TransactionLogs(OperationType.COMPACTION, cfs.metadata);
+ assertNotNull(transactionLogs);
+
+ TransactionLogs.SSTableTidier tidier = transactionLogs.obsoleted(sstable);
+ assertNotNull(tidier);
+
+ transactionLogs.finish();
+ sstable.markObsolete(tidier);
+ sstable.selfRef().release();
+
+ TransactionLogs.waitForDeletions();
+
+ assertFiles(transactionLogs.getDataFolder(), new HashSet<>());
+ assertFiles(transactionLogs.getLogsFolder(), Collections.<String>emptySet());
+ assertEquals(0, TransactionLogs.getLogFiles(cfs.metadata).size());
+ }
+
+ @Test
+ public void testAbortOnlyNew() throws Throwable
+ {
+ ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+ SSTableReader sstable = sstable(cfs, 0, 128);
+
+ TransactionLogs transactionLogs = new TransactionLogs(OperationType.COMPACTION, cfs.metadata);
+ assertNotNull(transactionLogs);
+
+ transactionLogs.trackNew(sstable);
+ transactionLogs.abort();
+
+ sstable.selfRef().release();
+
+ assertFiles(transactionLogs.getDataFolder(), new HashSet<>());
+ assertFiles(transactionLogs.getLogsFolder(), Collections.<String>emptySet());
+ assertEquals(0, TransactionLogs.getLogFiles(cfs.metadata).size());
+ }
+
+ @Test
+ public void testAbortOnlyOld() throws Throwable
+ {
+ ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+ SSTableReader sstable = sstable(cfs, 0, 128);
+
+ TransactionLogs transactionLogs = new TransactionLogs(OperationType.COMPACTION, cfs.metadata);
+ assertNotNull(transactionLogs);
+
+ TransactionLogs.SSTableTidier tidier = transactionLogs.obsoleted(sstable);
+ assertNotNull(tidier);
+
+ tidier.abort();
+ transactionLogs.abort();
+
+ sstable.selfRef().release();
+
+ assertFiles(transactionLogs.getDataFolder(), new HashSet<>(sstable.getAllFilePaths()));
+ assertFiles(transactionLogs.getLogsFolder(), Collections.<String>emptySet());
+ assertEquals(0, TransactionLogs.getLogFiles(cfs.metadata).size());
+ }
+
+ private File copyToTmpFile(File file) throws IOException
+ {
+ File ret = File.createTempFile(file.getName(), ".tmp");
+ ret.deleteOnExit();
+ Files.copy(file.toPath(), ret.toPath(), StandardCopyOption.REPLACE_EXISTING);
+ return ret;
+ }
+
+ @Test
+ public void testRemoveUnfinishedLeftovers_newLogFound() throws Throwable
+ {
+ ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+ SSTableReader sstableOld = sstable(cfs, 0, 128);
+ SSTableReader sstableNew = sstable(cfs, 1, 128);
+
+ // simulate tracking sstables with a failed transaction (new log file NOT deleted)
+ TransactionLogs transactionLogs = new TransactionLogs(OperationType.COMPACTION, cfs.metadata);
+ assertNotNull(transactionLogs);
+
+ transactionLogs.trackNew(sstableNew);
+ TransactionLogs.SSTableTidier tidier = transactionLogs.obsoleted(sstableOld);
+
+ File tmpNewLog = copyToTmpFile(transactionLogs.getData().newLog().file);
+ File tmpOldLog = copyToTmpFile(transactionLogs.getData().oldLog().file);
+
+ Set<File> tmpFiles = new HashSet<>(TransactionLogs.getLogFiles(cfs.metadata));
+ for (String p : sstableNew.getAllFilePaths())
+ tmpFiles.add(new File(p));
+
+ sstableNew.selfRef().release();
+ sstableOld.selfRef().release();
+
+ Assert.assertEquals(tmpFiles, TransactionLogs.getTemporaryFiles(cfs.metadata, sstableNew.descriptor.directory));
+
+ // normally called at startup
+ TransactionLogs.removeUnfinishedLeftovers(cfs.metadata);
+
+ // sstable should not have been removed because the new log was found
+ Directories directories = new Directories(cfs.metadata);
+ Map<Descriptor, Set<Component>> sstables = directories.sstableLister().list();
+ assertEquals(1, sstables.size());
+
+ assertFiles(transactionLogs.getDataFolder(), new HashSet<>(sstableOld.getAllFilePaths()));
+ assertFiles(transactionLogs.getLogsFolder(), Collections.<String>emptySet());
+
+ tidier.run();
+
+ // copy old transaction files contents back or transactionlogs will throw assertions
+ Files.move(tmpNewLog.toPath(), transactionLogs.getData().newLog().file.toPath());
+ Files.move(tmpOldLog.toPath(), transactionLogs.getData().oldLog().file.toPath());
+
+ transactionLogs.close();
+ }
+
+ @Test
+ public void testRemoveUnfinishedLeftovers_oldLogFound() throws Throwable
+ {
+ ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+ SSTableReader sstableOld = sstable(cfs, 0, 128);
+ SSTableReader sstableNew = sstable(cfs, 1, 128);
+
+ // simulate tracking sstables with a committed transaction (new log file deleted)
+ TransactionLogs transactionLogs = new TransactionLogs(OperationType.COMPACTION, cfs.metadata);
+ assertNotNull(transactionLogs);
+
+ transactionLogs.trackNew(sstableNew);
+ TransactionLogs.SSTableTidier tidier = transactionLogs.obsoleted(sstableOld);
+
+ File tmpNewLog = copyToTmpFile(transactionLogs.getData().newLog().file);
+ File tmpOldLog = copyToTmpFile(transactionLogs.getData().oldLog().file);
+
+ transactionLogs.getData().newLog().delete(false);
+
+ Set<File> tmpFiles = new HashSet<>(TransactionLogs.getLogFiles(cfs.metadata));
+ for (String p : sstableOld.getAllFilePaths())
+ tmpFiles.add(new File(p));
+
+ sstableNew.selfRef().release();
+ sstableOld.selfRef().release();
+
+ Assert.assertEquals(tmpFiles, TransactionLogs.getTemporaryFiles(cfs.metadata, sstableOld.descriptor.directory));
+
+ // normally called at startup
+ TransactionLogs.removeUnfinishedLeftovers(cfs.metadata);
+
+ // sstable should have been removed because there was no new log.
+ Directories directories = new Directories(cfs.metadata);
+ Map<Descriptor, Set<Component>> sstables = directories.sstableLister().list();
+ assertEquals(1, sstables.size());
+
+ assertFiles(transactionLogs.getDataFolder(), new HashSet<>(sstableNew.getAllFilePaths()));
+ assertFiles(transactionLogs.getLogsFolder(), Collections.<String>emptySet());
+
+ tidier.run();
+
+ // copy old transaction files contents back or transactionlogs will throw assertions
+ Files.move(tmpNewLog.toPath(), transactionLogs.getData().newLog().file.toPath());
+ Files.move(tmpOldLog.toPath(), transactionLogs.getData().oldLog().file.toPath());
+
+ transactionLogs.close();
+ }
+
+ @Test
+ public void testGetTemporaryFiles() throws IOException
+ {
+ ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
+ SSTableReader sstable1 = sstable(cfs, 0, 128);
+
+ File dataFolder = sstable1.descriptor.directory;
+
+ Set<File> tmpFiles = TransactionLogs.getTemporaryFiles(cfs.metadata, dataFolder);
+ assertNotNull(tmpFiles);
+ assertEquals(0, tmpFiles.size());
+
+ TransactionLogs transactionLogs = new TransactionLogs(OperationType.WRITE, cfs.metadata);
+ Directories directories = new Directories(cfs.metadata);
+
+ File[] beforeSecondSSTable = dataFolder.listFiles(pathname -> !pathname.isDirectory());
+
+ SSTableReader sstable2 = sstable(cfs, 1, 128);
+ transactionLogs.trackNew(sstable2);
+
+ Map<Descriptor, Set<Component>> sstables = directories.sstableLister().list();
+ assertEquals(2, sstables.size());
+
+ File[] afterSecondSSTable = dataFolder.listFiles(pathname -> !pathname.isDirectory());
+ int numNewFiles = afterSecondSSTable.length - beforeSecondSSTable.length;
+ assertTrue(numNewFiles == sstable2.getAllFilePaths().size());
+
+ tmpFiles = TransactionLogs.getTemporaryFiles(cfs.metadata, dataFolder);
+ assertNotNull(tmpFiles);
+ assertEquals(numNewFiles + 2, tmpFiles.size()); //the extra files are the transaction log files
+
+ File ssTable2DataFile = new File(sstable2.descriptor.filenameFor(Component.DATA));
+ File ssTable2IndexFile = new File(sstable2.descriptor.filenameFor(Component.PRIMARY_INDEX));
+
+ assertTrue(tmpFiles.contains(ssTable2DataFile));
+ assertTrue(tmpFiles.contains(ssTable2IndexFile));
+
+ List<File> files = directories.sstableLister().listFiles();
+ List<File> filesNoTmp = directories.sstableLister().skipTemporary(true).listFiles();
+ assertNotNull(files);
+ assertNotNull(filesNoTmp);
+
+ assertTrue(files.contains(ssTable2DataFile));
+ assertTrue(files.contains(ssTable2IndexFile));
+
+ assertFalse(filesNoTmp.contains(ssTable2DataFile));
+ assertFalse(filesNoTmp.contains(ssTable2IndexFile));
+
+ transactionLogs.finish();
+
+ //Now it should be empty since the transaction has finished
+ tmpFiles = TransactionLogs.getTemporaryFiles(cfs.metadata, dataFolder);
+ assertNotNull(tmpFiles);
+ assertEquals(0, tmpFiles.size());
+
+ filesNoTmp = directories.sstableLister().skipTemporary(true).listFiles();
+ assertNotNull(filesNoTmp);
+ assertTrue(filesNoTmp.contains(ssTable2DataFile));
+ assertTrue(filesNoTmp.contains(ssTable2IndexFile));
+
+ sstable1.selfRef().release();
+ sstable2.selfRef().release();
+ }
+
+ public static SSTableReader sstable(ColumnFamilyStore cfs, int generation, int size) throws IOException
+ {
+ Directories dir = new Directories(cfs.metadata);
+ Descriptor descriptor = new Descriptor(dir.getDirectoryForNewSSTables(), cfs.keyspace.getName(), cfs.getColumnFamilyName(), generation);
+ 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));
+ file.createNewFile();
+ try (RandomAccessFile raf = new RandomAccessFile(file, "rw"))
+ {
+ raf.setLength(size);
+ }
+ }
+
+ SegmentedFile dFile = new BufferedSegmentedFile(new ChannelProxy(new File(descriptor.filenameFor(Component.DATA))), 0);
+ SegmentedFile iFile = new BufferedSegmentedFile(new ChannelProxy(new File(descriptor.filenameFor(Component.PRIMARY_INDEX))), 0);
+
+ SerializationHeader header = SerializationHeader.make(cfs.metadata, Collections.EMPTY_LIST);
+ StatsMetadata metadata = (StatsMetadata) new MetadataCollector(cfs.metadata.comparator)
+ .finalizeMetadata(Murmur3Partitioner.instance.getClass().getCanonicalName(), 0.01f, -1, header)
+ .get(MetadataType.STATS);
+ SSTableReader reader = SSTableReader.internalOpen(descriptor,
+ components,
+ cfs.metadata,
+ Murmur3Partitioner.instance,
+ dFile,
+ iFile,
+ MockSchema.indexSummary.sharedCopy(),
+ new AlwaysPresentFilter(),
+ 1L,
+ metadata,
+ SSTableReader.OpenReason.NORMAL,
+ header);
+ reader.first = reader.last = MockSchema.readerBounds(generation);
+ return reader;
+ }
+
+ private static void assertFiles(String dirPath, Set<String> expectedFiles)
+ {
+ File dir = new File(dirPath);
+ for (File file : dir.listFiles())
+ {
+ if (file.isDirectory())
+ continue;
+
+ String filePath = file.getPath();
+ assertTrue(filePath, expectedFiles.contains(filePath));
+ expectedFiles.remove(filePath);
+ }
+
+ assertTrue(expectedFiles.isEmpty());
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/unit/org/apache/cassandra/io/sstable/BigTableWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/BigTableWriterTest.java b/test/unit/org/apache/cassandra/io/sstable/BigTableWriterTest.java
index 29875d5..08de62f 100644
--- a/test/unit/org/apache/cassandra/io/sstable/BigTableWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/BigTableWriterTest.java
@@ -61,19 +61,19 @@ public class BigTableWriterTest extends AbstractTransactionalTest
{
final File file;
final Descriptor descriptor;
- final SSTableWriter writer;
+ final SSTableTxnWriter writer;
private TestableBTW() throws IOException
{
- this(cfs.getTempSSTablePath(cfs.directories.getDirectoryForNewSSTables()));
+ this(cfs.getSSTablePath(cfs.directories.getDirectoryForNewSSTables()));
}
private TestableBTW(String file) throws IOException
{
- this(file, SSTableWriter.create(file, 0, 0, new SerializationHeader(cfs.metadata, cfs.metadata.partitionColumns(), EncodingStats.NO_STATS)));
+ this(file, SSTableTxnWriter.create(file, 0, 0, new SerializationHeader(cfs.metadata, cfs.metadata.partitionColumns(), EncodingStats.NO_STATS)));
}
- private TestableBTW(String file, SSTableWriter sw) throws IOException
+ private TestableBTW(String file, SSTableTxnWriter sw) throws IOException
{
super(sw);
this.file = new File(file);
@@ -91,22 +91,19 @@ public class BigTableWriterTest extends AbstractTransactionalTest
protected void assertInProgress() throws Exception
{
- assertExists(Descriptor.Type.TEMP, Component.DATA, Component.PRIMARY_INDEX);
- assertNotExists(Descriptor.Type.TEMP, Component.FILTER, Component.SUMMARY);
- assertNotExists(Descriptor.Type.FINAL, Component.DATA, Component.PRIMARY_INDEX, Component.FILTER, Component.SUMMARY);
+ assertExists(Component.DATA, Component.PRIMARY_INDEX);
+ assertNotExists(Component.FILTER, Component.SUMMARY);
Assert.assertTrue(file.length() > 0);
}
protected void assertPrepared() throws Exception
{
- assertNotExists(Descriptor.Type.TEMP, Component.DATA, Component.PRIMARY_INDEX, Component.FILTER, Component.SUMMARY);
- assertExists(Descriptor.Type.FINAL, Component.DATA, Component.PRIMARY_INDEX, Component.FILTER, Component.SUMMARY);
+ assertExists(Component.DATA, Component.PRIMARY_INDEX, Component.FILTER, Component.SUMMARY);
}
protected void assertAborted() throws Exception
{
- assertNotExists(Descriptor.Type.TEMP, Component.DATA, Component.PRIMARY_INDEX, Component.FILTER, Component.SUMMARY);
- assertNotExists(Descriptor.Type.FINAL, Component.DATA, Component.PRIMARY_INDEX, Component.FILTER, Component.SUMMARY);
+ assertNotExists(Component.DATA, Component.PRIMARY_INDEX, Component.FILTER, Component.SUMMARY);
Assert.assertFalse(file.exists());
}
@@ -115,15 +112,15 @@ public class BigTableWriterTest extends AbstractTransactionalTest
assertPrepared();
}
- private void assertExists(Descriptor.Type type, Component ... components)
+ private void assertExists(Component ... components)
{
for (Component component : components)
- Assert.assertTrue(new File(descriptor.asType(type).filenameFor(component)).exists());
+ Assert.assertTrue(new File(descriptor.filenameFor(component)).exists());
}
- private void assertNotExists(Descriptor.Type type, Component ... components)
+ private void assertNotExists(Component ... components)
{
for (Component component : components)
- Assert.assertFalse(type.toString() + " " + component.toString(), new File(descriptor.asType(type).filenameFor(component)).exists());
+ Assert.assertFalse(component.toString(), new File(descriptor.filenameFor(component)).exists());
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterClientTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterClientTest.java b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterClientTest.java
index dab88c8..9a558f1 100644
--- a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterClientTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterClientTest.java
@@ -28,10 +28,13 @@ import org.junit.Before;
import org.junit.Test;
import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.Directories;
import org.apache.cassandra.exceptions.InvalidRequestException;
import org.apache.cassandra.io.util.FileUtils;
import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
public class CQLSSTableWriterClientTest
{
@@ -97,5 +100,11 @@ public class CQLSSTableWriterClientTest
File[] dataFiles = this.testDirectory.listFiles(filter);
assertEquals(2, dataFiles.length);
+ File transactionsFolder = Directories.getTransactionsDirectory(testDirectory);
+ assertTrue(transactionsFolder.exists());
+
+ File[] opFiles = transactionsFolder.listFiles();
+ assertEquals(0, opFiles.length);
+
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/unit/org/apache/cassandra/io/sstable/DescriptorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/DescriptorTest.java b/test/unit/org/apache/cassandra/io/sstable/DescriptorTest.java
index ceb0f9c..19eca40 100644
--- a/test/unit/org/apache/cassandra/io/sstable/DescriptorTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/DescriptorTest.java
@@ -77,24 +77,19 @@ public class DescriptorTest
private void testFromFilenameFor(File dir)
{
// normal
- checkFromFilename(new Descriptor(dir, ksname, cfname, 1, Descriptor.Type.FINAL), false);
+ checkFromFilename(new Descriptor(dir, ksname, cfname, 1), false);
// skip component (for streaming lock file)
- checkFromFilename(new Descriptor(dir, ksname, cfname, 2, Descriptor.Type.FINAL), true);
- // tmp
- checkFromFilename(new Descriptor(dir, ksname, cfname, 3, Descriptor.Type.TEMP), false);
+ checkFromFilename(new Descriptor(dir, ksname, cfname, 2), true);
+
// secondary index
String idxName = "myidx";
File idxDir = new File(dir.getAbsolutePath() + File.separator + Directories.SECONDARY_INDEX_NAME_SEPARATOR + idxName);
- checkFromFilename(new Descriptor(idxDir, ksname, cfname + Directories.SECONDARY_INDEX_NAME_SEPARATOR + idxName, 4, Descriptor.Type.FINAL), false);
- // secondary index tmp
- checkFromFilename(new Descriptor(idxDir, ksname, cfname + Directories.SECONDARY_INDEX_NAME_SEPARATOR + idxName, 5, Descriptor.Type.TEMP), false);
+ checkFromFilename(new Descriptor(idxDir, ksname, cfname + Directories.SECONDARY_INDEX_NAME_SEPARATOR + idxName, 4), false);
// legacy version
- checkFromFilename(new Descriptor("ja", dir, ksname, cfname, 1, Descriptor.Type.FINAL, SSTableFormat.Type.LEGACY), false);
- // legacy tmp
- checkFromFilename(new Descriptor("ja", dir, ksname, cfname, 2, Descriptor.Type.TEMP, SSTableFormat.Type.LEGACY), false);
+ checkFromFilename(new Descriptor("ja", dir, ksname, cfname, 1, SSTableFormat.Type.LEGACY), false);
// legacy secondary index
- checkFromFilename(new Descriptor("ja", dir, ksname, cfname + Directories.SECONDARY_INDEX_NAME_SEPARATOR + idxName, 3, Descriptor.Type.FINAL, SSTableFormat.Type.LEGACY), false);
+ checkFromFilename(new Descriptor("ja", dir, ksname, cfname + Directories.SECONDARY_INDEX_NAME_SEPARATOR + idxName, 3, SSTableFormat.Type.LEGACY), false);
}
private void checkFromFilename(Descriptor original, boolean skipComponent)
@@ -109,7 +104,6 @@ public class DescriptorTest
assertEquals(original.cfname, desc.cfname);
assertEquals(original.version, desc.version);
assertEquals(original.generation, desc.generation);
- assertEquals(original.type, desc.type);
if (skipComponent)
{
[4/5] cassandra git commit: Introduce safer durable sstable
membership management (and simplify cleanup of compaction leftovers)
Posted by be...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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 ba85eef..14cb795 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
@@ -28,7 +28,6 @@ import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.RowIndexEntry;
import org.apache.cassandra.db.SerializationHeader;
import org.apache.cassandra.db.rows.UnfilteredRowIterator;
-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.format.SSTableReader;
@@ -52,17 +51,17 @@ public class SplittingSizeTieredCompactionWriter extends CompactionAwareWriter
private long currentBytesToWrite;
private int currentRatioIndex = 0;
- public SplittingSizeTieredCompactionWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, OperationType compactionType)
+ public SplittingSizeTieredCompactionWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables)
{
- this(cfs, txn, nonExpiredSSTables, compactionType, DEFAULT_SMALLEST_SSTABLE_BYTES);
+ this(cfs, txn, nonExpiredSSTables, DEFAULT_SMALLEST_SSTABLE_BYTES);
}
@SuppressWarnings("resource")
- public SplittingSizeTieredCompactionWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, OperationType compactionType, long smallestSSTable)
+ public SplittingSizeTieredCompactionWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, long smallestSSTable)
{
super(cfs, txn, nonExpiredSSTables, false);
this.allSSTables = txn.originals();
- totalSize = cfs.getExpectedCompactedFileSize(nonExpiredSSTables, compactionType);
+ totalSize = cfs.getExpectedCompactedFileSize(nonExpiredSSTables, txn.opType());
double[] potentialRatios = new double[20];
double currentRatio = 1;
for (int i = 0; i < potentialRatios.length; i++)
@@ -86,13 +85,14 @@ public class SplittingSizeTieredCompactionWriter extends CompactionAwareWriter
long currentPartitionsToWrite = Math.round(estimatedTotalKeys * ratios[currentRatioIndex]);
currentBytesToWrite = Math.round(totalSize * ratios[currentRatioIndex]);
@SuppressWarnings("resource")
- SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(sstableDirectory)),
- currentPartitionsToWrite,
- minRepairedAt,
- cfs.metadata,
- cfs.partitioner,
- new MetadataCollector(allSSTables, cfs.metadata.comparator, 0),
- SerializationHeader.make(cfs.metadata, nonExpiredSSTables));
+ SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getSSTablePath(sstableDirectory)),
+ currentPartitionsToWrite,
+ minRepairedAt,
+ cfs.metadata,
+ cfs.partitioner,
+ new MetadataCollector(allSSTables, cfs.metadata.comparator, 0),
+ SerializationHeader.make(cfs.metadata, nonExpiredSSTables),
+ txn);
sstableWriter.switchWriter(writer);
logger.debug("Ratios={}, expectedKeys = {}, totalSize = {}, currentPartitionsToWrite = {}, currentBytesToWrite = {}", ratios, estimatedTotalKeys, totalSize, currentPartitionsToWrite, currentBytesToWrite);
@@ -109,13 +109,14 @@ public class SplittingSizeTieredCompactionWriter extends CompactionAwareWriter
long currentPartitionsToWrite = Math.round(ratios[currentRatioIndex] * estimatedTotalKeys);
File sstableDirectory = cfs.directories.getLocationForDisk(getWriteDirectory(Math.round(totalSize * ratios[currentRatioIndex])));
@SuppressWarnings("resource")
- SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(sstableDirectory)),
- currentPartitionsToWrite,
- minRepairedAt,
- cfs.metadata,
- cfs.partitioner,
- new MetadataCollector(allSSTables, cfs.metadata.comparator, 0),
- SerializationHeader.make(cfs.metadata, nonExpiredSSTables));
+ SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getSSTablePath(sstableDirectory)),
+ currentPartitionsToWrite,
+ minRepairedAt,
+ cfs.metadata,
+ cfs.partitioner,
+ new MetadataCollector(allSSTables, cfs.metadata.comparator, 0),
+ SerializationHeader.make(cfs.metadata, nonExpiredSSTables),
+ txn);
sstableWriter.switchWriter(writer);
logger.debug("Switching writer, currentPartitionsToWrite = {}", currentPartitionsToWrite);
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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
index 536e13c..2b94d7a 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/Helpers.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/Helpers.java
@@ -23,6 +23,7 @@ import com.google.common.base.Predicate;
import com.google.common.collect.*;
import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.utils.Throwables;
import static com.google.common.base.Predicates.*;
import static com.google.common.collect.Iterables.any;
@@ -70,6 +71,16 @@ class Helpers
* 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 void setupKeyCache(Iterable<SSTableReader> readers)
+ {
+ for (SSTableReader reader : readers)
+ reader.setupKeyCache();
+ }
+
+ /**
+ * 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)
@@ -105,18 +116,51 @@ class Helpers
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(Tracker tracker, Iterable<SSTableReader> readers, Throwable accumulate)
+ static Throwable markObsolete(List<TransactionLogs.Obsoletion> obsoletions, Throwable accumulate)
+ {
+ if (obsoletions == null || obsoletions.isEmpty())
+ return accumulate;
+
+ for (TransactionLogs.Obsoletion obsoletion : obsoletions)
+ {
+ try
+ {
+ obsoletion.reader.markObsolete(obsoletion.tidier);
+ }
+ catch (Throwable t)
+ {
+ accumulate = merge(accumulate, t);
+ }
+ }
+ return accumulate;
+ }
+
+ static Throwable prepareForObsoletion(Iterable<SSTableReader> readers, TransactionLogs txnLogs, List<TransactionLogs.Obsoletion> obsoletions, Throwable accumulate)
{
for (SSTableReader reader : readers)
{
try
{
- boolean firstToCompact = reader.markObsolete(tracker);
- assert firstToCompact : reader + " was already marked compacted";
+ obsoletions.add(new TransactionLogs.Obsoletion(reader, txnLogs.obsoleted(reader)));
+ }
+ catch (Throwable t)
+ {
+ accumulate = Throwables.merge(accumulate, t);
+ }
+ }
+ return accumulate;
+ }
+
+ static Throwable abortObsoletion(List<TransactionLogs.Obsoletion> obsoletions, Throwable accumulate)
+ {
+ if (obsoletions == null || obsoletions.isEmpty())
+ return accumulate;
+
+ for (TransactionLogs.Obsoletion obsoletion : obsoletions)
+ {
+ try
+ {
+ obsoletion.tidier.abort();
}
catch (Throwable t)
{
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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
index e14e2a1..b743633 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
@@ -17,6 +17,7 @@
*/
package org.apache.cassandra.db.lifecycle;
+import java.io.File;
import java.util.*;
import com.google.common.annotations.VisibleForTesting;
@@ -27,7 +28,9 @@ import com.google.common.collect.*;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.io.sstable.SSTable;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.sstable.format.SSTableReader.UniqueIdentifier;
import org.apache.cassandra.utils.concurrent.Transactional;
@@ -44,12 +47,18 @@ 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;
+/**
+ * IMPORTANT: When this object is involved in a transactional graph, for correct behaviour its commit MUST occur before
+ * any others, since it may legitimately fail. This is consistent with the Transactional API, which permits one failing
+ * action to occur at the beginning of the commit phase, but also *requires* that the prepareToCommit() phase only take
+ * actions that can be rolled back.
+ */
public class LifecycleTransaction extends Transactional.AbstractTransactional
{
private static final Logger logger = LoggerFactory.getLogger(LifecycleTransaction.class);
/**
- * a class that represents accumulated modifications to the Tracker.
+ * 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()
*/
@@ -86,7 +95,8 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
}
public final Tracker tracker;
- private final OperationType operationType;
+ // The transaction logs keep track of new and old sstable files
+ private final TransactionLogs transactionLogs;
// 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<>();
@@ -95,13 +105,16 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
// 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>());
+ private final Set<UniqueIdentifier> identities = Collections.newSetFromMap(new IdentityHashMap<>());
// changes that have been made visible
private final State logged = new State();
// changes that are pending
private final State staged = new State();
+ // the tidier and their readers, to be used for marking readers obsoleted during a commit
+ private List<TransactionLogs.Obsoletion> obsoletions;
+
/**
* construct a Transaction for use in an offline operation
*/
@@ -122,10 +135,33 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
return new LifecycleTransaction(dummy, operationType, readers);
}
+ /**
+ * construct an empty Transaction with no existing readers
+ */
+ public static LifecycleTransaction offline(OperationType operationType, CFMetaData metadata)
+ {
+ Tracker dummy = new Tracker(null, false);
+ return new LifecycleTransaction(dummy, new TransactionLogs(operationType, metadata, dummy), Collections.emptyList());
+ }
+
+ /**
+ * construct an empty Transaction with no existing readers
+ */
+ public static LifecycleTransaction offline(OperationType operationType, File operationFolder)
+ {
+ Tracker dummy = new Tracker(null, false);
+ return new LifecycleTransaction(dummy, new TransactionLogs(operationType, operationFolder, dummy), Collections.emptyList());
+ }
+
LifecycleTransaction(Tracker tracker, OperationType operationType, Iterable<SSTableReader> readers)
{
+ this(tracker, new TransactionLogs(operationType, getMetadata(tracker, readers), tracker), readers);
+ }
+
+ LifecycleTransaction(Tracker tracker, TransactionLogs transactionLogs, Iterable<SSTableReader> readers)
+ {
this.tracker = tracker;
- this.operationType = operationType;
+ this.transactionLogs = transactionLogs;
for (SSTableReader reader : readers)
{
originals.add(reader);
@@ -134,6 +170,36 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
}
}
+ private static CFMetaData getMetadata(Tracker tracker, Iterable<SSTableReader> readers)
+ {
+ if (tracker.cfstore != null)
+ return tracker.cfstore.metadata;
+
+ for (SSTableReader reader : readers)
+ {
+ if (reader.metadata != null)
+ return reader.metadata;
+ }
+
+ assert false : "Expected cfstore or at least one reader with metadata";
+ return null;
+ }
+
+ public TransactionLogs logs()
+ {
+ return transactionLogs;
+ }
+
+ public OperationType opType()
+ {
+ return transactionLogs.getType();
+ }
+
+ public UUID opId()
+ {
+ return transactionLogs.getId();
+ }
+
public void doPrepare()
{
// note for future: in anticompaction two different operations use the same Transaction, and both prepareToCommit()
@@ -141,6 +207,11 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
// (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();
+
+ // prepare for compaction obsolete readers as long as they were part of the original set
+ // since those that are not original are early readers that share the same desc with the finals
+ maybeFail(prepareForObsoletion(filterIn(logged.obsolete, originals), transactionLogs, obsoletions = new ArrayList<>(), null));
+ transactionLogs.prepareToCommit();
}
/**
@@ -149,16 +220,23 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
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);
+ // accumulate must be null if we have been used correctly, so fail immediately if it is not
+ maybeFail(accumulate);
+
+ // transaction log commit failure means we must abort; safe commit is not possible
+ maybeFail(transactionLogs.commit(null));
+
// 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 = markObsolete(tracker, logged.obsolete, accumulate);
+
+ accumulate = markObsolete(obsoletions, accumulate);
accumulate = tracker.updateSizeTracking(logged.obsolete, logged.update, accumulate);
accumulate = release(selfRefs(logged.obsolete), accumulate);
- accumulate = tracker.notifySSTablesChanged(originals, logged.update, operationType, accumulate);
+ accumulate = tracker.notifySSTablesChanged(originals, logged.update, transactionLogs.getType(), accumulate);
+
return accumulate;
}
@@ -170,15 +248,20 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
if (logger.isDebugEnabled())
logger.debug("Aborting transaction over {}, with ({},{}) logged and ({},{}) staged", originals, logged.update, logged.obsolete, staged.update, staged.obsolete);
+ accumulate = abortObsoletion(obsoletions, accumulate);
+
if (logged.isEmpty() && staged.isEmpty())
- return accumulate;
+ return transactionLogs.abort(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);
- // we don't pass the tracker in for the obsoletion, since these readers have never been notified externally
- // nor had their size accounting affected
- accumulate = markObsolete(null, obsolete, accumulate);
+
+ accumulate = prepareForObsoletion(obsolete, transactionLogs, obsoletions = new ArrayList<>(), accumulate);
+ // it's safe to abort even if committed, see maybeFail in doCommit() above, in this case it will just report
+ // a failure to abort, which is useful information to have for debug
+ accumulate = transactionLogs.abort(accumulate);
+ accumulate = markObsolete(obsoletions, accumulate);
// replace all updated readers with a version restored to its original state
accumulate = tracker.apply(updateLiveSet(logged.update, restoreUpdatedOriginals()), accumulate);
@@ -189,6 +272,7 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
// 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;
@@ -270,8 +354,7 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
}
/**
- * 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
+ * mark this reader as for obsoletion : on checkpoint() the reader will be removed from the live set
*/
public void obsolete(SSTableReader reader)
{
@@ -312,8 +395,7 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
*/
private Iterable<SSTableReader> fresh()
{
- return filterOut(staged.update,
- originals, logged.update);
+ return filterOut(staged.update, originals, logged.update);
}
/**
@@ -332,14 +414,7 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
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);
- }
- }));
+ return ImmutableList.copyOf(transform(torestore, (reader) -> current(reader).cloneWithNewStart(reader.first, null)));
}
/**
@@ -415,7 +490,7 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
originals.remove(reader);
marked.remove(reader);
}
- return new LifecycleTransaction(tracker, operationType, readers);
+ return new LifecycleTransaction(tracker, transactionLogs.getType(), readers);
}
/**
@@ -446,6 +521,31 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
return getFirst(originals, null);
}
+ public void trackNew(SSTable table)
+ {
+ transactionLogs.trackNew(table);
+ }
+
+ public void untrackNew(SSTable table)
+ {
+ transactionLogs.untrackNew(table);
+ }
+
+ public static void removeUnfinishedLeftovers(CFMetaData metadata)
+ {
+ TransactionLogs.removeUnfinishedLeftovers(metadata);
+ }
+
+ public static Set<File> getTemporaryFiles(CFMetaData metadata, File folder)
+ {
+ return TransactionLogs.getTemporaryFiles(metadata, folder);
+ }
+
+ public static Set<File> getLogFiles(CFMetaData metadata)
+ {
+ return TransactionLogs.getLogFiles(metadata);
+ }
+
// 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
@@ -453,7 +553,7 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
@VisibleForTesting
public static class ReaderState
{
- public static enum Action
+ public enum Action
{
UPDATED, OBSOLETED, NONE;
public static Action get(boolean updated, boolean obsoleted)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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
index f1c4685..241eb4b 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
@@ -42,6 +42,7 @@ 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.Throwables;
import org.apache.cassandra.utils.concurrent.OpOrder;
import static com.google.common.base.Predicates.and;
@@ -162,9 +163,11 @@ public class Tracker
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
+
+ // we don't subtract from total until the sstable is deleted, see TransactionLogs.SSTableTidier
cfstore.metric.totalDiskSpaceUsed.inc(add);
return accumulate;
}
@@ -224,29 +227,47 @@ public class Tracker
*/
public Throwable dropSSTables(final Predicate<SSTableReader> remove, OperationType operationType, Throwable accumulate)
{
- Pair<View, View> result = apply(new Function<View, View>()
+ try (TransactionLogs txnLogs = new TransactionLogs(operationType, cfstore.metadata, this))
{
- public View apply(View view)
- {
+ Pair<View, View> result = apply(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);
+ Set<SSTableReader> removed = Sets.difference(result.left.sstables, result.right.sstables);
+ assert Iterables.all(removed, remove);
- if (!removed.isEmpty())
+ // It is important that any method accepting/returning a Throwable never throws an exception, and does its best
+ // to complete the instructions given to it
+ List<TransactionLogs.Obsoletion> obsoletions = new ArrayList<>();
+ accumulate = prepareForObsoletion(removed, txnLogs, obsoletions, accumulate);
+ try
+ {
+ txnLogs.finish();
+ if (!removed.isEmpty())
+ {
+ accumulate = markObsolete(obsoletions, accumulate);
+ accumulate = updateSizeTracking(removed, emptySet(), accumulate);
+ accumulate = release(selfRefs(removed), accumulate);
+ // notifySSTablesChanged -> LeveledManifest.promote doesn't like a no-op "promotion"
+ accumulate = notifySSTablesChanged(removed, Collections.<SSTableReader>emptySet(), txnLogs.getType(), accumulate);
+ }
+ }
+ catch (Throwable t)
+ {
+ accumulate = abortObsoletion(obsoletions, accumulate);
+ accumulate = Throwables.merge(accumulate, t);
+ }
+ }
+ catch (Throwable t)
{
- // 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(this, removed, accumulate);
- accumulate = release(selfRefs(removed), accumulate);
+ accumulate = Throwables.merge(accumulate, t);
}
+
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.
@@ -370,7 +391,6 @@ public class Tracker
sstable.createLinks(FileUtils.getCanonicalPath(backupsDir));
}
-
// NOTIFICATION
Throwable notifySSTablesChanged(Collection<SSTableReader> removed, Collection<SSTableReader> added, OperationType compactionType, Throwable accumulate)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/src/java/org/apache/cassandra/db/lifecycle/TransactionLogs.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/TransactionLogs.java b/src/java/org/apache/cassandra/db/lifecycle/TransactionLogs.java
new file mode 100644
index 0000000..ab6c72a
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/lifecycle/TransactionLogs.java
@@ -0,0 +1,786 @@
+/*
+ * 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.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.util.*;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.lang3.StringUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.cassandra.utils.Throwables.merge;
+
+import org.apache.cassandra.concurrent.ScheduledExecutors;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.Directories;
+import org.apache.cassandra.db.SystemKeyspace;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.CLibrary;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.cassandra.utils.UUIDGen;
+import org.apache.cassandra.utils.concurrent.Blocker;
+import org.apache.cassandra.utils.concurrent.Ref;
+import org.apache.cassandra.utils.concurrent.RefCounted;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+/**
+ * IMPORTANT: When this object is involved in a transactional graph, and is not encapsulated in a LifecycleTransaction,
+ * for correct behaviour its commit MUST occur before any others, since it may legitimately fail. This is consistent
+ * with the Transactional API, which permits one failing action to occur at the beginning of the commit phase, but also
+ * *requires* that the prepareToCommit() phase only take actions that can be rolled back.
+ *
+ * A class that tracks sstable files involved in a transaction across sstables:
+ * if the transaction succeeds the old files should be deleted and the new ones kept; vice-versa if it fails.
+ *
+ * Two log files, NEW and OLD, contain new and old sstable files respectively. The log files also track each
+ * other by referencing each others path in the contents.
+ *
+ * If the transaction finishes successfully:
+ * - the OLD transaction file is deleted along with its contents, this includes the NEW transaction file.
+ * Before deleting we must let the SSTableTidier instances run first for any old readers that are being obsoleted
+ * (mark as compacted) by the transaction, see LifecycleTransaction
+ *
+ * If the transaction is aborted:
+ * - the NEW transaction file and its contents are deleted, this includes the OLD transaction file
+ *
+ * On start-up:
+ * - If we find a NEW transaction file, it means the transaction did not complete and we delete the NEW file and its contents
+ * - If we find an OLD transaction file but not a NEW file, it means the transaction must have completed and so we delete
+ * all the contents of the OLD file, if they still exist, and the OLD file itself.
+ *
+ * See CASSANDRA-7066 for full details.
+ */
+public class TransactionLogs extends Transactional.AbstractTransactional implements Transactional
+{
+ private static final Logger logger = LoggerFactory.getLogger(TransactionLogs.class);
+
+ /**
+ * A single transaction log file, either NEW or OLD.
+ */
+ final static class TransactionFile
+ {
+ static String EXT = ".log";
+ static char SEP = '_';
+ static String REGEX_STR = String.format("^(.*)_(.*)_(%s|%s)%s$", Type.NEW.txt, Type.OLD.txt, EXT);
+ static Pattern REGEX = Pattern.compile(REGEX_STR); //(opname)_(id)_(new|old).data
+
+ public enum Type
+ {
+ NEW (0, "new"),
+ OLD (1, "old");
+
+ public final int idx;
+ public final String txt;
+
+ Type(int idx, String txt)
+ {
+ this.idx = idx;
+ this.txt = txt;
+ }
+ };
+
+ public final Type type;
+ public final File file;
+ public final TransactionData parent;
+ public final Set<String> lines = new HashSet<>();
+
+ public TransactionFile(Type type, TransactionData parent)
+ {
+ this.type = type;
+ this.file = new File(parent.getFileName(type));
+ this.parent = parent;
+
+ if (exists())
+ lines.addAll(FileUtils.readLines(file));
+ }
+
+ public boolean add(SSTable table)
+ {
+ return add(table.descriptor.baseFilename());
+ }
+
+ private boolean add(String path)
+ {
+ String relativePath = FileUtils.getRelativePath(parent.getParentFolder(), path);
+ if (lines.contains(relativePath))
+ return false;
+
+ lines.add(relativePath);
+ FileUtils.append(file, relativePath);
+ return true;
+ }
+
+ public void remove(SSTable table)
+ {
+ String relativePath = FileUtils.getRelativePath(parent.getParentFolder(), table.descriptor.baseFilename());
+ assert lines.contains(relativePath) : String.format("%s is not tracked by %s", relativePath, file);
+
+ lines.remove(relativePath);
+ delete(relativePath);
+ }
+
+ public boolean contains(SSTable table)
+ {
+ String relativePath = FileUtils.getRelativePath(parent.getParentFolder(), table.descriptor.baseFilename());
+ return lines.contains(relativePath);
+ }
+
+ private void deleteContents()
+ {
+ deleteOpposite();
+
+ // we sync the parent file descriptor between opposite log deletion and
+ // contents deletion to ensure there is a happens before edge between them
+ parent.sync();
+
+ lines.forEach(line -> delete(line));
+ lines.clear();
+ }
+
+ private void deleteOpposite()
+ {
+ Type oppositeType = type == Type.NEW ? Type.OLD : Type.NEW;
+ String oppositeFile = FileUtils.getRelativePath(parent.getParentFolder(), parent.getFileName(oppositeType));
+ assert lines.contains(oppositeFile) : String.format("Could not find %s amongst lines", oppositeFile);
+
+ delete(oppositeFile);
+ lines.remove(oppositeFile);
+ }
+
+ private void delete(String relativePath)
+ {
+ getTrackedFiles(relativePath).forEach(file -> TransactionLogs.delete(file));
+ }
+
+ public Set<File> getTrackedFiles()
+ {
+ Set<File> ret = new HashSet<>();
+ FileUtils.readLines(file).forEach(line -> ret.addAll(getTrackedFiles(line)));
+ ret.add(file);
+ return ret;
+ }
+
+ private List<File> getTrackedFiles(String relativePath)
+ {
+ List<File> ret = new ArrayList<>();
+ File file = new File(StringUtils.join(parent.getParentFolder(), File.separator, relativePath));
+ if (file.exists())
+ ret.add(file);
+ else
+ ret.addAll(Arrays.asList(new File(parent.getParentFolder()).listFiles((dir, name) -> {
+ return name.startsWith(relativePath);
+ })));
+
+ return ret;
+ }
+
+ public void delete(boolean deleteContents)
+ {
+ assert file.exists() : String.format("Expected %s to exists", file);
+
+ if (deleteContents)
+ deleteContents();
+
+ // we sync the parent file descriptor between contents and log deletion
+ // to ensure there is a happens before edge between them
+ parent.sync();
+
+ TransactionLogs.delete(file);
+ }
+
+ public boolean exists()
+ {
+ return file.exists();
+ }
+ }
+
+ /**
+ * We split the transaction data from the behavior because we need
+ * to reconstruct any left-overs and clean them up, as well as work
+ * out which files are temporary. So for these cases we don't want the full
+ * transactional behavior, plus it's handy for the TransactionTidier.
+ */
+ final static class TransactionData implements AutoCloseable
+ {
+ private final OperationType opType;
+ private final UUID id;
+ private final File folder;
+ private final TransactionFile[] files;
+ private int folderDescriptor;
+ private boolean succeeded;
+
+ static TransactionData make(File logFile)
+ {
+ Matcher matcher = TransactionFile.REGEX.matcher(logFile.getName());
+ assert matcher.matches();
+
+ OperationType operationType = OperationType.fromFileName(matcher.group(1));
+ UUID id = UUID.fromString(matcher.group(2));
+
+ return new TransactionData(operationType, logFile.getParentFile(), id);
+ }
+
+ TransactionData(OperationType opType, File folder, UUID id)
+ {
+ this.opType = opType;
+ this.id = id;
+ this.folder = folder;
+ this.files = new TransactionFile[TransactionFile.Type.values().length];
+ for (TransactionFile.Type t : TransactionFile.Type.values())
+ this.files[t.idx] = new TransactionFile(t, this);
+
+ this.folderDescriptor = CLibrary.tryOpenDirectory(folder.getPath());
+ this.succeeded = !newLog().exists() && oldLog().exists();
+ }
+
+ public void succeeded(boolean succeeded)
+ {
+ this.succeeded = succeeded;
+ }
+
+ public void close()
+ {
+ if (folderDescriptor > 0)
+ {
+ CLibrary.tryCloseFD(folderDescriptor);
+ folderDescriptor = -1;
+ }
+ }
+
+ void crossReference()
+ {
+ newLog().add(oldLog().file.getPath());
+ oldLog().add(newLog().file.getPath());
+ }
+
+ void sync()
+ {
+ if (folderDescriptor > 0)
+ CLibrary.trySync(folderDescriptor);
+ }
+
+ TransactionFile newLog()
+ {
+ return files[TransactionFile.Type.NEW.idx];
+ }
+
+ TransactionFile oldLog()
+ {
+ return files[TransactionFile.Type.OLD.idx];
+ }
+
+ OperationType getType()
+ {
+ return opType;
+ }
+
+ UUID getId()
+ {
+ return id;
+ }
+
+ Throwable removeUnfinishedLeftovers(Throwable accumulate)
+ {
+ try
+ {
+ if (succeeded)
+ oldLog().delete(true);
+ else
+ newLog().delete(true);
+ }
+ catch (Throwable t)
+ {
+ accumulate = merge(accumulate, t);
+ }
+
+ return accumulate;
+ }
+
+ Set<File> getTemporaryFiles()
+ {
+ sync();
+
+ if (newLog().exists())
+ return newLog().getTrackedFiles();
+ else
+ return oldLog().getTrackedFiles();
+ }
+
+ String getFileName(TransactionFile.Type type)
+ {
+ String fileName = StringUtils.join(opType.fileName,
+ TransactionFile.SEP,
+ id.toString(),
+ TransactionFile.SEP,
+ type.txt,
+ TransactionFile.EXT);
+ return StringUtils.join(folder, File.separator, fileName);
+ }
+
+ String getParentFolder()
+ {
+ return folder.getParent();
+ }
+
+ static boolean isLogFile(String name)
+ {
+ return TransactionFile.REGEX.matcher(name).matches();
+ }
+ }
+
+ private final Tracker tracker;
+ private final TransactionData data;
+ private final Ref<TransactionLogs> selfRef;
+ // Deleting sstables is tricky because the mmapping might not have been finalized yet,
+ // and delete will fail (on Windows) until it is (we only force the unmapping on SUN VMs).
+ // 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 Queue<Runnable> failedDeletions = new ConcurrentLinkedQueue<>();
+ private static final Blocker blocker = new Blocker();
+
+ TransactionLogs(OperationType opType, CFMetaData metadata)
+ {
+ this(opType, metadata, null);
+ }
+
+ TransactionLogs(OperationType opType, CFMetaData metadata, Tracker tracker)
+ {
+ this(opType, new Directories(metadata), tracker);
+ }
+
+ TransactionLogs(OperationType opType, Directories directories, Tracker tracker)
+ {
+ this(opType, directories.getDirectoryForNewSSTables(), tracker);
+ }
+
+ TransactionLogs(OperationType opType, File folder, Tracker tracker)
+ {
+ this.tracker = tracker;
+ this.data = new TransactionData(opType,
+ Directories.getTransactionsDirectory(folder),
+ UUIDGen.getTimeUUID());
+ this.selfRef = new Ref<>(this, new TransactionTidier(data));
+
+ data.crossReference();
+ if (logger.isDebugEnabled())
+ logger.debug("Created transaction logs with id {}", data.id);
+ }
+
+ /**
+ * Track a reader as new.
+ **/
+ void trackNew(SSTable table)
+ {
+ if (!data.newLog().add(table))
+ throw new IllegalStateException(table + " is already tracked as new");
+
+ data.newLog().add(table);
+ }
+
+ /**
+ * Stop tracking a reader as new.
+ */
+ void untrackNew(SSTable table)
+ {
+ data.newLog().remove(table);
+ }
+
+ /**
+ * Schedule a reader for deletion as soon as it is fully unreferenced and the transaction
+ * has been committed.
+ */
+ SSTableTidier obsoleted(SSTableReader reader)
+ {
+ if (data.newLog().contains(reader))
+ {
+ if (data.oldLog().contains(reader))
+ throw new IllegalArgumentException();
+
+ return new SSTableTidier(reader, true, this);
+ }
+
+ if (!data.oldLog().add(reader))
+ throw new IllegalStateException();
+
+ if (tracker != null)
+ tracker.notifyDeleting(reader);
+
+ return new SSTableTidier(reader, false, this);
+ }
+
+ OperationType getType()
+ {
+ return data.getType();
+ }
+
+ UUID getId()
+ {
+ return data.getId();
+ }
+
+ @VisibleForTesting
+ String getDataFolder()
+ {
+ return data.getParentFolder();
+ }
+
+ @VisibleForTesting
+ String getLogsFolder()
+ {
+ return StringUtils.join(getDataFolder(), File.separator, Directories.TRANSACTIONS_SUBDIR);
+ }
+
+ @VisibleForTesting
+ TransactionData getData()
+ {
+ return data;
+ }
+
+ private static void delete(File file)
+ {
+ try
+ {
+ if (logger.isDebugEnabled())
+ logger.debug("Deleting {}", file);
+
+ Files.delete(file.toPath());
+ }
+ catch (NoSuchFileException e)
+ {
+ logger.warn("Unable to delete {} as it does not exist", file);
+ }
+ catch (IOException e)
+ {
+ logger.error("Unable to delete {}", file, e);
+ throw new RuntimeException(e);
+ }
+ }
+
+ /**
+ * The transaction tidier.
+ *
+ * When the transaction reference is fully released we try to delete all the obsolete files
+ * depending on the transaction result.
+ */
+ private static class TransactionTidier implements RefCounted.Tidy, Runnable
+ {
+ private final TransactionData data;
+
+ public TransactionTidier(TransactionData data)
+ {
+ this.data = data;
+ }
+
+ public void tidy() throws Exception
+ {
+ run();
+ }
+
+ public String name()
+ {
+ return data.id.toString();
+ }
+
+ public void run()
+ {
+ if (logger.isDebugEnabled())
+ logger.debug("Removing files for transaction {}", name());
+
+ Throwable err = data.removeUnfinishedLeftovers(null);
+
+ if (err != null)
+ {
+ logger.info("Failed deleting files for transaction {}, we'll retry after GC and on on server restart", name(), err);
+ failedDeletions.add(this);
+ }
+ else
+ {
+ if (logger.isDebugEnabled())
+ logger.debug("Closing file transaction {}", name());
+ data.close();
+ }
+ }
+ }
+
+ static class Obsoletion
+ {
+ final SSTableReader reader;
+ final SSTableTidier tidier;
+
+ public Obsoletion(SSTableReader reader, SSTableTidier tidier)
+ {
+ this.reader = reader;
+ this.tidier = tidier;
+ }
+ }
+
+ /**
+ * The SSTableReader tidier. When a reader is fully released and no longer referenced
+ * by any one, we run this. It keeps a reference to the parent transaction and releases
+ * it when done, so that the final transaction cleanup can run when all obsolete readers
+ * are released.
+ */
+ public static class SSTableTidier implements Runnable
+ {
+ // must not retain a reference to the SSTableReader, else leak detection cannot kick in
+ private final Descriptor desc;
+ private final long sizeOnDisk;
+ private final Tracker tracker;
+ private final boolean wasNew;
+ private final Ref<TransactionLogs> parentRef;
+
+ public SSTableTidier(SSTableReader referent, boolean wasNew, TransactionLogs parent)
+ {
+ this.desc = referent.descriptor;
+ this.sizeOnDisk = referent.bytesOnDisk();
+ this.tracker = parent.tracker;
+ this.wasNew = wasNew;
+ this.parentRef = parent.selfRef.tryRef();
+ }
+
+ public void run()
+ {
+ blocker.ask();
+
+ SystemKeyspace.clearSSTableReadMeter(desc.ksname, desc.cfname, desc.generation);
+
+ try
+ {
+ // If we can't successfully delete the DATA component, set the task to be retried later: see TransactionTidier
+ File datafile = new File(desc.filenameFor(Component.DATA));
+
+ delete(datafile);
+ // let the remainder be cleaned up by delete
+ SSTable.delete(desc, SSTable.discoverComponentsFor(desc));
+ }
+ catch (Throwable t)
+ {
+ logger.error("Failed deletion for {}, we'll retry after GC and on server restart", desc);
+ failedDeletions.add(this);
+ return;
+ }
+
+ if (tracker != null && !wasNew)
+ tracker.cfstore.metric.totalDiskSpaceUsed.dec(sizeOnDisk);
+
+ // release the referent to the parent so that the all transaction files can be released
+ parentRef.release();
+ }
+
+ public void abort()
+ {
+ parentRef.release();
+ }
+ }
+
+ /**
+ * Retry all deletions that failed the first time around (presumably b/c the sstable was still mmap'd.)
+ * Useful because there are times when we know GC has been invoked; also exposed as an mbean.
+ */
+ public static void rescheduleFailedDeletions()
+ {
+ Runnable task;
+ while ( null != (task = failedDeletions.poll()))
+ ScheduledExecutors.nonPeriodicTasks.submit(task);
+ }
+
+ /**
+ * Deletions run on the nonPeriodicTasks executor, (both failedDeletions or global tidiers in SSTableReader)
+ * so by scheduling a new empty task and waiting for it we ensure any prior deletion has completed.
+ */
+ public static void waitForDeletions()
+ {
+ FBUtilities.waitOnFuture(ScheduledExecutors.nonPeriodicTasks.schedule(() -> {
+ }, 0, TimeUnit.MILLISECONDS));
+ }
+
+ @VisibleForTesting
+ public static void pauseDeletions(boolean stop)
+ {
+ blocker.block(stop);
+ }
+
+ private Throwable complete(Throwable accumulate)
+ {
+ try
+ {
+ try
+ {
+ if (data.succeeded)
+ data.newLog().delete(false);
+ else
+ data.oldLog().delete(false);
+ }
+ catch (Throwable t)
+ {
+ accumulate = merge(accumulate, t);
+ }
+
+ accumulate = selfRef.ensureReleased(accumulate);
+ return accumulate;
+ }
+ catch (Throwable t)
+ {
+ logger.error("Failed to complete file transaction {}", getId(), t);
+ return Throwables.merge(accumulate, t);
+ }
+ }
+
+ protected Throwable doCommit(Throwable accumulate)
+ {
+ data.succeeded(true);
+ return complete(accumulate);
+ }
+
+ protected Throwable doAbort(Throwable accumulate)
+ {
+ data.succeeded(false);
+ return complete(accumulate);
+ }
+
+ protected void doPrepare() { }
+
+ /**
+ * Called on startup to scan existing folders for any unfinished leftovers of
+ * operations that were ongoing when the process exited.
+ *
+ * We check if the new transaction file exists first, and if so we clean it up
+ * along with its contents, which includes the old file, else if only the old file exists
+ * it means the operation has completed and we only cleanup the old file with its contents.
+ */
+ static void removeUnfinishedLeftovers(CFMetaData metadata)
+ {
+ Throwable accumulate = null;
+ Set<UUID> ids = new HashSet<>();
+
+ for (File dir : getFolders(metadata, null))
+ {
+ File[] logs = dir.listFiles((dir1, name) -> {
+ return TransactionData.isLogFile(name);
+ });
+
+ for (File log : logs)
+ {
+ try (TransactionData data = TransactionData.make(log))
+ {
+ // we need to check this because there are potentially 2 log files per operation
+ if (ids.contains(data.id))
+ continue;
+
+ ids.add(data.id);
+ accumulate = data.removeUnfinishedLeftovers(accumulate);
+ }
+ }
+ }
+
+ if (accumulate != null)
+ logger.error("Failed to remove unfinished transaction leftovers", accumulate);
+ }
+
+ /**
+ * Return a set of files that are temporary, that is they are involved with
+ * a transaction that hasn't completed yet.
+ *
+ * Only return the files that exist and that are located in the folder
+ * specified as a parameter or its sub-folders.
+ */
+ static Set<File> getTemporaryFiles(CFMetaData metadata, File folder)
+ {
+ Set<File> ret = new HashSet<>();
+ Set<UUID> ids = new HashSet<>();
+
+ for (File dir : getFolders(metadata, folder))
+ {
+ File[] logs = dir.listFiles((dir1, name) -> {
+ return TransactionData.isLogFile(name);
+ });
+
+ for (File log : logs)
+ {
+ try(TransactionData data = TransactionData.make(log))
+ {
+ // we need to check this because there are potentially 2 log files per transaction
+ if (ids.contains(data.id))
+ continue;
+
+ ids.add(data.id);
+ ret.addAll(data.getTemporaryFiles()
+ .stream()
+ .filter(file -> FileUtils.isContained(folder, file))
+ .collect(Collectors.toSet()));
+ }
+ }
+ }
+
+ return ret;
+ }
+
+ /**
+ * Return the transaction log files that currently exist for this table.
+ */
+ static Set<File> getLogFiles(CFMetaData metadata)
+ {
+ Set<File> ret = new HashSet<>();
+ for (File dir : getFolders(metadata, null))
+ ret.addAll(Arrays.asList(dir.listFiles((dir1, name) -> {
+ return TransactionData.isLogFile(name);
+ })));
+
+ return ret;
+ }
+
+ /**
+ * A utility method to work out the existing transaction sub-folders
+ * either for a table, or a specific parent folder, or both.
+ */
+ private static List<File> getFolders(CFMetaData metadata, File folder)
+ {
+ List<File> ret = new ArrayList<>();
+ if (metadata != null)
+ {
+ Directories directories = new Directories(metadata);
+ ret.addAll(directories.getExistingDirectories(Directories.TRANSACTIONS_SUBDIR));
+ }
+
+ if (folder != null)
+ {
+ File opDir = Directories.getExistingDirectory(folder, Directories.TRANSACTIONS_SUBDIR);
+ if (opDir != null)
+ ret.add(opDir);
+ }
+
+ return ret;
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
index 174e634..acc9141 100644
--- a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
@@ -32,7 +32,6 @@ import org.apache.cassandra.db.rows.EncodingStats;
import org.apache.cassandra.db.partitions.PartitionUpdate;
import org.apache.cassandra.dht.IPartitioner;
import org.apache.cassandra.io.sstable.format.SSTableFormat;
-import org.apache.cassandra.io.sstable.format.SSTableWriter;
import org.apache.cassandra.service.ActiveRepairService;
import org.apache.cassandra.utils.Pair;
@@ -60,18 +59,19 @@ abstract class AbstractSSTableSimpleWriter implements Closeable
this.formatType = type;
}
- protected SSTableWriter createWriter()
+ protected SSTableTxnWriter createWriter()
{
- return SSTableWriter.create(createDescriptor(directory, metadata.ksName, metadata.cfName, formatType),
+ return SSTableTxnWriter.create(createDescriptor(directory, metadata.ksName, metadata.cfName, formatType),
0,
ActiveRepairService.UNREPAIRED_SSTABLE,
+ 0,
new SerializationHeader(metadata, columns, EncodingStats.NO_STATS));
}
private static Descriptor createDescriptor(File directory, final String keyspace, final String columnFamily, final SSTableFormat.Type fmt)
{
int maxGen = getNextGeneration(directory, columnFamily);
- return new Descriptor(directory, keyspace, columnFamily, maxGen + 1, Descriptor.Type.TEMP, fmt);
+ return new Descriptor(directory, keyspace, columnFamily, maxGen + 1, fmt);
}
private static int getNextGeneration(File directory, final String columnFamily)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/src/java/org/apache/cassandra/io/sstable/Descriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/Descriptor.java b/src/java/org/apache/cassandra/io/sstable/Descriptor.java
index 5ab99e7..519f14e 100644
--- a/src/java/org/apache/cassandra/io/sstable/Descriptor.java
+++ b/src/java/org/apache/cassandra/io/sstable/Descriptor.java
@@ -18,9 +18,7 @@
package org.apache.cassandra.io.sstable;
import java.io.File;
-import java.util.ArrayDeque;
-import java.util.Deque;
-import java.util.StringTokenizer;
+import java.util.*;
import com.google.common.base.CharMatcher;
import com.google.common.base.Objects;
@@ -46,47 +44,35 @@ import static org.apache.cassandra.io.sstable.Component.separator;
*/
public class Descriptor
{
- public static enum Type
- {
- TEMP("tmp", true), TEMPLINK("tmplink", true), FINAL(null, false);
- public final boolean isTemporary;
- public final String marker;
- Type(String marker, boolean isTemporary)
- {
- this.isTemporary = isTemporary;
- this.marker = marker;
- }
- }
-
+ public static String TMP_EXT = ".tmp";
public final File directory;
/** version has the following format: <code>[a-z]+</code> */
public final Version version;
public final String ksname;
public final String cfname;
public final int generation;
- public final Type type;
public final SSTableFormat.Type formatType;
private final int hashCode;
/**
* A descriptor that assumes CURRENT_VERSION.
*/
- public Descriptor(File directory, String ksname, String cfname, int generation, Type temp)
+ public Descriptor(File directory, String ksname, String cfname, int generation)
{
- this(DatabaseDescriptor.getSSTableFormat().info.getLatestVersion(), directory, ksname, cfname, generation, temp, DatabaseDescriptor.getSSTableFormat());
+ this(DatabaseDescriptor.getSSTableFormat().info.getLatestVersion(), directory, ksname, cfname, generation, DatabaseDescriptor.getSSTableFormat());
}
- public Descriptor(File directory, String ksname, String cfname, int generation, Type temp, SSTableFormat.Type formatType)
+ public Descriptor(File directory, String ksname, String cfname, int generation, SSTableFormat.Type formatType)
{
- this(formatType.info.getLatestVersion(), directory, ksname, cfname, generation, temp, formatType);
+ this(formatType.info.getLatestVersion(), directory, ksname, cfname, generation, formatType);
}
- public Descriptor(String version, File directory, String ksname, String cfname, int generation, Type temp, SSTableFormat.Type formatType)
+ public Descriptor(String version, File directory, String ksname, String cfname, int generation, SSTableFormat.Type formatType)
{
- this(formatType.info.getVersion(version), directory, ksname, cfname, generation, temp, formatType);
+ this(formatType.info.getVersion(version), directory, ksname, cfname, generation, formatType);
}
- public Descriptor(Version version, File directory, String ksname, String cfname, int generation, Type temp, SSTableFormat.Type formatType)
+ public Descriptor(Version version, File directory, String ksname, String cfname, int generation, SSTableFormat.Type formatType)
{
assert version != null && directory != null && ksname != null && cfname != null && formatType.info.getLatestVersion().getClass().equals(version.getClass());
this.version = version;
@@ -94,20 +80,24 @@ public class Descriptor
this.ksname = ksname;
this.cfname = cfname;
this.generation = generation;
- this.type = temp;
this.formatType = formatType;
- hashCode = Objects.hashCode(version, directory, generation, ksname, cfname, temp, formatType);
+ hashCode = Objects.hashCode(version, directory, generation, ksname, cfname, formatType);
}
public Descriptor withGeneration(int newGeneration)
{
- return new Descriptor(version, directory, ksname, cfname, newGeneration, type, formatType);
+ return new Descriptor(version, directory, ksname, cfname, newGeneration, formatType);
}
public Descriptor withFormatType(SSTableFormat.Type newType)
{
- return new Descriptor(newType.info.getLatestVersion(), directory, ksname, cfname, generation, type, newType);
+ return new Descriptor(newType.info.getLatestVersion(), directory, ksname, cfname, generation, newType);
+ }
+
+ public String tmpFilenameFor(Component component)
+ {
+ return filenameFor(component) + TMP_EXT;
}
public String filenameFor(Component component)
@@ -130,8 +120,6 @@ public class Descriptor
buff.append(ksname).append(separator);
buff.append(cfname).append(separator);
}
- if (type.isTemporary)
- buff.append(type.marker).append(separator);
buff.append(version).append(separator);
buff.append(generation);
if (formatType != SSTableFormat.Type.LEGACY)
@@ -160,6 +148,37 @@ public class Descriptor
return baseFilename() + separator + suffix;
}
+
+ /** Return any temporary files found in the directory */
+ public List<File> getTemporaryFiles()
+ {
+ List<File> ret = new ArrayList<>();
+ File[] tmpFiles = directory.listFiles((dir, name) ->
+ name.endsWith(Descriptor.TMP_EXT));
+
+ for (File tmpFile : tmpFiles)
+ ret.add(tmpFile);
+
+ return ret;
+ }
+
+ /**
+ * Files obsoleted by CASSANDRA-7066 :
+ * - temporary files used to start with either tmp or tmplink
+ * - system.compactions_in_progress sstable files
+ */
+ public static boolean isLegacyFile(String fileName)
+ {
+ return fileName.startsWith("compactions_in_progress") ||
+ fileName.startsWith("tmp") ||
+ fileName.startsWith("tmplink");
+ }
+
+ public static boolean isValidFile(String fileName)
+ {
+ return fileName.endsWith(".db") && !isLegacyFile(fileName);
+ }
+
/**
* @see #fromFilename(File directory, String name)
* @param filename The SSTable filename
@@ -222,7 +241,7 @@ public class Descriptor
String component = skipComponent ? null : tokenStack.pop();
nexttok = tokenStack.pop();
- // generation OR Type
+ // generation OR format type
SSTableFormat.Type fmt = SSTableFormat.Type.LEGACY;
if (!CharMatcher.DIGIT.matchesAllOf(nexttok))
{
@@ -240,20 +259,6 @@ public class Descriptor
if (!version.validate(nexttok))
throw new UnsupportedOperationException("SSTable " + name + " is too old to open. Upgrade to 2.0 first, and run upgradesstables");
- // optional temporary marker
- Type type = Descriptor.Type.FINAL;
- nexttok = tokenStack.peek();
- if (Descriptor.Type.TEMP.marker.equals(nexttok))
- {
- type = Descriptor.Type.TEMP;
- tokenStack.pop();
- }
- else if (Descriptor.Type.TEMPLINK.marker.equals(nexttok))
- {
- type = Descriptor.Type.TEMPLINK;
- tokenStack.pop();
- }
-
// ks/cf names
String ksname, cfname;
if (version.hasNewFileName())
@@ -285,16 +290,7 @@ public class Descriptor
}
assert tokenStack.isEmpty() : "Invalid file name " + name + " in " + directory;
- return Pair.create(new Descriptor(version, parentDirectory, ksname, cfname, generation, type, fmt), component);
- }
-
- /**
- * @param type temporary flag
- * @return A clone of this descriptor with the given 'temporary' status.
- */
- public Descriptor asType(Type type)
- {
- return new Descriptor(version, directory, ksname, cfname, generation, type, formatType);
+ return Pair.create(new Descriptor(version, parentDirectory, ksname, cfname, generation, fmt), component);
}
public IMetadataSerializer getMetadataSerializer()
@@ -331,8 +327,7 @@ public class Descriptor
&& that.generation == this.generation
&& that.ksname.equals(this.ksname)
&& that.cfname.equals(this.cfname)
- && that.formatType == this.formatType
- && that.type == this.type;
+ && that.formatType == this.formatType;
}
@Override
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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 6f66fd3..e6558eb 100644
--- a/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
+++ b/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
@@ -259,11 +259,12 @@ public class IndexSummaryManager implements IndexSummaryManagerMBean
{
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())
{
+ // 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);
+
oldFormatSSTables.add(sstable);
txn.cancel(sstable);
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/src/java/org/apache/cassandra/io/sstable/SSTable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTable.java b/src/java/org/apache/cassandra/io/sstable/SSTable.java
index 2077152..516534d 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTable.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTable.java
@@ -22,6 +22,7 @@ import java.nio.charset.Charset;
import java.util.*;
import java.util.concurrent.CopyOnWriteArraySet;
+import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Predicates;
import com.google.common.collect.Collections2;
import com.google.common.collect.Sets;
@@ -38,7 +39,6 @@ import org.apache.cassandra.io.FSWriteError;
import org.apache.cassandra.io.util.FileUtils;
import org.apache.cassandra.io.util.RandomAccessReader;
import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.concurrent.RefCounted;
import org.apache.cassandra.utils.memory.HeapAllocator;
import org.apache.cassandra.utils.Pair;
@@ -71,7 +71,7 @@ public abstract class SSTable
protected SSTable(Descriptor descriptor, CFMetaData metadata, IPartitioner partitioner)
{
- this(descriptor, new HashSet<Component>(), metadata, partitioner);
+ this(descriptor, new HashSet<>(), metadata, partitioner);
}
protected SSTable(Descriptor descriptor, Set<Component> components, CFMetaData metadata, IPartitioner partitioner)
@@ -113,7 +113,9 @@ public abstract class SSTable
FileUtils.deleteWithConfirm(desc.filenameFor(component));
}
- FileUtils.delete(desc.filenameFor(Component.SUMMARY));
+
+ if (components.contains(Component.SUMMARY))
+ FileUtils.delete(desc.filenameFor(Component.SUMMARY));
logger.debug("Deleted {}", desc);
return true;
@@ -150,6 +152,15 @@ public abstract class SSTable
return descriptor.ksname;
}
+ @VisibleForTesting
+ public List<String> getAllFilePaths()
+ {
+ List<String> ret = new ArrayList<>();
+ for (Component component : components)
+ ret.add(descriptor.filenameFor(component));
+ return ret;
+ }
+
/**
* @return Descriptor and Component pair. null if given file is not acceptable as SSTable component.
* If component is of unknown type, returns CUSTOM component.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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
deleted file mode 100644
index f0eb67f..0000000
--- a/src/java/org/apache/cassandra/io/sstable/SSTableDeletingTask.java
+++ /dev/null
@@ -1,130 +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.io.sstable;
-
-import java.io.File;
-import java.util.Collections;
-import java.util.Queue;
-import java.util.Set;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.TimeUnit;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Sets;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.codahale.metrics.Counter;
-import org.apache.cassandra.concurrent.ScheduledExecutors;
-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
-{
- private static final Logger logger = LoggerFactory.getLogger(SSTableDeletingTask.class);
-
- // Deleting sstables is tricky because the mmapping might not have been finalized yet,
- // and delete will fail (on Windows) until it is (we only force the unmapping on SUN VMs).
- // 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 Queue<SSTableDeletingTask> failedTasks = new ConcurrentLinkedQueue<>();
- private static final Blocker blocker = new Blocker();
-
- private final Descriptor desc;
- private final Set<Component> components;
- private final long bytesOnDisk;
- private final Counter totalDiskSpaceUsed;
-
- /**
- * realDescriptor is the actual descriptor for the sstable, the descriptor inside
- * referent can be 'faked' as FINAL for early opened files. We need the real one
- * to be able to remove the files.
- */
- public SSTableDeletingTask(Descriptor realDescriptor, Set<Component> components, Counter totalDiskSpaceUsed, long bytesOnDisk)
- {
- this.desc = realDescriptor;
- this.bytesOnDisk = bytesOnDisk;
- this.totalDiskSpaceUsed = totalDiskSpaceUsed;
- switch (desc.type)
- {
- case FINAL:
- this.components = components;
- break;
- case TEMPLINK:
- this.components = Sets.newHashSet(Component.DATA, Component.PRIMARY_INDEX);
- break;
- default:
- throw new IllegalStateException();
- }
- }
-
- public void schedule()
- {
- ScheduledExecutors.nonPeriodicTasks.submit(this);
- }
-
- public void run()
- {
- blocker.ask();
- // If we can't successfully delete the DATA component, set the task to be retried later: see above
- File datafile = new File(desc.filenameFor(Component.DATA));
- if (!datafile.delete())
- {
- logger.error("Unable to delete {} (it will be removed on server restart; we'll also retry after GC)", datafile);
- failedTasks.add(this);
- return;
- }
- // let the remainder be cleaned up by delete
- SSTable.delete(desc, Sets.difference(components, Collections.singleton(Component.DATA)));
- if (totalDiskSpaceUsed != null)
- totalDiskSpaceUsed.dec(bytesOnDisk);
- }
-
- /**
- * Retry all deletions that failed the first time around (presumably b/c the sstable was still mmap'd.)
- * Useful because there are times when we know GC has been invoked; also exposed as an mbean.
- */
- public static void rescheduleFailedTasks()
- {
- SSTableDeletingTask task;
- while ( null != (task = failedTasks.poll()))
- task.schedule();
- }
-
- /** for tests */
- @VisibleForTesting
- public static void waitForDeletions()
- {
- Runnable runnable = new Runnable()
- {
- public void run()
- {
- }
- };
-
- FBUtilities.waitOnFuture(ScheduledExecutors.nonPeriodicTasks.schedule(runnable, 0, TimeUnit.MILLISECONDS));
- }
-
- @VisibleForTesting
- public static void pauseDeletions(boolean stop)
- {
- blocker.block(stop);
- }
-}
-
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java b/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
index b99003b..f25d3ff 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
@@ -28,6 +28,7 @@ import com.google.common.collect.Multimap;
import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.dht.IPartitioner;
import org.apache.cassandra.dht.Range;
import org.apache.cassandra.dht.Token;
@@ -78,13 +79,17 @@ public class SSTableLoader implements StreamEventHandler
directory.list(new FilenameFilter()
{
+ final Map<File, Set<File>> allTemporaryFiles = new HashMap<>();
public boolean accept(File dir, String name)
{
- if (new File(dir, name).isDirectory())
+ File file = new File(dir, name);
+
+ if (file.isDirectory())
return false;
+
Pair<Descriptor, Component> p = SSTable.tryComponentFromFilename(dir, name);
Descriptor desc = p == null ? null : p.left;
- if (p == null || !p.right.equals(Component.DATA) || desc.type.isTemporary)
+ if (p == null || !p.right.equals(Component.DATA))
return false;
if (!new File(desc.filenameFor(Component.PRIMARY_INDEX)).exists())
@@ -100,6 +105,19 @@ public class SSTableLoader implements StreamEventHandler
return false;
}
+ Set<File> temporaryFiles = allTemporaryFiles.get(dir);
+ if (temporaryFiles == null)
+ {
+ temporaryFiles = LifecycleTransaction.getTemporaryFiles(metadata, dir);
+ allTemporaryFiles.put(dir, temporaryFiles);
+ }
+
+ if (temporaryFiles.contains(file))
+ {
+ outputHandler.output(String.format("Skipping temporary file %s", name));
+ return false;
+ }
+
Set<Component> components = new HashSet<>();
components.add(Component.DATA);
components.add(Component.PRIMARY_INDEX);
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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 8580644..4eebb0c 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
@@ -17,11 +17,9 @@
*/
package org.apache.cassandra.io.sstable;
-import java.io.File;
import java.util.*;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.util.concurrent.Runnables;
import org.apache.cassandra.cache.InstrumentingCache;
import org.apache.cassandra.cache.KeyCacheKey;
@@ -69,6 +67,7 @@ public class SSTableRewriter extends Transactional.AbstractTransactional impleme
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 boolean keepOriginals; // true if we do not want to obsolete the originals
private SSTableWriter writer;
private Map<DecoratedKey, RowIndexEntry> cachedKeys = new HashMap<>();
@@ -95,9 +94,16 @@ public class SSTableRewriter extends Transactional.AbstractTransactional impleme
this.cfs = cfs;
this.maxAge = maxAge;
this.isOffline = isOffline;
+ this.keepOriginals = false;
this.preemptiveOpenInterval = preemptiveOpenInterval;
}
+ public SSTableRewriter keepOriginals(boolean val)
+ {
+ keepOriginals = val;
+ return this;
+ }
+
private static long calculateOpenInterval(boolean shouldOpenEarly)
{
long interval = DatabaseDescriptor.getSSTablePreempiveOpenIntervalInMB() * (1L << 20);
@@ -189,6 +195,7 @@ public class SSTableRewriter extends Transactional.AbstractTransactional impleme
{
for (SSTableWriter writer : writers)
accumulate = writer.commit(accumulate);
+
accumulate = transaction.commit(accumulate);
return accumulate;
}
@@ -280,17 +287,19 @@ public class SSTableRewriter extends Transactional.AbstractTransactional impleme
if (writer != null)
{
writer.abort();
+
+ transaction.untrackNew(writer);
writers.remove(writer);
}
writer = newWriter;
+
return;
}
- SSTableReader reader = null;
if (preemptiveOpenInterval != Long.MAX_VALUE)
{
// we leave it as a tmp file, but we open it and add it to the Tracker
- reader = writer.setMaxDataAge(maxAge).openFinalEarly();
+ SSTableReader reader = writer.setMaxDataAge(maxAge).openFinalEarly();
transaction.update(reader, false);
moveStarts(reader, reader.last);
transaction.checkpoint();
@@ -357,8 +366,7 @@ public class SSTableRewriter extends Transactional.AbstractTransactional impleme
if (throwLate)
throw new RuntimeException("exception thrown after all sstables finished, for testing");
- // TODO: do we always want to avoid obsoleting if offline?
- if (!isOffline)
+ if (!keepOriginals)
transaction.obsoleteOriginals();
transaction.prepareToCommit();
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
index db6ed42..a70b92f 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
@@ -34,7 +34,6 @@ import org.apache.cassandra.db.rows.EncodingStats;
import org.apache.cassandra.db.rows.UnfilteredSerializer;
import org.apache.cassandra.db.partitions.PartitionUpdate;
import org.apache.cassandra.dht.IPartitioner;
-import org.apache.cassandra.io.sstable.format.SSTableWriter;
import org.apache.cassandra.utils.JVMStabilityInspector;
/**
@@ -131,11 +130,13 @@ class SSTableSimpleUnsortedWriter extends AbstractSSTableSimpleWriter
try
{
diskWriter.join();
+ checkForWriterException();
}
- catch (InterruptedException e)
+ catch (Throwable e)
{
throw new RuntimeException(e);
}
+
checkForWriterException();
}
@@ -203,7 +204,7 @@ class SSTableSimpleUnsortedWriter extends AbstractSSTableSimpleWriter
if (b == SENTINEL)
return;
- try (SSTableWriter writer = createWriter())
+ try (SSTableTxnWriter writer = createWriter())
{
for (Map.Entry<DecoratedKey, PartitionUpdate> entry : b.entrySet())
writer.append(entry.getValue().unfilteredIterator());
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
index 0b06405..b22a048 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
@@ -26,7 +26,6 @@ import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.db.*;
import org.apache.cassandra.db.partitions.PartitionUpdate;
import org.apache.cassandra.dht.IPartitioner;
-import org.apache.cassandra.io.sstable.format.SSTableWriter;
/**
* A SSTable writer that assumes rows are in (partitioner) sorted order.
@@ -43,14 +42,14 @@ class SSTableSimpleWriter extends AbstractSSTableSimpleWriter
protected DecoratedKey currentKey;
protected PartitionUpdate update;
- private SSTableWriter writer;
+ private SSTableTxnWriter writer;
protected SSTableSimpleWriter(File directory, CFMetaData metadata, IPartitioner partitioner, PartitionColumns columns)
{
super(directory, metadata, partitioner, columns);
}
- private SSTableWriter getOrCreateWriter()
+ private SSTableTxnWriter getOrCreateWriter()
{
if (writer == null)
writer = createWriter();
[3/5] cassandra git commit: Introduce safer durable sstable
membership management (and simplify cleanup of compaction leftovers)
Posted by be...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/src/java/org/apache/cassandra/io/sstable/SSTableTxnWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableTxnWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableTxnWriter.java
new file mode 100644
index 0000000..8514dcc
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableTxnWriter.java
@@ -0,0 +1,101 @@
+/*
+ * 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.io.sstable;
+
+import org.apache.cassandra.db.RowIndexEntry;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+/**
+ * A wrapper for SSTableWriter and LifecycleTransaction to be used when
+ * the writer is the only participant in the transaction and therefore
+ * it can safely own the transaction.
+ */
+public class SSTableTxnWriter extends Transactional.AbstractTransactional implements Transactional
+{
+ private final LifecycleTransaction txn;
+ private final SSTableWriter writer;
+
+ public SSTableTxnWriter(LifecycleTransaction txn, SSTableWriter writer)
+ {
+ this.txn = txn;
+ this.writer = writer;
+ }
+
+ public RowIndexEntry append(UnfilteredRowIterator iterator)
+ {
+ return writer.append(iterator);
+ }
+
+ public String getFilename()
+ {
+ return writer.getFilename();
+ }
+
+ public long getFilePointer()
+ {
+ return writer.getFilePointer();
+ }
+
+ protected Throwable doCommit(Throwable accumulate)
+ {
+ return txn.commit(writer.commit(accumulate));
+ }
+
+ protected Throwable doAbort(Throwable accumulate)
+ {
+ return txn.abort(writer.abort(accumulate));
+ }
+
+ protected void doPrepare()
+ {
+ writer.prepareToCommit();
+ txn.prepareToCommit();
+ }
+
+ public SSTableReader finish(boolean openResult)
+ {
+ writer.setOpenResult(openResult);
+ finish();
+ return writer.finished();
+ }
+
+ public static SSTableTxnWriter create(Descriptor descriptor, long keyCount, long repairedAt, int sstableLevel, SerializationHeader header)
+ {
+ LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.WRITE, descriptor.directory);
+ SSTableWriter writer = SSTableWriter.create(descriptor, keyCount, repairedAt, sstableLevel, header, txn);
+ return new SSTableTxnWriter(txn, writer);
+ }
+
+ public static SSTableTxnWriter create(String filename, long keyCount, long repairedAt, int sstableLevel, SerializationHeader header)
+ {
+ Descriptor desc = Descriptor.fromFilename(filename);
+ return create(desc, keyCount, repairedAt, sstableLevel, header);
+ }
+
+ public static SSTableTxnWriter create(String filename, long keyCount, long repairedAt, SerializationHeader header)
+ {
+ return create(filename, keyCount, repairedAt, 0, header);
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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 3d9fe82..6a09d5a 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -26,7 +26,6 @@ import java.util.concurrent.atomic.AtomicLong;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Predicate;
-import com.google.common.collect.Iterables;
import com.google.common.collect.Iterators;
import com.google.common.collect.Ordering;
import com.google.common.primitives.Longs;
@@ -35,7 +34,6 @@ import com.google.common.util.concurrent.RateLimiter;
import com.clearspring.analytics.stream.cardinality.CardinalityMergeException;
import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus;
import com.clearspring.analytics.stream.cardinality.ICardinality;
-import com.codahale.metrics.Counter;
import org.apache.cassandra.cache.CachingOptions;
import org.apache.cassandra.cache.InstrumentingCache;
import org.apache.cassandra.cache.KeyCacheKey;
@@ -47,7 +45,7 @@ import org.apache.cassandra.db.filter.ColumnFilter;
import org.apache.cassandra.db.rows.*;
import org.apache.cassandra.db.commitlog.ReplayPosition;
import org.apache.cassandra.db.index.SecondaryIndex;
-import org.apache.cassandra.db.lifecycle.Tracker;
+import org.apache.cassandra.db.lifecycle.TransactionLogs;
import org.apache.cassandra.dht.*;
import org.apache.cassandra.io.FSError;
import org.apache.cassandra.io.compress.CompressionMetadata;
@@ -651,6 +649,11 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
keyCache = CacheService.instance.keyCache;
}
+ public boolean isKeyCacheSetup()
+ {
+ return keyCache != null;
+ }
+
private void load(ValidationMetadata validation) throws IOException
{
if (metadata.getBloomFilterFpChance() == 1.0)
@@ -989,7 +992,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
}
}
- // runOnClose must NOT be an anonymous or non-static inner class, nor must it retain a reference chain to this reader
+ // These runnables must NOT be an anonymous or non-static inner class, nor must it retain a reference chain to this reader
public void runOnClose(final Runnable runOnClose)
{
synchronized (tidy.global)
@@ -1166,6 +1169,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
long newSize = bytesOnDisk();
StorageMetrics.load.inc(newSize - oldSize);
parent.metric.liveDiskSpaceUsed.inc(newSize - oldSize);
+ parent.metric.totalDiskSpaceUsed.inc(newSize - oldSize);
return cloneAndReplace(first, OpenReason.METADATA_CHANGE, newSummary);
}
@@ -1646,7 +1650,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
* @return true if the this is the first time the file was marked obsolete. Calling this
* multiple times is usually buggy (see exceptions in Tracker.unmarkCompacting and removeOldSSTablesSize).
*/
- public boolean markObsolete(Tracker tracker)
+ public void markObsolete(TransactionLogs.SSTableTidier tidier)
{
if (logger.isDebugEnabled())
logger.debug("Marking {} compacted", getFilename());
@@ -1654,18 +1658,16 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
synchronized (tidy.global)
{
assert !tidy.isReplaced;
+ assert tidy.global.obsoletion == null: this + " was already marked compacted";
+
+ tidy.global.obsoletion = tidier;
+ tidy.global.stopReadMeterPersistence();
}
- if (!tidy.global.isCompacted.getAndSet(true))
- {
- tidy.type.markObsolete(this, tracker);
- return true;
- }
- return false;
}
public boolean isMarkedCompacted()
{
- return tidy.global.isCompacted.get();
+ return tidy.global.obsoletion != null;
}
public void markSuspect()
@@ -1759,6 +1761,8 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
for (Component component : components)
{
File sourceFile = new File(descriptor.filenameFor(component));
+ if (!sourceFile.exists())
+ continue;
File targetLink = new File(snapshotDirectoryPath, sourceFile.getName());
FileUtils.createHardLink(sourceFile, targetLink);
}
@@ -2065,12 +2069,12 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
* InstanceTidier => DescriptorTypeTitdy => GlobalTidy
*
* We can create many InstanceTidiers (one for every time we reopen an sstable with MOVED_START for example), but there can only be
- * two DescriptorTypeTidy (FINAL and TEMPLINK) and only one GlobalTidy for one single logical sstable.
+ * one GlobalTidy for one single logical sstable.
*
- * When the InstanceTidier cleansup, it releases its reference to its DescriptorTypeTidy; when all InstanceTidiers
- * for that type have run, the DescriptorTypeTidy cleansup. DescriptorTypeTidy behaves in the same way towards GlobalTidy.
+ * When the InstanceTidier cleansup, it releases its reference to its GlobalTidy; when all InstanceTidiers
+ * for that type have run, the GlobalTidy cleans up.
*
- * For ease, we stash a direct reference to both our type-shared and global tidier
+ * For ease, we stash a direct reference to our global tidier
*/
private static final class InstanceTidier implements Tidy
{
@@ -2084,13 +2088,9 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
private Runnable runOnClose;
private boolean isReplaced = false;
- // a reference to our shared per-Descriptor.Type tidy instance, that
+ // a reference to our shared tidy instance, that
// we will release when we are ourselves released
- private Ref<DescriptorTypeTidy> typeRef;
-
- // a convenience stashing of the shared per-descriptor-type tidy instance itself
- // and the per-logical-sstable globally shared state that it is linked to
- private DescriptorTypeTidy type;
+ private Ref<GlobalTidy> globalRef;
private GlobalTidy global;
private boolean setup;
@@ -2103,9 +2103,8 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
this.dfile = reader.dfile;
this.ifile = reader.ifile;
// get a new reference to the shared descriptor-type tidy
- this.typeRef = DescriptorTypeTidy.get(reader);
- this.type = typeRef.get();
- this.global = type.globalRef.get();
+ this.globalRef = GlobalTidy.get(reader);
+ this.global = globalRef.get();
if (!isOffline)
global.ensureReadMeter();
}
@@ -2148,7 +2147,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
dfile.close();
if (ifile != null)
ifile.close();
- typeRef.release();
+ globalRef.release();
}
});
}
@@ -2167,102 +2166,16 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
}
/**
- * One shared between all instances of a given Descriptor.Type.
- * Performs only two things: the deletion of the sstables for the type,
- * if necessary; and the shared reference to the globally shared state.
+ * One instance per logical sstable. This both tracks shared cleanup and some shared state related
+ * to the sstable's lifecycle.
*
* All InstanceTidiers, on setup(), ask the static get() method for their shared state,
* and stash a reference to it to be released when they are. Once all such references are
- * released, the shared tidy will be performed.
- */
- static final class DescriptorTypeTidy implements Tidy
- {
- // keyed by REAL descriptor (TMPLINK/FINAL), mapping to the shared DescriptorTypeTidy for that descriptor
- static final ConcurrentMap<Descriptor, Ref<DescriptorTypeTidy>> lookup = new ConcurrentHashMap<>();
-
- private final Descriptor desc;
- private final Ref<GlobalTidy> globalRef;
- private final Set<Component> components;
- private long sizeOnDelete;
- private Counter totalDiskSpaceUsed;
-
- DescriptorTypeTidy(Descriptor desc, SSTableReader sstable)
- {
- this.desc = desc;
- // get a new reference to the shared global tidy
- this.globalRef = GlobalTidy.get(sstable);
- this.components = sstable.components;
- }
-
- void markObsolete(SSTableReader instance, 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 (tracker != null && tracker.cfstore != null && desc.type == Descriptor.Type.FINAL)
- {
- sizeOnDelete = instance.bytesOnDisk();
- totalDiskSpaceUsed = tracker.cfstore.metric.totalDiskSpaceUsed;
- tracker.notifyDeleting(instance);
- }
- }
-
- public void tidy()
- {
- lookup.remove(desc);
- boolean isCompacted = globalRef.get().isCompacted.get();
- globalRef.release();
- switch (desc.type)
- {
- case FINAL:
- if (isCompacted)
- new SSTableDeletingTask(desc, components, totalDiskSpaceUsed, sizeOnDelete).run();
- break;
- case TEMPLINK:
- new SSTableDeletingTask(desc, components, null, 0).run();
- break;
- default:
- throw new IllegalStateException();
- }
- }
-
- public String name()
- {
- return desc.toString();
- }
-
- // get a new reference to the shared DescriptorTypeTidy for this sstable
- @SuppressWarnings("resource")
- public static Ref<DescriptorTypeTidy> get(SSTableReader sstable)
- {
- Descriptor desc = sstable.descriptor;
- if (sstable.openReason == OpenReason.EARLY)
- desc = desc.asType(Descriptor.Type.TEMPLINK);
- Ref<DescriptorTypeTidy> refc = lookup.get(desc);
- if (refc != null)
- return refc.ref();
- final DescriptorTypeTidy tidy = new DescriptorTypeTidy(desc, sstable);
- refc = new Ref<>(tidy, tidy);
- Ref<?> ex = lookup.putIfAbsent(desc, refc);
- if (ex != null)
- {
- refc.close();
- throw new AssertionError();
- }
- return refc;
- }
- }
-
- /**
- * One instance per logical sstable. This both tracks shared cleanup and some shared state related
- * to the sstable's lifecycle. All DescriptorTypeTidy instances, on construction, obtain a reference to us
- * via our static get(). There should only ever be at most two such references extant at any one time,
- * since only TMPLINK and FINAL type descriptors should be open as readers. When all files of both
- * kinds have been released, this shared tidy will be performed.
+ * released, this shared tidy will be performed.
*/
static final class GlobalTidy implements Tidy
{
- // keyed by FINAL descriptor, mapping to the shared GlobalTidy for that descriptor
+ // keyed by descriptor, mapping to the shared GlobalTidy for that descriptor
static final ConcurrentMap<Descriptor, Ref<GlobalTidy>> lookup = new ConcurrentHashMap<>();
private final Descriptor desc;
@@ -2272,14 +2185,12 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
// the scheduled persistence of the readMeter, that we will cancel once all instances of this logical
// sstable have been released
private ScheduledFuture readMeterSyncFuture;
- // shared state managing if the logical sstable has been compacted; this is used in cleanup both here
- // and in the FINAL type tidier
- private final AtomicBoolean isCompacted;
+ // shared state managing if the logical sstable has been compacted; this is used in cleanup
+ private volatile TransactionLogs.SSTableTidier obsoletion;
GlobalTidy(final SSTableReader reader)
{
this.desc = reader.descriptor;
- this.isCompacted = new AtomicBoolean();
}
void ensureReadMeter()
@@ -2302,7 +2213,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
{
public void run()
{
- if (!isCompacted.get())
+ if (obsoletion == null)
{
meterSyncThrottle.acquire();
SystemKeyspace.persistSSTableReadMeter(desc.ksname, desc.cfname, desc.generation, readMeter);
@@ -2311,13 +2222,22 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
}, 1, 5, TimeUnit.MINUTES);
}
- public void tidy()
+ private void stopReadMeterPersistence()
{
- lookup.remove(desc);
if (readMeterSyncFuture != null)
+ {
readMeterSyncFuture.cancel(true);
- if (isCompacted.get())
- SystemKeyspace.clearSSTableReadMeter(desc.ksname, desc.cfname, desc.generation);
+ readMeterSyncFuture = null;
+ }
+ }
+
+ public void tidy()
+ {
+ lookup.remove(desc);
+
+ if (obsoletion != null)
+ obsoletion.run();
+
// don't ideally want to dropPageCache for the file until all instances have been released
CLibrary.trySkipCache(desc.filenameFor(Component.DATA), 0, 0);
CLibrary.trySkipCache(desc.filenameFor(Component.PRIMARY_INDEX), 0, 0);
@@ -2352,7 +2272,6 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
public static void resetTidying()
{
GlobalTidy.lookup.clear();
- DescriptorTypeTidy.lookup.clear();
}
public static abstract class Factory
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
index 900c948..08a9dcc 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
@@ -23,6 +23,7 @@ import java.util.HashSet;
import java.util.Map;
import java.util.Set;
+import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Sets;
import org.apache.cassandra.config.CFMetaData;
@@ -30,6 +31,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.config.Schema;
import org.apache.cassandra.db.RowIndexEntry;
import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.db.rows.UnfilteredRowIterator;
import org.apache.cassandra.dht.IPartitioner;
import org.apache.cassandra.io.sstable.Component;
@@ -68,7 +70,13 @@ public abstract class SSTableWriter extends SSTable implements Transactional
protected boolean openResult;
}
- protected SSTableWriter(Descriptor descriptor, long keyCount, long repairedAt, CFMetaData metadata, IPartitioner partitioner, MetadataCollector metadataCollector, SerializationHeader header)
+ protected SSTableWriter(Descriptor descriptor,
+ long keyCount,
+ long repairedAt,
+ CFMetaData metadata,
+ IPartitioner partitioner,
+ MetadataCollector metadataCollector,
+ SerializationHeader header)
{
super(descriptor, components(metadata), metadata, partitioner);
this.keyCount = keyCount;
@@ -84,21 +92,17 @@ public abstract class SSTableWriter extends SSTable implements Transactional
CFMetaData metadata,
IPartitioner partitioner,
MetadataCollector metadataCollector,
- SerializationHeader header)
+ SerializationHeader header,
+ LifecycleTransaction txn)
{
Factory writerFactory = descriptor.getFormat().getWriterFactory();
- return writerFactory.open(descriptor, keyCount, repairedAt, metadata, partitioner, metadataCollector, header);
+ return writerFactory.open(descriptor, keyCount, repairedAt, metadata, partitioner, metadataCollector, header, txn);
}
- public static SSTableWriter create(Descriptor descriptor, long keyCount, long repairedAt, SerializationHeader header)
- {
- return create(descriptor, keyCount, repairedAt, 0, header);
- }
-
- public static SSTableWriter create(Descriptor descriptor, long keyCount, long repairedAt, int sstableLevel, SerializationHeader header)
+ public static SSTableWriter create(Descriptor descriptor, long keyCount, long repairedAt, int sstableLevel, SerializationHeader header, LifecycleTransaction txn)
{
CFMetaData metadata = Schema.instance.getCFMetaData(descriptor);
- return create(metadata, descriptor, keyCount, repairedAt, sstableLevel, DatabaseDescriptor.getPartitioner(), header);
+ return create(metadata, descriptor, keyCount, repairedAt, sstableLevel, DatabaseDescriptor.getPartitioner(), header, txn);
}
public static SSTableWriter create(CFMetaData metadata,
@@ -107,20 +111,22 @@ public abstract class SSTableWriter extends SSTable implements Transactional
long repairedAt,
int sstableLevel,
IPartitioner partitioner,
- SerializationHeader header)
+ SerializationHeader header,
+ LifecycleTransaction txn)
{
MetadataCollector collector = new MetadataCollector(metadata.comparator).sstableLevel(sstableLevel);
- return create(descriptor, keyCount, repairedAt, metadata, partitioner, collector, header);
+ return create(descriptor, keyCount, repairedAt, metadata, partitioner, collector, header, txn);
}
- public static SSTableWriter create(String filename, long keyCount, long repairedAt, int sstableLevel, SerializationHeader header)
+ public static SSTableWriter create(String filename, long keyCount, long repairedAt, int sstableLevel, SerializationHeader header,LifecycleTransaction txn)
{
- return create(Descriptor.fromFilename(filename), keyCount, repairedAt, sstableLevel, header);
+ return create(Descriptor.fromFilename(filename), keyCount, repairedAt, sstableLevel, header, txn);
}
- public static SSTableWriter create(String filename, long keyCount, long repairedAt, SerializationHeader header)
+ @VisibleForTesting
+ public static SSTableWriter create(String filename, long keyCount, long repairedAt, SerializationHeader header, LifecycleTransaction txn)
{
- return create(Descriptor.fromFilename(filename), keyCount, repairedAt, 0, header);
+ return create(Descriptor.fromFilename(filename), keyCount, repairedAt, 0, header, txn);
}
private static Set<Component> components(CFMetaData metadata)
@@ -207,7 +213,7 @@ public abstract class SSTableWriter extends SSTable implements Transactional
public SSTableReader finish(boolean openResult)
{
- txnProxy.openResult = openResult;
+ setOpenResult(openResult);
txnProxy.finish();
return finished();
}
@@ -260,13 +266,6 @@ public abstract class SSTableWriter extends SSTable implements Transactional
return (StatsMetadata) finalizeMetadata().get(MetadataType.STATS);
}
- public static Descriptor rename(Descriptor tmpdesc, Set<Component> components)
- {
- Descriptor newdesc = tmpdesc.asType(Descriptor.Type.FINAL);
- rename(tmpdesc, newdesc, components);
- return newdesc;
- }
-
public static void rename(Descriptor tmpdesc, Descriptor newdesc, Set<Component> components)
{
for (Component component : Sets.difference(components, Sets.newHashSet(Component.DATA, Component.SUMMARY)))
@@ -290,6 +289,7 @@ public abstract class SSTableWriter extends SSTable implements Transactional
CFMetaData metadata,
IPartitioner partitioner,
MetadataCollector metadataCollector,
- SerializationHeader header);
+ SerializationHeader header,
+ LifecycleTransaction txn);
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
index a5419f1..a072d4d 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
@@ -22,6 +22,7 @@ import java.util.Set;
import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.db.RowIndexEntry;
import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.dht.IPartitioner;
import org.apache.cassandra.io.sstable.Component;
import org.apache.cassandra.io.sstable.Descriptor;
@@ -81,9 +82,16 @@ public class BigFormat implements SSTableFormat
static class WriterFactory extends SSTableWriter.Factory
{
@Override
- public SSTableWriter open(Descriptor descriptor, long keyCount, long repairedAt, CFMetaData metadata, IPartitioner partitioner, MetadataCollector metadataCollector, SerializationHeader header)
+ public SSTableWriter open(Descriptor descriptor,
+ long keyCount,
+ long repairedAt,
+ CFMetaData metadata,
+ IPartitioner partitioner,
+ MetadataCollector metadataCollector,
+ SerializationHeader header,
+ LifecycleTransaction txn)
{
- return new BigTableWriter(descriptor, keyCount, repairedAt, metadata, partitioner, metadataCollector, header);
+ return new BigTableWriter(descriptor, keyCount, repairedAt, metadata, partitioner, metadataCollector, header, txn);
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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 6759293..13c9954 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
@@ -22,6 +22,7 @@ import java.util.Map;
import java.util.Set;
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;
@@ -45,7 +46,6 @@ import org.apache.cassandra.utils.FilterFactory;
import org.apache.cassandra.utils.IFilter;
import org.apache.cassandra.utils.concurrent.Transactional;
-import static org.apache.cassandra.utils.Throwables.merge;
import org.apache.cassandra.utils.SyncUtil;
public class BigTableWriter extends SSTableWriter
@@ -58,9 +58,17 @@ public class BigTableWriter extends SSTableWriter
private DecoratedKey lastWrittenKey;
private FileMark dataMark;
- public BigTableWriter(Descriptor descriptor, Long keyCount, Long repairedAt, CFMetaData metadata, IPartitioner partitioner, MetadataCollector metadataCollector, SerializationHeader header)
+ public BigTableWriter(Descriptor descriptor,
+ Long keyCount,
+ Long repairedAt,
+ CFMetaData metadata,
+ IPartitioner partitioner,
+ MetadataCollector metadataCollector,
+ SerializationHeader header,
+ LifecycleTransaction txn)
{
super(descriptor, keyCount, repairedAt, metadata, partitioner, metadataCollector, header);
+ txn.trackNew(this); // must track before any files are created
if (compression)
{
@@ -76,6 +84,10 @@ public class BigTableWriter extends SSTableWriter
dbuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getDiskAccessMode(), false);
}
iwriter = new IndexWriter(keyCount, dataFile);
+
+ // txnLogs will delete if safe to do so (early readers)
+ iwriter.indexFile.deleteFile(false);
+ dataFile.deleteFile(false);
}
public void mark()
@@ -220,18 +232,6 @@ public class BigTableWriter extends SSTableWriter
}
}
- private Descriptor makeTmpLinks()
- {
- // create temp links if they don't already exist
- Descriptor link = descriptor.asType(Descriptor.Type.TEMPLINK);
- if (!new File(link.filenameFor(Component.PRIMARY_INDEX)).exists())
- {
- FileUtils.createHardLink(new File(descriptor.filenameFor(Component.PRIMARY_INDEX)), new File(link.filenameFor(Component.PRIMARY_INDEX)));
- FileUtils.createHardLink(new File(descriptor.filenameFor(Component.DATA)), new File(link.filenameFor(Component.DATA)));
- }
- return link;
- }
-
@SuppressWarnings("resource")
public SSTableReader openEarly()
{
@@ -242,11 +242,10 @@ public class BigTableWriter extends SSTableWriter
StatsMetadata stats = statsMetadata();
assert boundary.indexLength > 0 && boundary.dataLength > 0;
- Descriptor link = makeTmpLinks();
- // open the reader early, giving it a FINAL descriptor type so that it is indistinguishable for other consumers
- SegmentedFile ifile = iwriter.builder.complete(link.filenameFor(Component.PRIMARY_INDEX), boundary.indexLength);
- SegmentedFile dfile = dbuilder.complete(link.filenameFor(Component.DATA), boundary.dataLength);
- SSTableReader sstable = SSTableReader.internalOpen(descriptor.asType(Descriptor.Type.FINAL),
+ // open the reader early
+ SegmentedFile ifile = iwriter.builder.complete(descriptor.filenameFor(Component.PRIMARY_INDEX), boundary.indexLength);
+ SegmentedFile dfile = dbuilder.complete(descriptor.filenameFor(Component.DATA), boundary.dataLength);
+ SSTableReader sstable = SSTableReader.internalOpen(descriptor,
components, metadata,
partitioner, ifile,
dfile, iwriter.summary.build(partitioner, boundary),
@@ -263,7 +262,8 @@ public class BigTableWriter extends SSTableWriter
// we must ensure the data is completely flushed to disk
dataFile.sync();
iwriter.indexFile.sync();
- return openFinal(makeTmpLinks(), SSTableReader.OpenReason.EARLY);
+
+ return openFinal(descriptor, SSTableReader.OpenReason.EARLY);
}
@SuppressWarnings("resource")
@@ -276,7 +276,7 @@ public class BigTableWriter extends SSTableWriter
// finalize in-memory state for the reader
SegmentedFile ifile = iwriter.builder.complete(desc.filenameFor(Component.PRIMARY_INDEX));
SegmentedFile dfile = dbuilder.complete(desc.filenameFor(Component.DATA));
- SSTableReader sstable = SSTableReader.internalOpen(desc.asType(Descriptor.Type.FINAL),
+ SSTableReader sstable = SSTableReader.internalOpen(desc,
components,
this.metadata,
partitioner,
@@ -314,11 +314,8 @@ public class BigTableWriter extends SSTableWriter
// save the table of components
SSTable.appendTOC(descriptor, components);
- // rename to final
- rename(descriptor, components);
-
if (openResult)
- finalReader = openFinal(descriptor.asType(Descriptor.Type.FINAL), SSTableReader.OpenReason.NORMAL);
+ finalReader = openFinal(descriptor, SSTableReader.OpenReason.NORMAL);
}
protected Throwable doCommit(Throwable accumulate)
@@ -339,26 +336,6 @@ public class BigTableWriter extends SSTableWriter
{
accumulate = iwriter.abort(accumulate);
accumulate = dataFile.abort(accumulate);
-
- accumulate = delete(descriptor, accumulate);
- if (!openResult)
- accumulate = delete(descriptor.asType(Descriptor.Type.FINAL), accumulate);
- return accumulate;
- }
-
- private Throwable delete(Descriptor desc, Throwable accumulate)
- {
- try
- {
- Set<Component> components = SSTable.discoverComponentsFor(desc);
- if (!components.isEmpty())
- SSTable.delete(desc, components);
- }
- catch (Throwable t)
- {
- logger.error(String.format("Failed deleting temp components for %s", descriptor), t);
- accumulate = merge(accumulate, t);
- }
return accumulate;
}
}
@@ -366,7 +343,7 @@ public class BigTableWriter extends SSTableWriter
private static void writeMetadata(Descriptor desc, Map<MetadataType, MetadataComponent> components)
{
File file = new File(desc.filenameFor(Component.STATS));
- try (SequentialWriter out = SequentialWriter.open(file);)
+ try (SequentialWriter out = SequentialWriter.open(file))
{
desc.getMetadataSerializer().serialize(components, out.stream);
out.setDescriptor(desc).finish();
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java
index fcdf57a..9a5eae8 100644
--- a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java
@@ -149,9 +149,8 @@ public class MetadataSerializer implements IMetadataSerializer
private void rewriteSSTableMetadata(Descriptor descriptor, Map<MetadataType, MetadataComponent> currentComponents) throws IOException
{
- Descriptor tmpDescriptor = descriptor.asType(Descriptor.Type.TEMP);
-
- try (DataOutputStreamPlus out = new BufferedDataOutputStreamPlus(new FileOutputStream(tmpDescriptor.filenameFor(Component.STATS))))
+ String filePath = descriptor.tmpFilenameFor(Component.STATS);
+ try (DataOutputStreamPlus out = new BufferedDataOutputStreamPlus(new FileOutputStream(filePath)))
{
serialize(currentComponents, out);
out.flush();
@@ -159,7 +158,7 @@ public class MetadataSerializer implements IMetadataSerializer
// we cant move a file on top of another file in windows:
if (FBUtilities.isWindows())
FileUtils.delete(descriptor.filenameFor(Component.STATS));
- FileUtils.renameWithConfirm(tmpDescriptor.filenameFor(Component.STATS), descriptor.filenameFor(Component.STATS));
+ FileUtils.renameWithConfirm(filePath, descriptor.filenameFor(Component.STATS));
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/src/java/org/apache/cassandra/io/util/FileUtils.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/FileUtils.java b/src/java/org/apache/cassandra/io/util/FileUtils.java
index 480ecea..f415f2b 100644
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@ -20,9 +20,11 @@ package org.apache.cassandra.io.util;
import java.io.*;
import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
+import java.nio.charset.Charset;
import java.nio.file.*;
import java.text.DecimalFormat;
import java.util.Arrays;
+import java.util.List;
import org.apache.cassandra.config.Config;
import sun.nio.ch.DirectBuffer;
@@ -306,6 +308,29 @@ public class FileUtils
}
}
+ /** Return true if file is contained in folder */
+ public static boolean isContained(File folder, File file)
+ {
+ String folderPath = getCanonicalPath(folder);
+ String filePath = getCanonicalPath(file);
+
+ return filePath.startsWith(folderPath);
+ }
+
+ /** Convert absolute path into a path relative to the base path */
+ public static String getRelativePath(String basePath, String absolutePath)
+ {
+ try
+ {
+ return Paths.get(basePath).relativize(Paths.get(absolutePath)).toString();
+ }
+ catch(Exception ex)
+ {
+ String absDataPath = FileUtils.getCanonicalPath(basePath);
+ return Paths.get(absDataPath).relativize(Paths.get(absolutePath)).toString();
+ }
+ }
+
public static boolean isCleanerAvailable()
{
return canCleanDirectBuffers;
@@ -555,4 +580,44 @@ public class FileUtils
}
return false;
}
+
+ public static void append(File file, String ... lines)
+ {
+ if (file.exists())
+ write(file, StandardOpenOption.APPEND, lines);
+ else
+ write(file, StandardOpenOption.CREATE, lines);
+ }
+
+ public static void replace(File file, String ... lines)
+ {
+ write(file, StandardOpenOption.TRUNCATE_EXISTING, lines);
+ }
+
+ public static void write(File file, StandardOpenOption op, String ... lines)
+ {
+ try
+ {
+ Files.write(file.toPath(),
+ Arrays.asList(lines),
+ Charset.forName("utf-8"),
+ op);
+ }
+ catch (IOException ex)
+ {
+ throw new RuntimeException(ex);
+ }
+ }
+
+ public static List<String> readLines(File file)
+ {
+ try
+ {
+ return Files.readAllLines(file.toPath(), Charset.forName("utf-8"));
+ }
+ catch (IOException ex)
+ {
+ throw new RuntimeException(ex);
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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 ee6e5b4..90340ca 100644
--- a/src/java/org/apache/cassandra/io/util/SequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/util/SequentialWriter.java
@@ -79,6 +79,8 @@ public class SequentialWriter extends OutputStream implements WritableByteChanne
// due to lack of multiple-inheritance, we proxy our transactional implementation
protected class TransactionalProxy extends AbstractTransactional
{
+ private boolean deleteFile = true;
+
@Override
protected Throwable doPreCleanup(Throwable accumulate)
{
@@ -118,7 +120,10 @@ public class SequentialWriter extends OutputStream implements WritableByteChanne
protected Throwable doAbort(Throwable accumulate)
{
- return FileUtils.deleteWithConfirm(filePath, false, accumulate);
+ if (deleteFile)
+ return FileUtils.deleteWithConfirm(filePath, false, accumulate);
+ else
+ return accumulate;
}
}
@@ -487,6 +492,11 @@ public class SequentialWriter extends OutputStream implements WritableByteChanne
return new TransactionalProxy();
}
+ public void deleteFile(boolean val)
+ {
+ txnProxy.deleteFile = val;
+ }
+
public void releaseFileHandle()
{
try
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/src/java/org/apache/cassandra/service/CassandraDaemon.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java
index 8dadb91..548cbc7 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -169,7 +169,10 @@ public class CassandraDaemon
try
{
- SystemKeyspace.snapshotOnVersionChange();
+ if (SystemKeyspace.snapshotOnVersionChange())
+ {
+ SystemKeyspace.migrateDataDirs();
+ }
}
catch (IOException e)
{
@@ -218,17 +221,6 @@ public class CassandraDaemon
// load schema from disk
Schema.instance.loadFromDisk();
- // clean up compaction leftovers
- Map<Pair<String, String>, Map<Integer, UUID>> unfinishedCompactions = SystemKeyspace.getUnfinishedCompactions();
- for (Pair<String, String> kscf : unfinishedCompactions.keySet())
- {
- CFMetaData cfm = Schema.instance.getCFMetaData(kscf.left, kscf.right);
- // CFMetaData can be null if CF is already dropped
- if (cfm != null)
- ColumnFamilyStore.removeUnfinishedCompactionLeftovers(cfm, unfinishedCompactions.get(kscf));
- }
- SystemKeyspace.discardCompactionsInProgress();
-
// clean up debris in the rest of the keyspaces
for (String keyspaceName : Schema.instance.getKeyspaces())
{
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/src/java/org/apache/cassandra/service/GCInspector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/GCInspector.java b/src/java/org/apache/cassandra/service/GCInspector.java
index 4e03a49..fc7ff3d 100644
--- a/src/java/org/apache/cassandra/service/GCInspector.java
+++ b/src/java/org/apache/cassandra/service/GCInspector.java
@@ -39,7 +39,7 @@ import org.slf4j.LoggerFactory;
import com.sun.management.GarbageCollectionNotificationInfo;
import com.sun.management.GcInfo;
-import org.apache.cassandra.io.sstable.SSTableDeletingTask;
+import org.apache.cassandra.db.lifecycle.TransactionLogs;
import org.apache.cassandra.utils.StatusLogger;
public class GCInspector implements NotificationListener, GCInspectorMXBean
@@ -193,10 +193,10 @@ public class GCInspector implements NotificationListener, GCInspectorMXBean
}
/*
- * Assume that a GC type is an old generation collection so SSTableDeletingTask.rescheduleFailedTasks()
+ * Assume that a GC type is an old generation collection so TransactionLogs.rescheduleFailedTasks()
* should be invoked.
*
- * Defaults to not invoking SSTableDeletingTask.rescheduleFailedTasks() on unrecognized GC names
+ * Defaults to not invoking TransactionLogs.rescheduleFailedTasks() on unrecognized GC names
*/
private static boolean assumeGCIsOldGen(GarbageCollectorMXBean gc)
{
@@ -214,7 +214,7 @@ public class GCInspector implements NotificationListener, GCInspectorMXBean
return true;
default:
//Assume not old gen otherwise, don't call
- //SSTableDeletingTask.rescheduleFailedTasks()
+ //TransactionLogs.rescheduleFailedTasks()
return false;
}
}
@@ -284,7 +284,7 @@ public class GCInspector implements NotificationListener, GCInspectorMXBean
// if we just finished an old gen collection and we're still using a lot of memory, try to reduce the pressure
if (gcState.assumeGCIsOldGen)
- SSTableDeletingTask.rescheduleFailedTasks();
+ TransactionLogs.rescheduleFailedDeletions();
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/src/java/org/apache/cassandra/service/StartupChecks.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StartupChecks.java b/src/java/org/apache/cassandra/service/StartupChecks.java
index 700fd1a..fee9789 100644
--- a/src/java/org/apache/cassandra/service/StartupChecks.java
+++ b/src/java/org/apache/cassandra/service/StartupChecks.java
@@ -218,7 +218,7 @@ public class StartupChecks
{
public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException
{
- if (!file.toString().endsWith(".db"))
+ if (!Descriptor.isValidFile(file.getFileName().toString()))
return FileVisitResult.CONTINUE;
try
@@ -236,7 +236,9 @@ public class StartupChecks
public FileVisitResult preVisitDirectory(Path dir, BasicFileAttributes attrs) throws IOException
{
String name = dir.getFileName().toString();
- return (name.equals("snapshots") || name.equals("backups"))
+ return (name.equals(Directories.SNAPSHOT_SUBDIR)
+ || name.equals(Directories.BACKUPS_SUBDIR)
+ || name.equals(Directories.TRANSACTIONS_SUBDIR))
? FileVisitResult.SKIP_SUBTREE
: FileVisitResult.CONTINUE;
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 2d434cb..da53bf7 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -70,6 +70,7 @@ import org.apache.cassandra.config.*;
import org.apache.cassandra.db.*;
import org.apache.cassandra.db.commitlog.CommitLog;
import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.lifecycle.TransactionLogs;
import org.apache.cassandra.dht.BootStrapper;
import org.apache.cassandra.dht.IPartitioner;
import org.apache.cassandra.dht.Range;
@@ -93,7 +94,6 @@ import org.apache.cassandra.gms.IEndpointStateChangeSubscriber;
import org.apache.cassandra.gms.IFailureDetector;
import org.apache.cassandra.gms.TokenSerializer;
import org.apache.cassandra.gms.VersionedValue;
-import org.apache.cassandra.io.sstable.SSTableDeletingTask;
import org.apache.cassandra.io.sstable.SSTableLoader;
import org.apache.cassandra.io.util.FileUtils;
import org.apache.cassandra.locator.AbstractReplicationStrategy;
@@ -4216,7 +4216,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
public void rescheduleFailedDeletions()
{
- SSTableDeletingTask.rescheduleFailedTasks();
+ TransactionLogs.rescheduleFailedDeletions();
}
/**
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/src/java/org/apache/cassandra/streaming/StreamLockfile.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamLockfile.java b/src/java/org/apache/cassandra/streaming/StreamLockfile.java
deleted file mode 100644
index 83ae5f0..0000000
--- a/src/java/org/apache/cassandra/streaming/StreamLockfile.java
+++ /dev/null
@@ -1,128 +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.streaming;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.StandardOpenOption;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.UUID;
-
-import com.google.common.base.Charsets;
-import org.apache.cassandra.io.sstable.format.SSTableWriter;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.SSTable;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.utils.JVMStabilityInspector;
-
-/**
- * Encapsulates the behavior for 'locking' any streamed sttables to a node.
- * If a process crashes while converting a set of SSTableWriters to SSTReaders
- * (meaning, some subset of SSTWs were converted, but not the entire set), we want
- * to disregard the entire set as we will surely have missing data (by definition).
- *
- * Basic behavior id to write out the names of all SSTWs to a file, one SSTW per line,
- * and then delete the file when complete (normal behavior). This should happen before
- * converting any SSTWs. Thus, the lockfile is created, some SSTWs are converted,
- * and if the process crashes, on restart, we look for any existing lockfile, and delete
- * any referenced SSTRs.
- */
-public class StreamLockfile
-{
- public static final String FILE_EXT = ".lockfile";
- private static final Logger logger = LoggerFactory.getLogger(StreamLockfile.class);
-
- private final File lockfile;
-
- public StreamLockfile(File directory, UUID uuid)
- {
- lockfile = new File(directory, uuid + FILE_EXT);
- }
-
- public StreamLockfile(File lockfile)
- {
- assert lockfile != null;
- this.lockfile = lockfile;
- }
-
- public void create(Collection<SSTableWriter> sstables)
- {
- List<String> sstablePaths = new ArrayList<>(sstables.size());
- for (SSTableWriter writer : sstables)
- {
- /* write out the file names *without* the 'tmp-file' flag in the file name.
- this class will not need to clean up tmp files (on restart), CassandraDaemon does that already,
- just make sure we delete the fully-formed SSTRs. */
- sstablePaths.add(writer.descriptor.asType(Descriptor.Type.FINAL).baseFilename());
- }
-
- try
- {
- Files.write(lockfile.toPath(), sstablePaths, Charsets.UTF_8,
- StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE, StandardOpenOption.DSYNC);
- }
- catch (IOException e)
- {
- logger.warn(String.format("Could not create lockfile %s for stream session, nothing to worry too much about", lockfile), e);
- }
- }
-
- public void delete()
- {
- FileUtils.delete(lockfile);
- }
-
- public void cleanup()
- {
- List<String> files = readLockfile(lockfile);
- for (String file : files)
- {
- try
- {
- Descriptor desc = Descriptor.fromFilename(file, true);
- SSTable.delete(desc, SSTable.componentsFor(desc));
- }
- catch (Exception e)
- {
- JVMStabilityInspector.inspectThrowable(e);
- logger.warn("failed to delete a potentially stale sstable {}", file);
- }
- }
- }
-
- private List<String> readLockfile(File lockfile)
- {
- try
- {
- return Files.readAllLines(lockfile.toPath(), Charsets.UTF_8);
- }
- catch (IOException e)
- {
- logger.info("couldn't read lockfile {}, ignoring", lockfile.getAbsolutePath());
- return Collections.emptyList();
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/src/java/org/apache/cassandra/streaming/StreamReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamReader.java b/src/java/org/apache/cassandra/streaming/StreamReader.java
index 132a095..61656b3 100644
--- a/src/java/org/apache/cassandra/streaming/StreamReader.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReader.java
@@ -129,9 +129,9 @@ public class StreamReader
Directories.DataDirectory localDir = cfs.directories.getWriteableLocation(totalSize);
if (localDir == null)
throw new IOException("Insufficient disk space to store " + totalSize + " bytes");
- desc = Descriptor.fromFilename(cfs.getTempSSTablePath(cfs.directories.getLocationForDisk(localDir), format));
+ desc = Descriptor.fromFilename(cfs.getSSTablePath(cfs.directories.getLocationForDisk(localDir), format));
- return SSTableWriter.create(desc, estimatedKeys, repairedAt, sstableLevel, header.toHeader(cfs.metadata));
+ return SSTableWriter.create(desc, estimatedKeys, repairedAt, sstableLevel, header.toHeader(cfs.metadata), session.getTransaction(cfId));
}
protected void drain(InputStream dis, long bytesRead) throws IOException
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
index d4d49b3..b7db989 100644
--- a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
@@ -17,9 +17,6 @@
*/
package org.apache.cassandra.streaming;
-import java.io.File;
-import java.io.IOError;
-import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
@@ -31,6 +28,8 @@ import org.apache.cassandra.concurrent.NamedThreadFactory;
import org.apache.cassandra.config.Schema;
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.db.compaction.OperationType;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.sstable.format.SSTableWriter;
import org.apache.cassandra.utils.Pair;
@@ -49,6 +48,9 @@ public class StreamReceiveTask extends StreamTask
// total size of files to receive
private final long totalSize;
+ // Transaction tracking new files received
+ public final LifecycleTransaction txn;
+
// true if task is done (either completed or aborted)
private boolean done = false;
@@ -60,6 +62,9 @@ public class StreamReceiveTask extends StreamTask
super(session, cfId);
this.totalFiles = totalFiles;
this.totalSize = totalSize;
+ // this is an "offline" transaction, as we currently manually expose the sstables once done;
+ // this should be revisited at a later date, so that LifecycleTransaction manages all sstable state changes
+ this.txn = LifecycleTransaction.offline(OperationType.STREAM, Schema.instance.getCFMetaData(cfId));
this.sstables = new ArrayList<>(totalFiles);
}
@@ -111,19 +116,15 @@ public class StreamReceiveTask extends StreamTask
for (SSTableWriter writer : task.sstables)
writer.abort();
task.sstables.clear();
+ task.txn.abort();
return;
}
ColumnFamilyStore cfs = Keyspace.open(kscf.left).getColumnFamilyStore(kscf.right);
- File lockfiledir = cfs.directories.getWriteableLocationAsFile(task.sstables.size() * 256L);
- if (lockfiledir == null)
- throw new IOError(new IOException("All disks full"));
- StreamLockfile lockfile = new StreamLockfile(lockfiledir, UUID.randomUUID());
- lockfile.create(task.sstables);
List<SSTableReader> readers = new ArrayList<>();
for (SSTableWriter writer : task.sstables)
readers.add(writer.finish(true));
- lockfile.delete();
+ task.txn.finish();
task.sstables.clear();
try (Refs<SSTableReader> refs = Refs.ref(readers))
@@ -151,6 +152,7 @@ public class StreamReceiveTask extends StreamTask
done = true;
for (SSTableWriter writer : sstables)
writer.abort();
+ txn.abort();
sstables.clear();
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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 c9ebdad..744a03a 100644
--- a/src/java/org/apache/cassandra/streaming/StreamSession.java
+++ b/src/java/org/apache/cassandra/streaming/StreamSession.java
@@ -27,6 +27,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
import com.google.common.base.Function;
import com.google.common.collect.*;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.db.lifecycle.View;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.slf4j.Logger;
@@ -208,6 +209,12 @@ public class StreamSession implements IEndpointStateChangeSubscriber
}
+ public LifecycleTransaction getTransaction(UUID cfId)
+ {
+ assert receivers.containsKey(cfId);
+ return receivers.get(cfId).txn;
+ }
+
/**
* Bind this session to report to specific {@link StreamResultFuture} and
* perform pre-streaming initialization.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/src/java/org/apache/cassandra/tools/StandaloneLister.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/StandaloneLister.java b/src/java/org/apache/cassandra/tools/StandaloneLister.java
new file mode 100644
index 0000000..71575c7
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/StandaloneLister.java
@@ -0,0 +1,214 @@
+/**
+ * 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.tools;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.db.Directories;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.commons.cli.*;
+
+import java.io.File;
+
+import static org.apache.cassandra.tools.BulkLoader.CmdLineOptions;
+
+public class StandaloneLister
+{
+ private static final String TOOL_NAME = "sstablelister";
+ private static final String TYPE_OPTION = "type";
+ private static final String OP_LOG_OPTION = "oplog";
+ private static final String VERBOSE_OPTION = "verbose";
+ private static final String DEBUG_OPTION = "debug";
+ private static final String HELP_OPTION = "help";
+
+ public static void main(String args[])
+ {
+ Options options = Options.parseArgs(args);
+ try
+ {
+ // load keyspace descriptions.
+ Schema.instance.loadFromDisk(false);
+
+ CFMetaData metadata = Schema.instance.getCFMetaData(options.keyspaceName, options.cfName);
+ if (metadata == null)
+ throw new IllegalArgumentException(String.format("Unknown keyspace/table %s.%s",
+ options.keyspaceName,
+ options.cfName));
+
+ OutputHandler handler = new OutputHandler.SystemOutput(options.verbose, options.debug);
+
+ Directories directories = new Directories(metadata);
+ Directories.SSTableLister lister = directories.sstableLister();
+
+ if (options.type == Options.FileType.FINAL)
+ lister.skipTemporary(true);
+ else if (options.type == Options.FileType.TMP)
+ lister.onlyTemporary(true);
+
+ for (File file : lister.listFiles())
+ handler.output(file.getCanonicalPath());
+
+ if (options.oplogs)
+ {
+ for (File file : LifecycleTransaction.getLogFiles(metadata))
+ {
+ handler.output(file.getCanonicalPath());
+ }
+ }
+
+ System.exit(0);
+ }
+ catch (Exception e)
+ {
+ System.err.println(e.getMessage());
+ if (options.debug)
+ e.printStackTrace(System.err);
+ System.exit(1);
+ }
+ }
+
+ private static class Options
+ {
+ public enum FileType
+ {
+ ALL("all", "list all files, final or temporary"),
+ TMP("tmp", "list temporary files only"),
+ FINAL("final", "list final files only");
+
+ public String option;
+ public String descr;
+ FileType(String option, String descr)
+ {
+ this.option = option;
+ this.descr = descr;
+ }
+
+ static FileType fromOption(String option)
+ {
+ for (FileType fileType : FileType.values())
+ {
+ if (fileType.option.equals(option))
+ return fileType;
+ }
+
+ return FileType.ALL;
+ }
+
+ static String descr()
+ {
+ StringBuilder str = new StringBuilder();
+ for (FileType fileType : FileType.values())
+ {
+ str.append(fileType.option);
+ str.append(" (");
+ str.append(fileType.descr);
+ str.append("), ");
+ }
+ return str.toString();
+ }
+ }
+
+ public final String keyspaceName;
+ public final String cfName;
+
+ public boolean debug;
+ public boolean verbose;
+ public boolean oplogs;
+ public FileType type;
+
+ private Options(String keyspaceName, String cfName)
+ {
+ this.keyspaceName = keyspaceName;
+ this.cfName = cfName;
+ }
+
+ public static Options parseArgs(String cmdArgs[])
+ {
+ CommandLineParser parser = new GnuParser();
+ CmdLineOptions options = getCmdLineOptions();
+ try
+ {
+ CommandLine cmd = parser.parse(options, cmdArgs, false);
+
+ if (cmd.hasOption(HELP_OPTION))
+ {
+ printUsage(options);
+ System.exit(0);
+ }
+
+ String[] args = cmd.getArgs();
+ if (args.length != 2)
+ {
+ String msg = args.length < 2 ? "Missing arguments" : "Too many arguments";
+ System.err.println(msg);
+ printUsage(options);
+ System.exit(1);
+ }
+
+ String keyspaceName = args[0];
+ String cfName = args[1];
+
+ Options opts = new Options(keyspaceName, cfName);
+
+ opts.debug = cmd.hasOption(DEBUG_OPTION);
+ opts.verbose = cmd.hasOption(VERBOSE_OPTION);
+ opts.type = FileType.fromOption(cmd.getOptionValue(TYPE_OPTION));
+ opts.oplogs = cmd.hasOption(OP_LOG_OPTION);
+
+ return opts;
+ }
+ catch (ParseException e)
+ {
+ errorMsg(e.getMessage(), options);
+ return null;
+ }
+ }
+
+ private static void errorMsg(String msg, CmdLineOptions options)
+ {
+ System.err.println(msg);
+ printUsage(options);
+ System.exit(1);
+ }
+
+ private static CmdLineOptions getCmdLineOptions()
+ {
+ CmdLineOptions options = new CmdLineOptions();
+ options.addOption("d", DEBUG_OPTION, "display stack traces");
+ options.addOption("h", HELP_OPTION, "display this help message");
+ options.addOption("o", OP_LOG_OPTION, "include operation logs");
+ options.addOption("t", TYPE_OPTION, true, FileType.descr());
+ options.addOption("v", VERBOSE_OPTION, "verbose output");
+
+ return options;
+ }
+
+ public static void printUsage(CmdLineOptions options)
+ {
+ String usage = String.format("%s [options] <keyspace> <column_family>", TOOL_NAME);
+ StringBuilder header = new StringBuilder();
+ header.append("--\n");
+ header.append("List sstable files for the provided table." );
+ header.append("\n--\n");
+ header.append("Options are:");
+ new HelpFormatter().printHelp(usage, header.toString(), options, "");
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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 be73cf7..c00d036 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
@@ -34,6 +34,7 @@ import org.apache.cassandra.db.Directories;
import org.apache.cassandra.db.Keyspace;
import org.apache.cassandra.db.compaction.*;
import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.db.lifecycle.TransactionLogs;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.sstable.*;
import org.apache.cassandra.utils.JVMStabilityInspector;
@@ -120,7 +121,7 @@ public class StandaloneScrubber
try (LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.SCRUB, sstable))
{
txn.obsoleteOriginals(); // make sure originals are deleted and avoid NPE if index is missing, CASSANDRA-9591
- try (Scrubber scrubber = new Scrubber(cfs, txn, options.skipCorrupted, handler, true, !options.noValidate))
+ try (Scrubber scrubber = new Scrubber(cfs, txn, options.skipCorrupted, handler, true, !options.noValidate, false))
{
scrubber.scrub();
}
@@ -144,7 +145,7 @@ public class StandaloneScrubber
// Check (and repair) manifests
checkManifest(cfs.getCompactionStrategyManager(), cfs, sstables);
CompactionManager.instance.finishCompactionsAndShutdown(5, TimeUnit.MINUTES);
- SSTableDeletingTask.waitForDeletions();
+ TransactionLogs.waitForDeletions();
System.exit(0); // We need that to stop non daemonized threads
}
catch (Exception e)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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 4545ab6..5a0c43f 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
@@ -23,6 +23,7 @@ import java.util.*;
import java.util.concurrent.TimeUnit;
import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.db.lifecycle.TransactionLogs;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.commons.cli.*;
@@ -154,8 +155,7 @@ public class StandaloneSplitter
new SSTableSplitter(cfs, transaction, options.sizeInMB).split();
// Remove the sstable (it's been copied by split and snapshotted)
- sstable.markObsolete(null);
- sstable.selfRef().release();
+ transaction.obsoleteOriginals();
}
catch (Exception e)
{
@@ -163,9 +163,13 @@ public class StandaloneSplitter
if (options.debug)
e.printStackTrace(System.err);
}
+ finally
+ {
+ sstable.selfRef().release();
+ }
}
CompactionManager.instance.finishCompactionsAndShutdown(5, TimeUnit.MINUTES);
- SSTableDeletingTask.waitForDeletions();
+ TransactionLogs.waitForDeletions();
System.exit(0); // We need that to stop non daemonized threads
}
catch (Exception e)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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 c5dfba0..0851d5b 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.tools;
import java.util.*;
import java.util.concurrent.TimeUnit;
+import org.apache.cassandra.db.lifecycle.TransactionLogs;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.commons.cli.*;
@@ -103,15 +104,7 @@ public class StandaloneUpgrader
try (LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.UPGRADE_SSTABLES, sstable))
{
Upgrader upgrader = new Upgrader(cfs, txn, handler);
- upgrader.upgrade();
-
- if (!options.keepSource)
- {
- // Remove the sstable (it's been copied by upgrade)
- System.out.format("Deleting table %s.%n", sstable.descriptor.baseFilename());
- sstable.markObsolete(null);
- sstable.selfRef().release();
- }
+ upgrader.upgrade(options.keepSource);
}
catch (Exception e)
{
@@ -119,9 +112,15 @@ public class StandaloneUpgrader
if (options.debug)
e.printStackTrace(System.err);
}
+ finally
+ {
+ // we should have released this through commit of the LifecycleTransaction,
+ // but in case the upgrade failed (or something else went wrong) make sure we don't retain a reference
+ sstable.selfRef().ensureReleased();
+ }
}
CompactionManager.instance.finishCompactionsAndShutdown(5, TimeUnit.MINUTES);
- SSTableDeletingTask.waitForDeletions();
+ TransactionLogs.waitForDeletions();
System.exit(0);
}
catch (Exception e)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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 5b0eb8e..f79a795 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/Transactional.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/Transactional.java
@@ -18,10 +18,6 @@
*/
package org.apache.cassandra.utils.concurrent;
-import java.util.Set;
-
-import com.google.common.collect.ImmutableSet;
-
import static org.apache.cassandra.utils.Throwables.maybeFail;
import static org.apache.cassandra.utils.Throwables.merge;
@@ -45,14 +41,17 @@ import static org.apache.cassandra.utils.Throwables.merge;
* If everything completes normally, then on exiting the try block the auto close method will invoke cleanup
* to release any temporary state/resources
*
- * No exceptions should be thrown during commit; if they are, it is not at all clear what the correct behaviour
+ * All exceptions and assertions that may be thrown should be checked and ruled out during commit preparation.
+ * Commit should generally never throw an exception unless there is a real correctness-affecting exception that
+ * cannot be moved to prepareToCommit, in which case this operation MUST be executed before any other commit
+ * methods in the object graph.
+ *
+ * If exceptions are generated by commit after this initial moment, it is not at all clear what the correct behaviour
* of the system should be, and so simply logging the exception is likely best (since it may have been an issue
- * during cleanup, say), and rollback cannot now occur. As such all exceptions and assertions that may be thrown
- * should be checked and ruled out during commit preparation.
+ * during cleanup, say), and rollback cannot now occur.
*/
public interface Transactional extends AutoCloseable
{
-
/**
* A simple abstract implementation of Transactional behaviour.
* In general this should be used as the base class for any transactional implementations.
@@ -60,9 +59,9 @@ public interface Transactional extends AutoCloseable
* If the implementation wraps any internal Transactional objects, it must proxy every
* commit() and abort() call onto each internal object to ensure correct behaviour
*/
- public static abstract class AbstractTransactional implements Transactional
+ abstract class AbstractTransactional implements Transactional
{
- public static enum State
+ public enum State
{
IN_PROGRESS,
READY_TO_COMMIT,
@@ -210,10 +209,10 @@ public interface Transactional extends AutoCloseable
// but if it does generate one it should accumulate it in the parameter and return the result
// IF a commit implementation has a real correctness affecting exception that cannot be moved to
// prepareToCommit, it MUST be executed before any other commit methods in the object graph
- public Throwable commit(Throwable accumulate);
+ Throwable commit(Throwable accumulate);
// release any resources, then rollback all state changes (unless commit() has already been invoked)
- public Throwable abort(Throwable accumulate);
+ Throwable abort(Throwable accumulate);
- public void prepareToCommit();
+ void prepareToCommit();
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-CompressionInfo.db b/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-CompressionInfo.db
new file mode 100644
index 0000000..f7a81f0
Binary files /dev/null and b/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-CompressionInfo.db differ
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-Data.db b/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-Data.db
new file mode 100644
index 0000000..2d5e60a
Binary files /dev/null and b/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-Data.db differ
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-Digest.adler32
----------------------------------------------------------------------
diff --git a/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-Digest.adler32 b/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-Digest.adler32
new file mode 100644
index 0000000..deffbd1
--- /dev/null
+++ b/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-Digest.adler32
@@ -0,0 +1 @@
+2055934203
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-Filter.db b/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-Filter.db
new file mode 100644
index 0000000..a749417
Binary files /dev/null and b/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-Filter.db differ
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-Index.db b/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-Index.db
new file mode 100644
index 0000000..d3923ab
Binary files /dev/null and b/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-Index.db differ
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-Statistics.db b/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-Statistics.db
new file mode 100644
index 0000000..664bfa5
Binary files /dev/null and b/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-Statistics.db differ
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-Summary.db b/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-Summary.db
new file mode 100644
index 0000000..a74f96f
Binary files /dev/null and b/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-Summary.db differ
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-TOC.txt b/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-TOC.txt
new file mode 100644
index 0000000..92dc9fe
--- /dev/null
+++ b/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/la-1-big-TOC.txt
@@ -0,0 +1,8 @@
+Statistics.db
+Summary.db
+TOC.txt
+Filter.db
+Data.db
+CompressionInfo.db
+Digest.adler32
+Index.db
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/tmp-la-2-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/tmp-la-2-big-Data.db b/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/tmp-la-2-big-Data.db
new file mode 100644
index 0000000..2d5e60a
Binary files /dev/null and b/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/tmp-la-2-big-Data.db differ
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/tmp-la-2-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/tmp-la-2-big-Index.db b/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/tmp-la-2-big-Index.db
new file mode 100644
index 0000000..d3923ab
Binary files /dev/null and b/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/tmp-la-2-big-Index.db differ
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/tmplink-la-2-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/tmplink-la-2-big-Data.db b/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/tmplink-la-2-big-Data.db
new file mode 100644
index 0000000..2d5e60a
Binary files /dev/null and b/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/tmplink-la-2-big-Data.db differ
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/tmplink-la-2-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/tmplink-la-2-big-Index.db b/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/tmplink-la-2-big-Index.db
new file mode 100644
index 0000000..d3923ab
Binary files /dev/null and b/test/data/migration-sstables/2.2/keyspace1/test-dfcc85801bc811e5aa694b06169f4ffa/tmplink-la-2-big-Index.db differ
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/data/migration-sstables/2.2/system/compactions_in_progress-55080ab05d9c388690a4acb25fe1f77b/snapshots/1435108403246-compactions_in_progress/manifest.json
----------------------------------------------------------------------
diff --git a/test/data/migration-sstables/2.2/system/compactions_in_progress-55080ab05d9c388690a4acb25fe1f77b/snapshots/1435108403246-compactions_in_progress/manifest.json b/test/data/migration-sstables/2.2/system/compactions_in_progress-55080ab05d9c388690a4acb25fe1f77b/snapshots/1435108403246-compactions_in_progress/manifest.json
new file mode 100644
index 0000000..d5fdb4f
--- /dev/null
+++ b/test/data/migration-sstables/2.2/system/compactions_in_progress-55080ab05d9c388690a4acb25fe1f77b/snapshots/1435108403246-compactions_in_progress/manifest.json
@@ -0,0 +1 @@
+{"files":[]}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/data/migration-sstables/2.2/system/compactions_in_progress-55080ab05d9c388690a4acb25fe1f77b/snapshots/1435298241281-upgrade-3.0.0-SNAPSHOT-2.2.0-rc1-SNAPSHOT/manifest.json
----------------------------------------------------------------------
diff --git a/test/data/migration-sstables/2.2/system/compactions_in_progress-55080ab05d9c388690a4acb25fe1f77b/snapshots/1435298241281-upgrade-3.0.0-SNAPSHOT-2.2.0-rc1-SNAPSHOT/manifest.json b/test/data/migration-sstables/2.2/system/compactions_in_progress-55080ab05d9c388690a4acb25fe1f77b/snapshots/1435298241281-upgrade-3.0.0-SNAPSHOT-2.2.0-rc1-SNAPSHOT/manifest.json
new file mode 100644
index 0000000..d5fdb4f
--- /dev/null
+++ b/test/data/migration-sstables/2.2/system/compactions_in_progress-55080ab05d9c388690a4acb25fe1f77b/snapshots/1435298241281-upgrade-3.0.0-SNAPSHOT-2.2.0-rc1-SNAPSHOT/manifest.json
@@ -0,0 +1 @@
+{"files":[]}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/test/data/migration-sstables/2.2/system/compactions_in_progress-55080ab05d9c388690a4acb25fe1f77b/snapshots/1435298241532-compactions_in_progress/manifest.json
----------------------------------------------------------------------
diff --git a/test/data/migration-sstables/2.2/system/compactions_in_progress-55080ab05d9c388690a4acb25fe1f77b/snapshots/1435298241532-compactions_in_progress/manifest.json b/test/data/migration-sstables/2.2/system/compactions_in_progress-55080ab05d9c388690a4acb25fe1f77b/snapshots/1435298241532-compactions_in_progress/manifest.json
new file mode 100644
index 0000000..d5fdb4f
--- /dev/null
+++ b/test/data/migration-sstables/2.2/system/compactions_in_progress-55080ab05d9c388690a4acb25fe1f77b/snapshots/1435298241532-compactions_in_progress/manifest.json
@@ -0,0 +1 @@
+{"files":[]}
[5/5] cassandra git commit: Introduce safer durable sstable
membership management (and simplify cleanup of compaction leftovers)
Posted by be...@apache.org.
Introduce safer durable sstable membership management
(and simplify cleanup of compaction leftovers)
Instead of using temporary files and system tables,
this patch introduces a simple transaction log for sstable
membership edits that can be committed/aborted atomically
and simply replayed on startup.
patch by stefania; reviewed by benedict for CASSANDRA-7066
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/b09e60f7
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/b09e60f7
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/b09e60f7
Branch: refs/heads/trunk
Commit: b09e60f72bb2f37235d9e9190c25db36371b3c18
Parents: e338d2f
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Mon Apr 27 14:38:53 2015 +0800
Committer: Benedict Elliott Smith <be...@apache.org>
Committed: Fri Jul 24 14:41:51 2015 +0100
----------------------------------------------------------------------
CHANGES.txt | 4 +
bin/sstablelister | 55 ++
bin/sstablelister.bat | 41 +
build.xml | 4 +-
.../org/apache/cassandra/config/Config.java | 2 +-
.../cassandra/config/DatabaseDescriptor.java | 2 +-
.../apache/cassandra/db/ColumnFamilyStore.java | 149 +---
.../org/apache/cassandra/db/Directories.java | 57 +-
src/java/org/apache/cassandra/db/Memtable.java | 31 +-
.../org/apache/cassandra/db/SystemKeyspace.java | 126 +--
.../db/compaction/CompactionController.java | 2 +-
.../db/compaction/CompactionManager.java | 22 +-
.../cassandra/db/compaction/CompactionTask.java | 24 +-
.../db/compaction/LeveledCompactionTask.java | 8 +-
.../cassandra/db/compaction/OperationType.java | 18 +-
.../db/compaction/SSTableSplitter.java | 6 +-
.../cassandra/db/compaction/Scrubber.java | 18 +-
.../SizeTieredCompactionStrategy.java | 6 +-
.../cassandra/db/compaction/Upgrader.java | 13 +-
.../writers/CompactionAwareWriter.java | 10 +-
.../writers/DefaultCompactionWriter.java | 11 +-
.../writers/MajorLeveledCompactionWriter.java | 15 +-
.../writers/MaxSSTableSizeWriter.java | 25 +-
.../SplittingSizeTieredCompactionWriter.java | 39 +-
.../apache/cassandra/db/lifecycle/Helpers.java | 58 +-
.../db/lifecycle/LifecycleTransaction.java | 150 +++-
.../apache/cassandra/db/lifecycle/Tracker.java | 50 +-
.../cassandra/db/lifecycle/TransactionLogs.java | 786 +++++++++++++++++++
.../io/sstable/AbstractSSTableSimpleWriter.java | 8 +-
.../apache/cassandra/io/sstable/Descriptor.java | 107 ++-
.../io/sstable/IndexSummaryManager.java | 7 +-
.../apache/cassandra/io/sstable/SSTable.java | 17 +-
.../io/sstable/SSTableDeletingTask.java | 130 ---
.../cassandra/io/sstable/SSTableLoader.java | 22 +-
.../cassandra/io/sstable/SSTableRewriter.java | 20 +-
.../io/sstable/SSTableSimpleUnsortedWriter.java | 7 +-
.../io/sstable/SSTableSimpleWriter.java | 5 +-
.../cassandra/io/sstable/SSTableTxnWriter.java | 101 +++
.../io/sstable/format/SSTableReader.java | 171 ++--
.../io/sstable/format/SSTableWriter.java | 50 +-
.../io/sstable/format/big/BigFormat.java | 12 +-
.../io/sstable/format/big/BigTableWriter.java | 69 +-
.../io/sstable/metadata/MetadataSerializer.java | 7 +-
.../org/apache/cassandra/io/util/FileUtils.java | 65 ++
.../cassandra/io/util/SequentialWriter.java | 12 +-
.../cassandra/service/CassandraDaemon.java | 16 +-
.../apache/cassandra/service/GCInspector.java | 10 +-
.../apache/cassandra/service/StartupChecks.java | 6 +-
.../cassandra/service/StorageService.java | 4 +-
.../cassandra/streaming/StreamLockfile.java | 128 ---
.../cassandra/streaming/StreamReader.java | 4 +-
.../cassandra/streaming/StreamReceiveTask.java | 20 +-
.../cassandra/streaming/StreamSession.java | 7 +
.../cassandra/tools/StandaloneLister.java | 214 +++++
.../cassandra/tools/StandaloneScrubber.java | 5 +-
.../cassandra/tools/StandaloneSplitter.java | 10 +-
.../cassandra/tools/StandaloneUpgrader.java | 19 +-
.../utils/concurrent/Transactional.java | 25 +-
.../la-1-big-CompressionInfo.db | Bin 0 -> 43 bytes
.../la-1-big-Data.db | Bin 0 -> 93 bytes
.../la-1-big-Digest.adler32 | 1 +
.../la-1-big-Filter.db | Bin 0 -> 16 bytes
.../la-1-big-Index.db | Bin 0 -> 54 bytes
.../la-1-big-Statistics.db | Bin 0 -> 4442 bytes
.../la-1-big-Summary.db | Bin 0 -> 80 bytes
.../la-1-big-TOC.txt | 8 +
.../tmp-la-2-big-Data.db | Bin 0 -> 93 bytes
.../tmp-la-2-big-Index.db | Bin 0 -> 54 bytes
.../tmplink-la-2-big-Data.db | Bin 0 -> 93 bytes
.../tmplink-la-2-big-Index.db | Bin 0 -> 54 bytes
.../manifest.json | 1 +
.../manifest.json | 1 +
.../manifest.json | 1 +
.../io/sstable/CQLSSTableWriterLongTest.java | 5 +-
test/unit/org/apache/cassandra/MockSchema.java | 12 +-
test/unit/org/apache/cassandra/Util.java | 3 -
.../org/apache/cassandra/cql3/CQLTester.java | 2 +-
.../cassandra/db/ColumnFamilyStoreTest.java | 150 +---
.../apache/cassandra/db/DirectoriesTest.java | 24 +-
.../unit/org/apache/cassandra/db/ScrubTest.java | 7 +-
.../apache/cassandra/db/SystemKeyspaceTest.java | 48 ++
.../db/compaction/AntiCompactionTest.java | 4 +-
.../compaction/CompactionAwareWriterTest.java | 8 +-
.../db/compaction/CompactionsTest.java | 69 +-
.../LeveledCompactionStrategyTest.java | 5 +-
.../cassandra/db/lifecycle/HelpersTest.java | 25 +-
.../db/lifecycle/LifecycleTransactionTest.java | 9 +-
.../db/lifecycle/RealTransactionsTest.java | 228 ++++++
.../cassandra/db/lifecycle/TrackerTest.java | 40 +-
.../db/lifecycle/TransactionLogsTest.java | 558 +++++++++++++
.../io/sstable/BigTableWriterTest.java | 27 +-
.../io/sstable/CQLSSTableWriterClientTest.java | 9 +
.../cassandra/io/sstable/DescriptorTest.java | 18 +-
.../cassandra/io/sstable/LegacySSTableTest.java | 2 +-
.../cassandra/io/sstable/SSTableLoaderTest.java | 123 ++-
.../io/sstable/SSTableRewriterTest.java | 173 ++--
.../cassandra/io/sstable/SSTableUtils.java | 11 +-
.../metadata/MetadataSerializerTest.java | 2 +-
.../org/apache/cassandra/schema/DefsTest.java | 4 +-
.../streaming/StreamTransferTaskTest.java | 3 +-
.../concurrent/AbstractTransactionalTest.java | 31 +-
101 files changed, 3265 insertions(+), 1357 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 70b26f5..a0dadc3 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,5 +1,9 @@
3.0
+<<<<<<< HEAD
* Implement proper sandboxing for UDFs (CASSANDRA-9402)
+=======
+ * Simplify (and unify) cleanup of compaction leftovers (CASSANDRA-7066)
+>>>>>>> Introduce safer durable sstable membership management
* Allow extra schema definitions in cassandra-stress yaml (CASSANDRA-9850)
* Metrics should use up to date nomenclature (CASSANDRA-9448)
* Change CREATE/ALTER TABLE syntax for compression (CASSANDRA-8384)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/bin/sstablelister
----------------------------------------------------------------------
diff --git a/bin/sstablelister b/bin/sstablelister
new file mode 100755
index 0000000..a79409d
--- /dev/null
+++ b/bin/sstablelister
@@ -0,0 +1,55 @@
+#!/bin/sh
+
+# 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.
+
+if [ "x$CASSANDRA_INCLUDE" = "x" ]; then
+ for include in /usr/share/cassandra/cassandra.in.sh \
+ /usr/local/share/cassandra/cassandra.in.sh \
+ /opt/cassandra/cassandra.in.sh \
+ ~/.cassandra.in.sh \
+ "`dirname "$0"`/cassandra.in.sh"; do
+ if [ -r "$include" ]; then
+ . "$include"
+ break
+ fi
+ done
+elif [ -r "$CASSANDRA_INCLUDE" ]; then
+ . "$CASSANDRA_INCLUDE"
+fi
+
+# Use JAVA_HOME if set, otherwise look for java in PATH
+if [ -x "$JAVA_HOME/bin/java" ]; then
+ JAVA="$JAVA_HOME/bin/java"
+else
+ JAVA="`which java`"
+fi
+
+if [ -z "$CLASSPATH" ]; then
+ echo "You must set the CLASSPATH var" >&2
+ exit 1
+fi
+
+if [ "x$MAX_HEAP_SIZE" = "x" ]; then
+ MAX_HEAP_SIZE="256M"
+fi
+
+"$JAVA" $JAVA_AGENT -ea -cp "$CLASSPATH" $JVM_OPTS -Xmx$MAX_HEAP_SIZE \
+ -Dcassandra.storagedir="$cassandra_storagedir" \
+ -Dlogback.configurationFile=logback-tools.xml \
+ org.apache.cassandra.tools.StandaloneLister "$@"
+
+# vi:ai sw=4 ts=4 tw=0 et
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/bin/sstablelister.bat
----------------------------------------------------------------------
diff --git a/bin/sstablelister.bat b/bin/sstablelister.bat
new file mode 100644
index 0000000..cb50a08
--- /dev/null
+++ b/bin/sstablelister.bat
@@ -0,0 +1,41 @@
+@REM
+@REM Licensed to the Apache Software Foundation (ASF) under one or more
+@REM contributor license agreements. See the NOTICE file distributed with
+@REM this work for additional information regarding copyright ownership.
+@REM The ASF licenses this file to You under the Apache License, Version 2.0
+@REM (the "License"); you may not use this file except in compliance with
+@REM the License. You may obtain a copy of the License at
+@REM
+@REM http://www.apache.org/licenses/LICENSE-2.0
+@REM
+@REM Unless required by applicable law or agreed to in writing, software
+@REM distributed under the License is distributed on an "AS IS" BASIS,
+@REM WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+@REM See the License for the specific language governing permissions and
+@REM limitations under the License.
+
+@echo off
+if "%OS%" == "Windows_NT" setlocal
+
+pushd "%~dp0"
+call cassandra.in.bat
+
+if NOT DEFINED CASSANDRA_MAIN set CASSANDRA_MAIN=org.apache.cassandra.tools.StandaloneLister
+if NOT DEFINED JAVA_HOME goto :err
+
+REM ***** JAVA options *****
+set JAVA_OPTS=^
+ -Dlogback.configurationFile=logback-tools.xml
+
+set TOOLS_PARAMS=
+
+"%JAVA_HOME%\bin\java" %JAVA_OPTS% %CASSANDRA_PARAMS% -cp %CASSANDRA_CLASSPATH% "%CASSANDRA_MAIN%" %*
+goto finally
+
+:err
+echo JAVA_HOME environment variable must be set!
+pause
+
+:finally
+
+ENDLOCAL
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/build.xml
----------------------------------------------------------------------
diff --git a/build.xml b/build.xml
index 5a82d64..e6581ea 100644
--- a/build.xml
+++ b/build.xml
@@ -1214,7 +1214,7 @@
<jvmarg value="-Xss256k"/>
<jvmarg value="-Dcassandra.memtable_row_overhead_computation_step=100"/>
<jvmarg value="-Dcassandra.test.use_prepared=${cassandra.test.use_prepared}"/>
- <jvmarg value="-Dcassandra.test.offsetseed=@{poffset}"/>
+ <jvmarg value="-Dcassandra.test.offsetseed=@{poffset}"/>
<jvmarg value="-Dcassandra.test.sstableformatdevelopment=true"/>
<jvmarg value="-Dcassandra.testtag=@{testtag}"/>
<jvmarg value="-Dcassandra.keepBriefBrief=${cassandra.keepBriefBrief}" />
@@ -1882,7 +1882,7 @@
<option name="MAIN_CLASS_NAME" value="" />
<option name="METHOD_NAME" value="" />
<option name="TEST_OBJECT" value="class" />
- <option name="VM_PARAMETERS" value="-Dcassandra.config=file://$PROJECT_DIR$/test/conf/cassandra.yaml -Dlogback.configurationFile=file://$PROJECT_DIR$/test/conf/logback-test.xml -Dcassandra.logdir=$PROJECT_DIR$/build/test/logs -ea" />
+ <option name="VM_PARAMETERS" value="-Dcassandra.debugrefcount=true -Dcassandra.config=file://$PROJECT_DIR$/test/conf/cassandra.yaml -Dlogback.configurationFile=file://$PROJECT_DIR$/test/conf/logback-test.xml -Dcassandra.logdir=$PROJECT_DIR$/build/test/logs -Dcorrupt-sstable-root=$PROJECT_DIR$/test/data/corrupt-sstables -Dmigration-sstable-root=${test.data}/migration-sstables -ea" />
<option name="PARAMETERS" value="" />
<option name="WORKING_DIRECTORY" value="" />
<option name="ENV_VARIABLES" />
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/src/java/org/apache/cassandra/config/Config.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java
index 2a2062e..fe6752f 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -154,7 +154,7 @@ public class Config
public volatile Integer stream_throughput_outbound_megabits_per_sec = 200;
public volatile Integer inter_dc_stream_throughput_outbound_megabits_per_sec = 0;
- public String[] data_file_directories;
+ public String[] data_file_directories = new String[0];
public String saved_caches_directory;
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 01b1633..a25af65 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -490,7 +490,7 @@ public class DatabaseDescriptor
throw new ConfigurationException("saved_caches_directory is missing and -Dcassandra.storagedir is not set", false);
conf.saved_caches_directory += File.separator + "saved_caches";
}
- if (conf.data_file_directories == null)
+ if (conf.data_file_directories == null || conf.data_file_directories.length == 0)
{
String defaultDataDir = System.getProperty("cassandra.storagedir", null);
if (defaultDataDir == null)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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 d3ad4e6..8d14120 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -60,18 +60,14 @@ import org.apache.cassandra.db.partitions.*;
import org.apache.cassandra.dht.*;
import org.apache.cassandra.dht.Range;
import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.io.FSReadError;
import org.apache.cassandra.io.compress.CompressionParameters;
import org.apache.cassandra.io.sstable.Descriptor;
import org.apache.cassandra.io.sstable.*;
import org.apache.cassandra.io.sstable.format.*;
-import org.apache.cassandra.io.sstable.metadata.CompactionMetadata;
-import org.apache.cassandra.io.sstable.metadata.MetadataType;
import org.apache.cassandra.io.util.FileUtils;
import org.apache.cassandra.metrics.TableMetrics.Sampler;
import org.apache.cassandra.service.CacheService;
import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.streaming.StreamLockfile;
import org.apache.cassandra.utils.*;
import org.apache.cassandra.utils.concurrent.*;
import org.apache.cassandra.utils.TopKSampler.SamplerResult;
@@ -522,45 +518,20 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
{
Directories directories = new Directories(metadata);
- // clear ephemeral snapshots that were not properly cleared last session (CASSANDRA-7357)
+ // clear ephemeral snapshots that were not properly cleared last session (CASSANDRA-7357)
clearEphemeralSnapshots(directories);
- // remove any left-behind SSTables from failed/stalled streaming
- FileFilter filter = new FileFilter()
- {
- public boolean accept(File pathname)
- {
- return pathname.getPath().endsWith(StreamLockfile.FILE_EXT);
- }
- };
- for (File dir : directories.getCFDirectories())
- {
- File[] lockfiles = dir.listFiles(filter);
- // lock files can be null if I/O error happens
- if (lockfiles == null || lockfiles.length == 0)
- continue;
- logger.info("Removing SSTables from failed streaming session. Found {} files to cleanup.", lockfiles.length);
-
- for (File lockfile : lockfiles)
- {
- StreamLockfile streamLockfile = new StreamLockfile(lockfile);
- streamLockfile.cleanup();
- streamLockfile.delete();
- }
- }
-
- logger.debug("Removing compacted SSTable files from {} (see http://wiki.apache.org/cassandra/MemtableSSTable)", metadata.cfName);
+ logger.debug("Removing temporary or obsoleted files from unfinished operations for table", metadata.cfName);
+ LifecycleTransaction.removeUnfinishedLeftovers(metadata);
+ logger.debug("Further extra check for orphan sstable files for {}", metadata.cfName);
for (Map.Entry<Descriptor,Set<Component>> sstableFiles : directories.sstableLister().list().entrySet())
{
Descriptor desc = sstableFiles.getKey();
Set<Component> components = sstableFiles.getValue();
- if (desc.type.isTemporary)
- {
- SSTable.delete(desc, components);
- continue;
- }
+ for (File tmpFile : desc.getTemporaryFiles())
+ tmpFile.delete();
File dataFile = new File(desc.filenameFor(Component.DATA));
if (components.contains(Component.DATA) && dataFile.length() > 0)
@@ -571,7 +542,9 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
logger.warn("Removing orphans for {}: {}", desc, components);
for (Component component : components)
{
- FileUtils.deleteWithConfirm(desc.filenameFor(component));
+ File file = new File(desc.filenameFor(component));
+ if (file.exists())
+ FileUtils.deleteWithConfirm(desc.filenameFor(component));
}
}
@@ -600,91 +573,6 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
}
}
- /**
- * 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.
- *
- * If an unclean shutdown happens at the right time, we can thus end up with both the new ones and their
- * ancestors "live" in the system. This is harmless for normal data, but for counters it can cause overcounts.
- *
- * To prevent this, we record sstables being compacted in the system keyspace. If we find unfinished
- * compactions, we remove the new ones (since those may be incomplete -- under LCS, we may create multiple
- * sstables from any given ancestor).
- */
- public static void removeUnfinishedCompactionLeftovers(CFMetaData metadata, Map<Integer, UUID> unfinishedCompactions)
- {
- Directories directories = new Directories(metadata);
-
- Set<Integer> allGenerations = new HashSet<>();
- for (Descriptor desc : directories.sstableLister().list().keySet())
- allGenerations.add(desc.generation);
-
- // sanity-check unfinishedCompactions
- Set<Integer> unfinishedGenerations = unfinishedCompactions.keySet();
- if (!allGenerations.containsAll(unfinishedGenerations))
- {
- HashSet<Integer> missingGenerations = new HashSet<>(unfinishedGenerations);
- missingGenerations.removeAll(allGenerations);
- logger.debug("Unfinished compactions of {}.{} reference missing sstables of generations {}",
- metadata.ksName, metadata.cfName, missingGenerations);
- }
-
- // remove new sstables from compactions that didn't complete, and compute
- // set of ancestors that shouldn't exist anymore
- Set<Integer> completedAncestors = new HashSet<>();
- for (Map.Entry<Descriptor, Set<Component>> sstableFiles : directories.sstableLister().skipTemporary(true).list().entrySet())
- {
- Descriptor desc = sstableFiles.getKey();
-
- Set<Integer> ancestors;
- try
- {
- CompactionMetadata compactionMetadata = (CompactionMetadata) desc.getMetadataSerializer().deserialize(desc, MetadataType.COMPACTION);
- ancestors = compactionMetadata.ancestors;
- }
- catch (IOException e)
- {
- throw new FSReadError(e, desc.filenameFor(Component.STATS));
- }
- catch (NullPointerException e)
- {
- throw new FSReadError(e, "Failed to remove unfinished compaction leftovers (file: " + desc.filenameFor(Component.STATS) + "). See log for details.");
- }
-
- if (!ancestors.isEmpty()
- && unfinishedGenerations.containsAll(ancestors)
- && allGenerations.containsAll(ancestors))
- {
- // any of the ancestors would work, so we'll just lookup the compaction task ID with the first one
- UUID compactionTaskID = unfinishedCompactions.get(ancestors.iterator().next());
- assert compactionTaskID != null;
- logger.debug("Going to delete unfinished compaction product {}", desc);
- SSTable.delete(desc, sstableFiles.getValue());
- SystemKeyspace.finishCompaction(compactionTaskID);
- }
- else
- {
- completedAncestors.addAll(ancestors);
- }
- }
-
- // remove old sstables from compactions that did complete
- for (Map.Entry<Descriptor, Set<Component>> sstableFiles : directories.sstableLister().list().entrySet())
- {
- Descriptor desc = sstableFiles.getKey();
- if (completedAncestors.contains(desc.generation))
- {
- // if any of the ancestors were participating in a compaction, finish that compaction
- logger.debug("Going to delete leftover compaction ancestor {}", desc);
- SSTable.delete(desc, sstableFiles.getValue());
- UUID compactionTaskID = unfinishedCompactions.get(desc.generation);
- if (compactionTaskID != null)
- SystemKeyspace.finishCompaction(unfinishedCompactions.get(desc.generation));
- }
- }
- }
-
// must be called after all sstables are loaded since row cache merges all row versions
public void initRowCache()
{
@@ -750,8 +638,6 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
if (currentDescriptors.contains(descriptor))
continue; // old (initialized) SSTable found, skipping
- if (descriptor.type.isTemporary) // in the process of being written
- continue;
if (!descriptor.isCompatible())
throw new RuntimeException(String.format("Can't open incompatible SSTable! Current version %s, found file: %s",
@@ -780,7 +666,6 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
descriptor.ksname,
descriptor.cfname,
fileIndexGenerator.incrementAndGet(),
- Descriptor.Type.FINAL,
descriptor.formatType);
}
while (new File(newDescriptor.filenameFor(Component.DATA)).exists());
@@ -851,24 +736,23 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
return name;
}
- public String getTempSSTablePath(File directory)
+ public String getSSTablePath(File directory)
{
- return getTempSSTablePath(directory, DatabaseDescriptor.getSSTableFormat().info.getLatestVersion(), DatabaseDescriptor.getSSTableFormat());
+ return getSSTablePath(directory, DatabaseDescriptor.getSSTableFormat().info.getLatestVersion(), DatabaseDescriptor.getSSTableFormat());
}
- public String getTempSSTablePath(File directory, SSTableFormat.Type format)
+ public String getSSTablePath(File directory, SSTableFormat.Type format)
{
- return getTempSSTablePath(directory, format.info.getLatestVersion(), format);
+ return getSSTablePath(directory, format.info.getLatestVersion(), format);
}
- private String getTempSSTablePath(File directory, Version version, SSTableFormat.Type format)
+ private String getSSTablePath(File directory, Version version, SSTableFormat.Type format)
{
Descriptor desc = new Descriptor(version,
directory,
keyspace.getName(),
name,
fileIndexGenerator.incrementAndGet(),
- Descriptor.Type.TEMP,
format);
return desc.filenameFor(Component.DATA);
}
@@ -1883,11 +1767,6 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
return directories.getSnapshotDetails();
}
- public boolean hasUnreclaimedSpace()
- {
- return metric.liveDiskSpaceUsed.getCount() < metric.totalDiskSpaceUsed.getCount();
- }
-
/**
* @return the cached partition for @param key if it is already present in the cache.
* Not that this will not readAndCache the parition if it is not present, nor
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/src/java/org/apache/cassandra/db/Directories.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Directories.java b/src/java/org/apache/cassandra/db/Directories.java
index 8b61c68..bede4c4 100644
--- a/src/java/org/apache/cassandra/db/Directories.java
+++ b/src/java/org/apache/cassandra/db/Directories.java
@@ -44,6 +44,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.cassandra.config.*;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.io.FSError;
import org.apache.cassandra.io.FSWriteError;
import org.apache.cassandra.io.util.FileUtils;
@@ -91,6 +92,7 @@ public class Directories
public static final String BACKUPS_SUBDIR = "backups";
public static final String SNAPSHOT_SUBDIR = "snapshots";
+ public static final String TRANSACTIONS_SUBDIR = "transactions";
public static final String SECONDARY_INDEX_NAME_SEPARATOR = ".";
public static final DataDirectory[] dataDirectories;
@@ -466,6 +468,35 @@ public class Directories
}
}
+ public static File getTransactionsDirectory(File folder)
+ {
+ return getOrCreate(folder, TRANSACTIONS_SUBDIR);
+ }
+
+ public List<File> getExistingDirectories(String subFolder)
+ {
+ List<File> ret = new ArrayList<>();
+ for (File dir : dataPaths)
+ {
+ File subDir = getExistingDirectory(dir, subFolder);
+ if (subDir != null)
+ ret.add(subDir);
+
+ }
+ return ret;
+ }
+
+ public static File getExistingDirectory(File folder, String subFolder)
+ {
+ File subDir = new File(folder, join(subFolder));
+ if (subDir.exists())
+ {
+ assert(subDir.isDirectory());
+ return subDir;
+ }
+ return null;
+ }
+
public SSTableLister sstableLister()
{
return new SSTableLister();
@@ -521,6 +552,7 @@ public class Directories
public class SSTableLister
{
private boolean skipTemporary;
+ private boolean onlyTemporary;
private boolean includeBackups;
private boolean onlyBackups;
private int nbFiles;
@@ -536,6 +568,14 @@ public class Directories
return this;
}
+ public SSTableLister onlyTemporary(boolean b)
+ {
+ if (filtered)
+ throw new IllegalStateException("list() has already been called");
+ onlyTemporary = b;
+ return this;
+ }
+
public SSTableLister includeBackups(boolean b)
{
if (filtered)
@@ -593,21 +633,25 @@ public class Directories
if (snapshotName != null)
{
- getSnapshotDirectory(location, snapshotName).listFiles(getFilter());
+ getSnapshotDirectory(location, snapshotName).listFiles(getFilter(location));
continue;
}
if (!onlyBackups)
- location.listFiles(getFilter());
+ location.listFiles(getFilter(location));
if (includeBackups)
- getBackupsDirectory(location).listFiles(getFilter());
+ getBackupsDirectory(location).listFiles(getFilter(location));
}
filtered = true;
}
- private FileFilter getFilter()
+ private FileFilter getFilter(File location)
{
+ final Set<File> temporaryFiles = skipTemporary || onlyTemporary
+ ? LifecycleTransaction.getTemporaryFiles(metadata, location)
+ : Collections.<File>emptySet();
+
return new FileFilter()
{
// This function always return false since accepts adds to the components map
@@ -624,7 +668,10 @@ public class Directories
if (!pair.left.ksname.equals(metadata.ksName) || !pair.left.cfname.equals(metadata.cfName))
return false;
- if (skipTemporary && pair.left.type.isTemporary)
+ if (skipTemporary && temporaryFiles.contains(file))
+ return false;
+
+ if (onlyTemporary && !temporaryFiles.contains(file))
return false;
Set<Component> previous = components.get(pair.left);
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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 71e03d5..ecaf063 100644
--- a/src/java/org/apache/cassandra/db/Memtable.java
+++ b/src/java/org/apache/cassandra/db/Memtable.java
@@ -28,6 +28,9 @@ import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import com.google.common.annotations.VisibleForTesting;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.io.sstable.SSTableTxnWriter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -367,8 +370,7 @@ public class Memtable implements Comparable<Memtable>
logger.info("Writing {}", Memtable.this.toString());
SSTableReader ssTable;
- // errors when creating the writer that may leave empty temp files.
- try (SSTableWriter writer = createFlushWriter(cfs.getTempSSTablePath(sstableDirectory), columnsCollector.get(), statsCollector.get()))
+ try (SSTableTxnWriter writer = createFlushWriter(cfs.getSSTablePath(sstableDirectory), columnsCollector.get(), statsCollector.get()))
{
boolean trackContention = logger.isDebugEnabled();
int heavilyContendedRowCount = 0;
@@ -400,10 +402,10 @@ public class Memtable implements Comparable<Memtable>
{
logger.info(String.format("Completed flushing %s (%s) for commitlog position %s",
writer.getFilename(),
- FBUtilities.prettyPrintMemory(writer.getOnDiskFilePointer()),
+ FBUtilities.prettyPrintMemory(writer.getFilePointer()),
context));
- // temp sstables should contain non-repaired data.
+ // sstables should contain non-repaired data.
ssTable = writer.finish(true);
}
else
@@ -421,18 +423,23 @@ public class Memtable implements Comparable<Memtable>
}
}
- public SSTableWriter createFlushWriter(String filename,
+ public SSTableTxnWriter createFlushWriter(String filename,
PartitionColumns columns,
EncodingStats stats)
{
+ // we operate "offline" here, as we expose the resulting reader consciously when done
+ // (although we may want to modify this behaviour in future, to encapsulate full flush behaviour in LifecycleTransaction)
+ LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.FLUSH, cfs.metadata);
MetadataCollector sstableMetadataCollector = new MetadataCollector(cfs.metadata.comparator).replayPosition(context);
- return SSTableWriter.create(Descriptor.fromFilename(filename),
- (long)partitions.size(),
- ActiveRepairService.UNREPAIRED_SSTABLE,
- cfs.metadata,
- cfs.partitioner,
- sstableMetadataCollector,
- new SerializationHeader(cfs.metadata, columns, stats));
+ return new SSTableTxnWriter(txn,
+ SSTableWriter.create(Descriptor.fromFilename(filename),
+ (long)partitions.size(),
+ ActiveRepairService.UNREPAIRED_SSTABLE,
+ cfs.metadata,
+ cfs.partitioner,
+ sstableMetadataCollector,
+ new SerializationHeader(cfs.metadata, columns, stats),
+ txn));
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/src/java/org/apache/cassandra/db/SystemKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java
index 0957af6..6a4a847 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -26,7 +26,6 @@ import java.util.concurrent.TimeUnit;
import javax.management.openmbean.OpenDataException;
import javax.management.openmbean.TabularData;
-import com.google.common.base.Function;
import com.google.common.collect.*;
import com.google.common.io.ByteStreams;
@@ -47,9 +46,10 @@ import org.apache.cassandra.dht.IPartitioner;
import org.apache.cassandra.dht.Range;
import org.apache.cassandra.dht.Token;
import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.Descriptor;
import org.apache.cassandra.io.util.DataInputBuffer;
import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.FileUtils;
import org.apache.cassandra.io.util.NIODataInputStream;
import org.apache.cassandra.locator.IEndpointSnitch;
import org.apache.cassandra.metrics.RestorableMeter;
@@ -97,7 +97,6 @@ public final class SystemKeyspace
public static final String PEERS = "peers";
public static final String PEER_EVENTS = "peer_events";
public static final String RANGE_XFERS = "range_xfers";
- public static final String COMPACTIONS_IN_PROGRESS = "compactions_in_progress";
public static final String COMPACTION_HISTORY = "compaction_history";
public static final String SSTABLE_ACTIVITY = "sstable_activity";
public static final String SIZE_ESTIMATES = "size_estimates";
@@ -216,16 +215,6 @@ public final class SystemKeyspace
+ "requested_at timestamp,"
+ "PRIMARY KEY ((token_bytes)))");
- private static final CFMetaData CompactionsInProgress =
- compile(COMPACTIONS_IN_PROGRESS,
- "unfinished compactions",
- "CREATE TABLE %s ("
- + "id uuid,"
- + "columnfamily_name text,"
- + "inputs set<int>,"
- + "keyspace_name text,"
- + "PRIMARY KEY ((id)))");
-
private static final CFMetaData CompactionHistory =
compile(COMPACTION_HISTORY,
"week-long compaction history",
@@ -408,7 +397,6 @@ public final class SystemKeyspace
Peers,
PeerEvents,
RangeXfers,
- CompactionsInProgress,
CompactionHistory,
SSTableActivity,
SizeEstimates,
@@ -485,81 +473,6 @@ public final class SystemKeyspace
FBUtilities.getLocalAddress());
}
- /**
- * Write compaction log, except columfamilies under system keyspace.
- *
- * @param cfs cfs to compact
- * @param toCompact sstables to compact
- * @return compaction task id or null if cfs is under system keyspace
- */
- public static UUID startCompaction(ColumnFamilyStore cfs, Iterable<SSTableReader> toCompact)
- {
- if (Schema.isSystemKeyspace(cfs.keyspace.getName()))
- return null;
-
- UUID compactionId = UUIDGen.getTimeUUID();
- Iterable<Integer> generations = Iterables.transform(toCompact, new Function<SSTableReader, Integer>()
- {
- public Integer apply(SSTableReader sstable)
- {
- return sstable.descriptor.generation;
- }
- });
- String req = "INSERT INTO system.%s (id, keyspace_name, columnfamily_name, inputs) VALUES (?, ?, ?, ?)";
- executeInternal(String.format(req, COMPACTIONS_IN_PROGRESS), compactionId, cfs.keyspace.getName(), cfs.name, Sets.newHashSet(generations));
- forceBlockingFlush(COMPACTIONS_IN_PROGRESS);
- return compactionId;
- }
-
- /**
- * Deletes the entry for this compaction from the set of compactions in progress. The compaction does not need
- * to complete successfully for this to be called.
- * @param taskId what was returned from {@code startCompaction}
- */
- public static void finishCompaction(UUID taskId)
- {
- assert taskId != null;
-
- executeInternal(String.format("DELETE FROM system.%s WHERE id = ?", COMPACTIONS_IN_PROGRESS), taskId);
- forceBlockingFlush(COMPACTIONS_IN_PROGRESS);
- }
-
- /**
- * Returns a Map whose keys are KS.CF pairs and whose values are maps from sstable generation numbers to the
- * task ID of the compaction they were participating in.
- */
- public static Map<Pair<String, String>, Map<Integer, UUID>> getUnfinishedCompactions()
- {
- String req = "SELECT * FROM system.%s";
- UntypedResultSet resultSet = executeInternal(String.format(req, COMPACTIONS_IN_PROGRESS));
-
- Map<Pair<String, String>, Map<Integer, UUID>> unfinishedCompactions = new HashMap<>();
- for (UntypedResultSet.Row row : resultSet)
- {
- String keyspace = row.getString("keyspace_name");
- String columnfamily = row.getString("columnfamily_name");
- Set<Integer> inputs = row.getSet("inputs", Int32Type.instance);
- UUID taskID = row.getUUID("id");
-
- Pair<String, String> kscf = Pair.create(keyspace, columnfamily);
- Map<Integer, UUID> generationToTaskID = unfinishedCompactions.get(kscf);
- if (generationToTaskID == null)
- generationToTaskID = new HashMap<>(inputs.size());
-
- for (Integer generation : inputs)
- generationToTaskID.put(generation, taskID);
-
- unfinishedCompactions.put(kscf, generationToTaskID);
- }
- return unfinishedCompactions;
- }
-
- public static void discardCompactionsInProgress()
- {
- ColumnFamilyStore compactionLog = Keyspace.open(NAME).getColumnFamilyStore(COMPACTIONS_IN_PROGRESS);
- compactionLog.truncateBlocking();
- }
-
public static void updateCompactionHistory(String ksname,
String cfname,
long compactedAt,
@@ -1227,7 +1140,7 @@ public final class SystemKeyspace
*
* @throws IOException
*/
- public static void snapshotOnVersionChange() throws IOException
+ public static boolean snapshotOnVersionChange() throws IOException
{
String previous = getPreviousVersionString();
String next = FBUtilities.getReleaseVersionString();
@@ -1242,7 +1155,10 @@ public final class SystemKeyspace
next));
Keyspace systemKs = Keyspace.open(SystemKeyspace.NAME);
systemKs.snapshot(snapshotName, null);
+ return true;
}
+
+ return false;
}
/**
@@ -1282,6 +1198,36 @@ public final class SystemKeyspace
return result.one().getString("release_version");
}
+ /**
+ * Check data directories for old files that can be removed when migrating from 2.2 to 3.0,
+ * these checks can be removed in 4.0, see CASSANDRA-7066
+ */
+ public static void migrateDataDirs()
+ {
+ Iterable<String> dirs = Arrays.asList(DatabaseDescriptor.getAllDataFileLocations());
+ for (String dataDir : dirs)
+ {
+ logger.debug("Checking directory {} for old files", dataDir);
+ File dir = new File(dataDir);
+ assert dir.exists() : dir + " should have been created by startup checks";
+
+ for (File ksdir : dir.listFiles((d, n) -> d.isDirectory()))
+ {
+ for (File cfdir : ksdir.listFiles((d, n) -> d.isDirectory()))
+ {
+ if (Descriptor.isLegacyFile(cfdir.getName()))
+ {
+ FileUtils.deleteRecursive(cfdir);
+ }
+ else
+ {
+ FileUtils.delete(cfdir.listFiles((d, n) -> Descriptor.isLegacyFile(n)));
+ }
+ }
+ }
+ }
+ }
+
private static ByteBuffer rangeToBytes(Range<Token> range)
{
try (DataOutputBuffer out = new DataOutputBuffer())
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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 303de15..df3bc4e 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionController.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionController.java
@@ -63,7 +63,7 @@ public class CompactionController implements AutoCloseable
refreshOverlaps();
}
- void maybeRefreshOverlaps()
+ public void maybeRefreshOverlaps()
{
for (SSTableReader reader : overlappingSSTables)
{
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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 6cf2e18..616c310 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -823,7 +823,7 @@ public class CompactionManager implements CompactionManagerMBean
CompactionController controller = new CompactionController(cfs, txn.originals(), getDefaultGcBefore(cfs, nowInSec));
CompactionIterator ci = new CompactionIterator(OperationType.CLEANUP, Collections.singletonList(scanner), controller, nowInSec, UUIDGen.getTimeUUID(), metrics))
{
- writer.switchWriter(createWriter(cfs, compactionFileLocation, expectedBloomFilterSize, sstable.getSSTableMetadata().repairedAt, sstable));
+ writer.switchWriter(createWriter(cfs, compactionFileLocation, expectedBloomFilterSize, sstable.getSSTableMetadata().repairedAt, sstable, txn));
while (ci.hasNext())
{
@@ -948,24 +948,27 @@ public class CompactionManager implements CompactionManagerMBean
File compactionFileLocation,
int expectedBloomFilterSize,
long repairedAt,
- SSTableReader sstable)
+ SSTableReader sstable,
+ LifecycleTransaction txn)
{
FileUtils.createDirectory(compactionFileLocation);
return SSTableWriter.create(cfs.metadata,
- Descriptor.fromFilename(cfs.getTempSSTablePath(compactionFileLocation)),
+ Descriptor.fromFilename(cfs.getSSTablePath(compactionFileLocation)),
expectedBloomFilterSize,
repairedAt,
sstable.getSSTableLevel(),
cfs.partitioner,
- sstable.header);
+ sstable.header,
+ txn);
}
public static SSTableWriter createWriterForAntiCompaction(ColumnFamilyStore cfs,
File compactionFileLocation,
int expectedBloomFilterSize,
long repairedAt,
- Collection<SSTableReader> sstables)
+ Collection<SSTableReader> sstables,
+ LifecycleTransaction txn)
{
FileUtils.createDirectory(compactionFileLocation);
int minLevel = Integer.MAX_VALUE;
@@ -983,13 +986,14 @@ public class CompactionManager implements CompactionManagerMBean
break;
}
}
- return SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(compactionFileLocation)),
+ return SSTableWriter.create(Descriptor.fromFilename(cfs.getSSTablePath(compactionFileLocation)),
(long) expectedBloomFilterSize,
repairedAt,
cfs.metadata,
cfs.partitioner,
new MetadataCollector(sstables, cfs.metadata.comparator, minLevel),
- SerializationHeader.make(cfs.metadata, sstables));
+ SerializationHeader.make(cfs.metadata, sstables),
+ txn);
}
@@ -1198,8 +1202,8 @@ public class CompactionManager implements CompactionManagerMBean
{
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));
+ repairedSSTableWriter.switchWriter(CompactionManager.createWriterForAntiCompaction(cfs, destination, expectedBloomFilterSize, repairedAt, sstableAsSet, anticompactionGroup));
+ unRepairedSSTableWriter.switchWriter(CompactionManager.createWriterForAntiCompaction(cfs, destination, expectedBloomFilterSize, ActiveRepairService.UNREPAIRED_SSTABLE, sstableAsSet, anticompactionGroup));
while (ci.hasNext())
{
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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 6335834..7897a1a 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@ -17,7 +17,6 @@
*/
package org.apache.cassandra.db.compaction;
-import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
@@ -46,7 +45,6 @@ import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.service.ActiveRepairService;
import org.apache.cassandra.utils.CloseableIterator;
import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.UUIDGen;
import org.apache.cassandra.utils.concurrent.Refs;
public class CompactionTask extends AbstractCompactionTask
@@ -128,7 +126,7 @@ public class CompactionTask extends AbstractCompactionTask
}
});
- UUID taskId = SystemKeyspace.startCompaction(cfs, transaction.originals());
+ UUID taskId = transaction.opId();
// 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
@@ -139,8 +137,8 @@ public class CompactionTask extends AbstractCompactionTask
ssTableLoggerMsg.append(String.format("%s:level=%d, ", sstr.getFilename(), sstr.getSSTableLevel()));
}
ssTableLoggerMsg.append("]");
- String taskIdLoggerMsg = taskId == null ? UUIDGen.getTimeUUID().toString() : taskId.toString();
- logger.info("Compacting ({}) {}", taskIdLoggerMsg, ssTableLoggerMsg);
+
+ logger.info("Compacting ({}) {}", taskId, ssTableLoggerMsg);
long start = System.nanoTime();
long totalKeysWritten = 0;
@@ -186,16 +184,11 @@ public class CompactionTask extends AbstractCompactionTask
}
}
- // don't replace old sstables yet, as we need to mark the compaction finished in the system table
+ // point of no return
newSStables = writer.finish();
}
finally
{
- // point of no return -- the new sstables are live on disk; next we'll start deleting the old ones
- // (in replaceCompactedSSTables)
- if (taskId != null)
- SystemKeyspace.finishCompaction(taskId);
-
if (collector != null)
collector.finishCompaction(ci);
@@ -217,7 +210,7 @@ public class CompactionTask extends AbstractCompactionTask
long totalSourceRows = 0;
String mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), mergedRowCounts, 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, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, totalSourceRows, totalKeysWritten, mergeSummary));
+ taskId, 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));
@@ -227,10 +220,11 @@ public class CompactionTask extends AbstractCompactionTask
}
@Override
- public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, LifecycleTransaction transaction, Set<SSTableReader> nonExpiredSSTables)
+ public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs,
+ LifecycleTransaction transaction,
+ Set<SSTableReader> nonExpiredSSTables)
{
- return new DefaultCompactionWriter(cfs, transaction, nonExpiredSSTables, offline, compactionType);
-
+ return new DefaultCompactionWriter(cfs, transaction, nonExpiredSSTables, offline);
}
public static String updateCompactionHistory(String keyspaceName, String columnFamilyName, long[] mergedRowCounts, long startSize, long endSize)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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 1c3b686..d3d56ac 100644
--- a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionTask.java
@@ -41,11 +41,13 @@ public class LeveledCompactionTask extends CompactionTask
}
@Override
- public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables)
+ public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs,
+ LifecycleTransaction txn,
+ Set<SSTableReader> nonExpiredSSTables)
{
if (majorCompaction)
- return new MajorLeveledCompactionWriter(cfs, txn, nonExpiredSSTables, maxSSTableBytes, false, compactionType);
- return new MaxSSTableSizeWriter(cfs, txn, nonExpiredSSTables, maxSSTableBytes, getLevel(), false, compactionType);
+ return new MajorLeveledCompactionWriter(cfs, txn, nonExpiredSSTables, maxSSTableBytes, false);
+ return new MaxSSTableSizeWriter(cfs, txn, nonExpiredSSTables, maxSSTableBytes, getLevel(), false);
}
@Override
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/src/java/org/apache/cassandra/db/compaction/OperationType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/OperationType.java b/src/java/org/apache/cassandra/db/compaction/OperationType.java
index a14f13f..5b6ce05 100644
--- a/src/java/org/apache/cassandra/db/compaction/OperationType.java
+++ b/src/java/org/apache/cassandra/db/compaction/OperationType.java
@@ -32,13 +32,27 @@ public enum OperationType
TOMBSTONE_COMPACTION("Tombstone Compaction"),
UNKNOWN("Unknown compaction type"),
ANTICOMPACTION("Anticompaction after repair"),
- VERIFY("Verify");
+ VERIFY("Verify"),
+ FLUSH("Flush"),
+ STREAM("Stream"),
+ WRITE("Write");
- private final String type;
+ public final String type;
+ public final String fileName;
OperationType(String type)
{
this.type = type;
+ this.fileName = type.toLowerCase().replace(" ", "");
+ }
+
+ public static OperationType fromFileName(String fileName)
+ {
+ for (OperationType opType : OperationType.values())
+ if (opType.fileName.equals(fileName))
+ return opType;
+
+ throw new IllegalArgumentException("Invalid fileName for operation type: " + fileName);
}
public String toString()
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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 e9a4f05..8f382ea 100644
--- a/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java
+++ b/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java
@@ -74,9 +74,11 @@ public class SSTableSplitter {
}
@Override
- public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables)
+ public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs,
+ LifecycleTransaction txn,
+ Set<SSTableReader> nonExpiredSSTables)
{
- return new MaxSSTableSizeWriter(cfs, txn, nonExpiredSSTables, sstableSizeInMB * 1024L * 1024L, 0, true, compactionType);
+ return new MaxSSTableSizeWriter(cfs, txn, nonExpiredSSTables, sstableSizeInMB * 1024L * 1024L, 0, true);
}
@Override
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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 94f3af7..c853157 100644
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@ -59,6 +59,7 @@ public class Scrubber implements Closeable
private final RowIndexEntry.IndexSerializer rowIndexEntrySerializer;
private final boolean isOffline;
+ private final boolean keepOriginals;
private SSTableReader newSstable;
private SSTableReader newInOrderSstable;
@@ -85,11 +86,17 @@ public class Scrubber implements Closeable
public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, boolean isOffline, boolean checkData) throws IOException
{
- this(cfs, transaction, skipCorrupted, new OutputHandler.LogOutput(), isOffline, checkData);
+ this(cfs, transaction, skipCorrupted, new OutputHandler.LogOutput(), isOffline, checkData, false);
}
@SuppressWarnings("resource")
- public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, OutputHandler outputHandler, boolean isOffline, boolean checkData) throws IOException
+ public Scrubber(ColumnFamilyStore cfs,
+ LifecycleTransaction transaction,
+ boolean skipCorrupted,
+ OutputHandler outputHandler,
+ boolean isOffline,
+ boolean checkData,
+ boolean keepOriginals) throws IOException
{
this.cfs = cfs;
this.transaction = transaction;
@@ -97,6 +104,7 @@ public class Scrubber implements Closeable
this.outputHandler = outputHandler;
this.skipCorrupted = skipCorrupted;
this.isOffline = isOffline;
+ this.keepOriginals = keepOriginals;
this.rowIndexEntrySerializer = sstable.descriptor.version.getSSTableFormat().getIndexSerializer(sstable.metadata,
sstable.descriptor.version,
sstable.header);
@@ -149,7 +157,7 @@ public class Scrubber implements Closeable
{
outputHandler.output(String.format("Scrubbing %s (%s bytes)", sstable, dataFile.length()));
int nowInSec = FBUtilities.nowInSeconds();
- try (SSTableRewriter writer = new SSTableRewriter(cfs, transaction, sstable.maxDataAge, isOffline))
+ try (SSTableRewriter writer = new SSTableRewriter(cfs, transaction, sstable.maxDataAge, isOffline).keepOriginals(isOffline))
{
nextIndexKey = indexAvailable() ? ByteBufferUtil.readWithShortLength(indexFile) : null;
if (indexAvailable())
@@ -159,7 +167,7 @@ public class Scrubber implements Closeable
assert firstRowPositionFromIndex == 0 : firstRowPositionFromIndex;
}
- writer.switchWriter(CompactionManager.createWriter(cfs, destination, expectedBloomFilterSize, sstable.getSSTableMetadata().repairedAt, sstable));
+ writer.switchWriter(CompactionManager.createWriter(cfs, destination, expectedBloomFilterSize, sstable.getSSTableMetadata().repairedAt, sstable, transaction));
DecoratedKey prevKey = null;
@@ -291,7 +299,7 @@ public class Scrubber implements Closeable
{
// out of order rows, but no bad rows found - we can keep our repairedAt time
long repairedAt = badRows > 0 ? ActiveRepairService.UNREPAIRED_SSTABLE : sstable.getSSTableMetadata().repairedAt;
- try (SSTableWriter inOrderWriter = CompactionManager.createWriter(cfs, destination, expectedBloomFilterSize, repairedAt, sstable);)
+ try (SSTableWriter inOrderWriter = CompactionManager.createWriter(cfs, destination, expectedBloomFilterSize, repairedAt, sstable, transaction))
{
for (Partition partition : outOfOrder)
inOrderWriter.append(partition.unfilteredIterator());
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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 74a9757..0ece341 100644
--- a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
@@ -347,9 +347,11 @@ public class SizeTieredCompactionStrategy extends AbstractCompactionStrategy
}
@Override
- public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables)
+ public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs,
+ LifecycleTransaction txn,
+ Set<SSTableReader> nonExpiredSSTables)
{
- return new SplittingSizeTieredCompactionWriter(cfs, txn, nonExpiredSSTables, compactionType);
+ return new SplittingSizeTieredCompactionWriter(cfs, txn, nonExpiredSSTables);
}
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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 e3764c8..be0dd2a 100644
--- a/src/java/org/apache/cassandra/db/compaction/Upgrader.java
+++ b/src/java/org/apache/cassandra/db/compaction/Upgrader.java
@@ -43,7 +43,6 @@ public class Upgrader
private final LifecycleTransaction transaction;
private final File directory;
- private final OperationType compactionType = OperationType.UPGRADE_SSTABLES;
private final CompactionController controller;
private final CompactionStrategyManager strategyManager;
private final long estimatedRows;
@@ -80,23 +79,23 @@ public class Upgrader
sstableMetadataCollector.addAncestor(i);
}
sstableMetadataCollector.sstableLevel(sstable.getSSTableLevel());
- return SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(directory)),
+ return SSTableWriter.create(Descriptor.fromFilename(cfs.getSSTablePath(directory)),
estimatedRows,
repairedAt,
cfs.metadata,
cfs.partitioner,
sstableMetadataCollector,
- SerializationHeader.make(cfs.metadata, Sets.newHashSet(sstable)));
+ SerializationHeader.make(cfs.metadata, Sets.newHashSet(sstable)),
+ transaction);
}
- public void upgrade()
+ public void upgrade(boolean keepOriginals)
{
outputHandler.output("Upgrading " + sstable);
-
int nowInSec = FBUtilities.nowInSeconds();
- try (SSTableRewriter writer = new SSTableRewriter(cfs, transaction, CompactionTask.getMaxDataAge(transaction.originals()), true);
+ try (SSTableRewriter writer = new SSTableRewriter(cfs, transaction, CompactionTask.getMaxDataAge(transaction.originals()), true).keepOriginals(keepOriginals);
AbstractCompactionStrategy.ScannerList scanners = strategyManager.getScanners(transaction.originals());
- CompactionIterator iter = new CompactionIterator(compactionType, scanners.scanners, controller, nowInSec, UUIDGen.getTimeUUID()))
+ CompactionIterator iter = new CompactionIterator(transaction.opType(), scanners.scanners, controller, nowInSec, UUIDGen.getTimeUUID()))
{
writer.switchWriter(createCompactionWriter(sstable.getSSTableMetadata().repairedAt));
while (iter.hasNext())
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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 610592f..f8c73d3 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java
@@ -42,16 +42,22 @@ public abstract class CompactionAwareWriter extends Transactional.AbstractTransa
protected final long estimatedTotalKeys;
protected final long maxAge;
protected final long minRepairedAt;
+
+ protected final LifecycleTransaction txn;
protected final SSTableRewriter sstableWriter;
- public CompactionAwareWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, 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, txn, maxAge, offline);
+ this.txn = txn;
+ this.sstableWriter = new SSTableRewriter(cfs, txn, maxAge, offline).keepOriginals(offline);
}
/**
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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 8fc7bec..cdacddc 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java
@@ -27,14 +27,12 @@ import org.slf4j.LoggerFactory;
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.SerializationHeader;
import org.apache.cassandra.db.rows.UnfilteredRowIterator;
-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.format.SSTableReader;
import org.apache.cassandra.io.sstable.format.SSTableWriter;
import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
-
/**
* The default compaction writer - creates one output file in L0
*/
@@ -43,20 +41,21 @@ public class DefaultCompactionWriter extends CompactionAwareWriter
protected static final Logger logger = LoggerFactory.getLogger(DefaultCompactionWriter.class);
@SuppressWarnings("resource")
- public DefaultCompactionWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, boolean offline, OperationType compactionType)
+ public DefaultCompactionWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, boolean offline)
{
super(cfs, txn, nonExpiredSSTables, offline);
logger.debug("Expected bloom filter size : {}", estimatedTotalKeys);
- long expectedWriteSize = cfs.getExpectedCompactedFileSize(nonExpiredSSTables, compactionType);
+ long expectedWriteSize = cfs.getExpectedCompactedFileSize(nonExpiredSSTables, txn.opType());
File sstableDirectory = cfs.directories.getLocationForDisk(getWriteDirectory(expectedWriteSize));
@SuppressWarnings("resource")
- SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(sstableDirectory)),
+ SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getSSTablePath(sstableDirectory)),
estimatedTotalKeys,
minRepairedAt,
cfs.metadata,
cfs.partitioner,
new MetadataCollector(txn.originals(), cfs.metadata.comparator, 0),
- SerializationHeader.make(cfs.metadata, nonExpiredSSTables));
+ SerializationHeader.make(cfs.metadata, nonExpiredSSTables),
+ txn);
sstableWriter.switchWriter(writer);
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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 5328fa5..ad58967 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java
@@ -28,7 +28,6 @@ import org.apache.cassandra.db.RowIndexEntry;
import org.apache.cassandra.db.SerializationHeader;
import org.apache.cassandra.db.rows.UnfilteredRowIterator;
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.format.SSTableReader;
@@ -49,12 +48,12 @@ public class MajorLeveledCompactionWriter extends CompactionAwareWriter
private final boolean skipAncestors;
@SuppressWarnings("resource")
- public MajorLeveledCompactionWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, long maxSSTableSize, boolean offline, OperationType compactionType)
+ public MajorLeveledCompactionWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, long maxSSTableSize, boolean offline)
{
super(cfs, txn, nonExpiredSSTables, offline);
this.maxSSTableSize = maxSSTableSize;
this.allSSTables = txn.originals();
- expectedWriteSize = Math.min(maxSSTableSize, cfs.getExpectedCompactedFileSize(nonExpiredSSTables, compactionType));
+ expectedWriteSize = Math.min(maxSSTableSize, cfs.getExpectedCompactedFileSize(nonExpiredSSTables, txn.opType()));
long estimatedSSTables = Math.max(1, SSTableReader.getTotalBytes(nonExpiredSSTables) / maxSSTableSize);
long keysPerSSTable = estimatedTotalKeys / estimatedSSTables;
File sstableDirectory = cfs.directories.getLocationForDisk(getWriteDirectory(expectedWriteSize));
@@ -64,13 +63,14 @@ public class MajorLeveledCompactionWriter extends CompactionAwareWriter
logger.warn("Many sstables involved in compaction, skipping storing ancestor information to avoid running out of memory");
@SuppressWarnings("resource")
- SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(sstableDirectory)),
+ SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getSSTablePath(sstableDirectory)),
keysPerSSTable,
minRepairedAt,
cfs.metadata,
cfs.partitioner,
new MetadataCollector(allSSTables, cfs.metadata.comparator, currentLevel, skipAncestors),
- SerializationHeader.make(cfs.metadata, nonExpiredSSTables));
+ SerializationHeader.make(cfs.metadata, nonExpiredSSTables),
+ txn);
sstableWriter.switchWriter(writer);
}
@@ -92,13 +92,14 @@ public class MajorLeveledCompactionWriter extends CompactionAwareWriter
averageEstimatedKeysPerSSTable = Math.round(((double) averageEstimatedKeysPerSSTable * sstablesWritten + partitionsWritten) / (sstablesWritten + 1));
File sstableDirectory = cfs.directories.getLocationForDisk(getWriteDirectory(expectedWriteSize));
- SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(sstableDirectory)),
+ SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getSSTablePath(sstableDirectory)),
averageEstimatedKeysPerSSTable,
minRepairedAt,
cfs.metadata,
cfs.partitioner,
new MetadataCollector(allSSTables, cfs.metadata.comparator, currentLevel, skipAncestors),
- SerializationHeader.make(cfs.metadata, nonExpiredSSTables));
+ SerializationHeader.make(cfs.metadata, nonExpiredSSTables),
+ txn);
sstableWriter.switchWriter(writer);
partitionsWritten = 0;
sstablesWritten++;
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b09e60f7/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 4832fd5..9902357 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java
@@ -24,7 +24,6 @@ import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.RowIndexEntry;
import org.apache.cassandra.db.SerializationHeader;
import org.apache.cassandra.db.rows.UnfilteredRowIterator;
-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.format.SSTableReader;
@@ -41,25 +40,26 @@ public class MaxSSTableSizeWriter extends CompactionAwareWriter
private final Set<SSTableReader> allSSTables;
@SuppressWarnings("resource")
- public MaxSSTableSizeWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, 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)
{
super(cfs, txn, nonExpiredSSTables, offline);
this.allSSTables = txn.originals();
this.level = level;
this.maxSSTableSize = maxSSTableSize;
- long totalSize = cfs.getExpectedCompactedFileSize(nonExpiredSSTables, compactionType);
+ long totalSize = cfs.getExpectedCompactedFileSize(nonExpiredSSTables, txn.opType());
expectedWriteSize = Math.min(maxSSTableSize, totalSize);
estimatedTotalKeys = SSTableReader.getApproximateKeyCount(nonExpiredSSTables);
estimatedSSTables = Math.max(1, estimatedTotalKeys / maxSSTableSize);
File sstableDirectory = cfs.directories.getLocationForDisk(getWriteDirectory(expectedWriteSize));
@SuppressWarnings("resource")
- SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(sstableDirectory)),
+ SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getSSTablePath(sstableDirectory)),
estimatedTotalKeys / estimatedSSTables,
minRepairedAt,
cfs.metadata,
cfs.partitioner,
new MetadataCollector(allSSTables, cfs.metadata.comparator, level),
- SerializationHeader.make(cfs.metadata, nonExpiredSSTables));
+ SerializationHeader.make(cfs.metadata, nonExpiredSSTables),
+ txn);
sstableWriter.switchWriter(writer);
}
@@ -71,13 +71,14 @@ public class MaxSSTableSizeWriter extends CompactionAwareWriter
{
File sstableDirectory = cfs.directories.getLocationForDisk(getWriteDirectory(expectedWriteSize));
@SuppressWarnings("resource")
- SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(sstableDirectory)),
- estimatedTotalKeys / estimatedSSTables,
- minRepairedAt,
- cfs.metadata,
- cfs.partitioner,
- new MetadataCollector(allSSTables, cfs.metadata.comparator, level),
- SerializationHeader.make(cfs.metadata, nonExpiredSSTables));
+ SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getSSTablePath(sstableDirectory)),
+ estimatedTotalKeys / estimatedSSTables,
+ minRepairedAt,
+ cfs.metadata,
+ cfs.partitioner,
+ new MetadataCollector(allSSTables, cfs.metadata.comparator, level),
+ SerializationHeader.make(cfs.metadata, nonExpiredSSTables),
+ txn);
sstableWriter.switchWriter(writer);
}