You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by be...@apache.org on 2015/05/17 14:51:03 UTC
[1/7] cassandra git commit: Introduce Transactional API for internal
state changes
Repository: cassandra
Updated Branches:
refs/heads/cassandra-2.2 ff10d636c -> 8704006bf
refs/heads/trunk c2a1cd496 -> 75a34879b
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 3104a96..d39da61 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
@@ -52,6 +52,7 @@ import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.Pair;
import static org.junit.Assert.*;
+import static org.apache.cassandra.utils.Throwables.maybeFail;
public class SSTableRewriterTest extends SchemaLoader
{
@@ -93,8 +94,8 @@ public class SSTableRewriterTest extends SchemaLoader
cfs.forceBlockingFlush();
Set<SSTableReader> sstables = new HashSet<>(cfs.getSSTables());
assertEquals(1, sstables.size());
- SSTableRewriter writer = new SSTableRewriter(cfs, sstables, 1000, false);
- try (AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables);)
+ try (SSTableRewriter writer = new SSTableRewriter(cfs, sstables, 1000, false);
+ AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables);)
{
ISSTableScanner scanner = scanners.scanners.get(0);
CompactionController controller = new CompactionController(cfs, sstables, cfs.gcBefore(System.currentTimeMillis()));
@@ -104,9 +105,9 @@ public class SSTableRewriterTest extends SchemaLoader
AbstractCompactedRow row = new LazilyCompactedRow(controller, Arrays.asList(scanner.next()));
writer.append(row);
}
+ Collection<SSTableReader> newsstables = writer.finish();
+ cfs.getDataTracker().markCompactedSSTablesReplaced(sstables, newsstables , OperationType.COMPACTION);
}
- Collection<SSTableReader> newsstables = writer.finish();
- cfs.getDataTracker().markCompactedSSTablesReplaced(sstables, newsstables, OperationType.COMPACTION);
SSTableDeletingTask.waitForDeletions();
validateCFS(cfs);
@@ -126,8 +127,8 @@ public class SSTableRewriterTest extends SchemaLoader
Set<SSTableReader> sstables = new HashSet<>(cfs.getSSTables());
assertEquals(1, sstables.size());
SSTableRewriter.overrideOpenInterval(10000000);
- SSTableRewriter writer = new SSTableRewriter(cfs, sstables, 1000, false);
- try (AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables);)
+ try (SSTableRewriter writer = new SSTableRewriter(cfs, sstables, 1000, false);
+ AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables);)
{
ISSTableScanner scanner = scanners.scanners.get(0);
CompactionController controller = new CompactionController(cfs, sstables, cfs.gcBefore(System.currentTimeMillis()));
@@ -137,9 +138,9 @@ public class SSTableRewriterTest extends SchemaLoader
AbstractCompactedRow row = new LazilyCompactedRow(controller, Arrays.asList(scanner.next()));
writer.append(row);
}
+ Collection<SSTableReader> newsstables = writer.finish();
+ cfs.getDataTracker().markCompactedSSTablesReplaced(sstables, newsstables, OperationType.COMPACTION);
}
- Collection<SSTableReader> newsstables = writer.finish();
- cfs.getDataTracker().markCompactedSSTablesReplaced(sstables, newsstables, OperationType.COMPACTION);
SSTableDeletingTask.waitForDeletions();
validateCFS(cfs);
@@ -159,9 +160,9 @@ public class SSTableRewriterTest extends SchemaLoader
Set<SSTableReader> sstables = new HashSet<>(cfs.getSSTables());
assertEquals(1, sstables.size());
SSTableRewriter.overrideOpenInterval(10000000);
- SSTableRewriter writer = new SSTableRewriter(cfs, sstables, 1000, false);
boolean checked = false;
- try (AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables);)
+ try (SSTableRewriter writer = new SSTableRewriter(cfs, sstables, 1000, false);
+ AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables);)
{
ISSTableScanner scanner = scanners.scanners.get(0);
CompactionController controller = new CompactionController(cfs, sstables, cfs.gcBefore(System.currentTimeMillis()));
@@ -191,10 +192,10 @@ public class SSTableRewriterTest extends SchemaLoader
}
}
}
+ assertTrue(checked);
+ Collection<SSTableReader> newsstables = writer.finish();
+ cfs.getDataTracker().markCompactedSSTablesReplaced(sstables, newsstables, OperationType.COMPACTION);
}
- assertTrue(checked);
- Collection<SSTableReader> newsstables = writer.finish();
- cfs.getDataTracker().markCompactedSSTablesReplaced(sstables, newsstables, OperationType.COMPACTION);
SSTableDeletingTask.waitForDeletions();
validateCFS(cfs);
@@ -216,17 +217,17 @@ public class SSTableRewriterTest extends SchemaLoader
for (int i = 0; i < 100; i++)
cf.addColumn(Util.cellname(i), ByteBuffer.allocate(1000), 1);
File dir = cfs.directories.getDirectoryForNewSSTables();
- SSTableWriter writer = getWriter(cfs, dir);
- try
+
+ try (SSTableWriter writer = getWriter(cfs, dir);)
{
for (int i = 0; i < 10000; i++)
writer.append(StorageService.getPartitioner().decorateKey(random(i, 10)), cf);
- SSTableReader s = writer.openEarly(1000);
+ SSTableReader s = writer.setMaxDataAge(1000).openEarly();
assert s != null;
assertFileCounts(dir.list(), 2, 2);
for (int i = 10000; i < 20000; i++)
writer.append(StorageService.getPartitioner().decorateKey(random(i, 10)), cf);
- SSTableReader s2 = writer.openEarly(1000);
+ SSTableReader s2 = writer.setMaxDataAge(1000).openEarly();
assertTrue(s.last.compareTo(s2.last) < 0);
assertFileCounts(dir.list(), 2, 2);
s.markObsolete();
@@ -245,11 +246,6 @@ public class SSTableRewriterTest extends SchemaLoader
assertEquals(datafiles, 0);
validateCFS(cfs);
}
- catch (Throwable t)
- {
- writer.abort();
- throw t;
- }
}
@Test
@@ -264,13 +260,14 @@ public class SSTableRewriterTest extends SchemaLoader
long startStorageMetricsLoad = StorageMetrics.load.getCount();
Set<SSTableReader> compacting = Sets.newHashSet(s);
SSTableRewriter.overrideOpenInterval(10000000);
- SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
- rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
+ List<SSTableReader> sstables;
int files = 1;
- try (ISSTableScanner scanner = s.getScanner();
+ try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
+ ISSTableScanner scanner = s.getScanner();
CompactionController controller = new CompactionController(cfs, compacting, 0))
{
+ rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
while(scanner.hasNext())
{
rewriter.append(new LazilyCompactedRow(controller, Arrays.asList(scanner.next())));
@@ -284,9 +281,10 @@ public class SSTableRewriterTest extends SchemaLoader
}
}
+ sstables = rewriter.finish();
+ cfs.getDataTracker().markCompactedSSTablesReplaced(compacting, sstables, OperationType.COMPACTION);
+ assertEquals(files, sstables.size());
}
- List<SSTableReader> sstables = rewriter.finish();
- cfs.getDataTracker().markCompactedSSTablesReplaced(compacting, sstables, OperationType.COMPACTION);
long sum = 0;
for (SSTableReader x : cfs.getSSTables())
sum += x.bytesOnDisk();
@@ -314,13 +312,14 @@ public class SSTableRewriterTest extends SchemaLoader
Set<SSTableReader> compacting = Sets.newHashSet(s);
SSTableRewriter.overrideOpenInterval(10000000);
- SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
- rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
+ List<SSTableReader> sstables;
int files = 1;
- try (ISSTableScanner scanner = s.getScanner();
+ try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
+ ISSTableScanner scanner = s.getScanner();
CompactionController controller = new CompactionController(cfs, compacting, 0))
{
+ rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
while(scanner.hasNext())
{
rewriter.append(new LazilyCompactedRow(controller, Arrays.asList(scanner.next())));
@@ -331,14 +330,9 @@ public class SSTableRewriterTest extends SchemaLoader
assertEquals(cfs.getSSTables().size(), files); // we have one original file plus the ones we have switched out.
}
}
- }
- catch (Throwable t)
- {
- rewriter.abort();
- throw t;
+ sstables = rewriter.finish();
}
- List<SSTableReader> sstables = rewriter.finish();
assertEquals(files, sstables.size());
assertEquals(files, cfs.getSSTables().size());
assertEquals(1, cfs.getDataTracker().getView().shadowed.size());
@@ -445,19 +439,14 @@ public class SSTableRewriterTest extends SchemaLoader
long startSize = cfs.metric.liveDiskSpaceUsed.getCount();
Set<SSTableReader> compacting = Sets.newHashSet(s);
SSTableRewriter.overrideOpenInterval(10000000);
- SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
- rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
- try (ISSTableScanner scanner = s.getScanner();
+ try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
+ ISSTableScanner scanner = s.getScanner();
CompactionController controller = new CompactionController(cfs, compacting, 0))
{
+ rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
test.run(scanner, controller, s, cfs, rewriter);
}
- catch (Throwable t)
- {
- rewriter.abort();
- throw t;
- }
SSTableDeletingTask.waitForDeletions();
@@ -481,13 +470,13 @@ public class SSTableRewriterTest extends SchemaLoader
Set<SSTableReader> compacting = Sets.newHashSet(s);
SSTableRewriter.overrideOpenInterval(10000000);
- SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
- rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
int files = 1;
- try (ISSTableScanner scanner = s.getScanner();
+ try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
+ ISSTableScanner scanner = s.getScanner();
CompactionController controller = new CompactionController(cfs, compacting, 0))
{
+ rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
while(scanner.hasNext())
{
rewriter.append(new LazilyCompactedRow(controller, Arrays.asList(scanner.next())));
@@ -506,11 +495,6 @@ public class SSTableRewriterTest extends SchemaLoader
}
}
}
- catch (Throwable t)
- {
- rewriter.abort();
- throw t;
- }
SSTableDeletingTask.waitForDeletions();
@@ -531,13 +515,14 @@ public class SSTableRewriterTest extends SchemaLoader
cfs.addSSTable(s);
Set<SSTableReader> compacting = Sets.newHashSet(s);
SSTableRewriter.overrideOpenInterval(10000000);
- SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
- rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
+ List<SSTableReader> sstables;
int files = 1;
- try (ISSTableScanner scanner = s.getScanner();
+ try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
+ ISSTableScanner scanner = s.getScanner();
CompactionController controller = new CompactionController(cfs, compacting, 0))
{
+ rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
while(scanner.hasNext())
{
rewriter.append(new LazilyCompactedRow(controller, Arrays.asList(scanner.next())));
@@ -548,15 +533,11 @@ public class SSTableRewriterTest extends SchemaLoader
assertEquals(cfs.getSSTables().size(), files); // we have one original file plus the ones we have switched out.
}
}
- }
- catch (Throwable t)
- {
- rewriter.abort();
- throw t;
+
+ sstables = rewriter.finish();
+ cfs.getDataTracker().markCompactedSSTablesReplaced(compacting, sstables, OperationType.COMPACTION);
}
- List<SSTableReader> sstables = rewriter.finish();
- cfs.getDataTracker().markCompactedSSTablesReplaced(compacting, sstables, OperationType.COMPACTION);
SSTableDeletingTask.waitForDeletions();
assertFileCounts(s.descriptor.directory.list(), 0, 0);
cfs.truncateBlocking();
@@ -576,13 +557,14 @@ public class SSTableRewriterTest extends SchemaLoader
cfs.addSSTable(s);
Set<SSTableReader> compacting = Sets.newHashSet(s);
SSTableRewriter.overrideOpenInterval(1000000);
- SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
- rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
+ List<SSTableReader> sstables;
int files = 1;
- try (ISSTableScanner scanner = s.getScanner();
+ try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
+ ISSTableScanner scanner = s.getScanner();
CompactionController controller = new CompactionController(cfs, compacting, 0))
{
+ rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
while(scanner.hasNext())
{
rewriter.append(new LazilyCompactedRow(controller, Arrays.asList(scanner.next())));
@@ -593,15 +575,11 @@ public class SSTableRewriterTest extends SchemaLoader
files++;
}
}
- }
- catch (Throwable t)
- {
- rewriter.abort();
- throw t;
+
+ sstables = rewriter.finish();
+ cfs.getDataTracker().markCompactedSSTablesReplaced(compacting, sstables, OperationType.COMPACTION);
}
- List<SSTableReader> sstables = rewriter.finish();
- cfs.getDataTracker().markCompactedSSTablesReplaced(compacting, sstables, OperationType.COMPACTION);
assertEquals(files, sstables.size());
assertEquals(files, cfs.getSSTables().size());
SSTableDeletingTask.waitForDeletions();
@@ -668,12 +646,12 @@ public class SSTableRewriterTest extends SchemaLoader
Set<SSTableReader> compacting = Sets.newHashSet(s);
cfs.getDataTracker().markCompacting(compacting);
SSTableRewriter.overrideOpenInterval(10000000);
- SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, offline);
- SSTableWriter w = getWriter(cfs, s.descriptor.directory);
- rewriter.switchWriter(w);
- try (ISSTableScanner scanner = compacting.iterator().next().getScanner();
+
+ try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, offline);
+ ISSTableScanner scanner = compacting.iterator().next().getScanner();
CompactionController controller = new CompactionController(cfs, compacting, 0))
{
+ rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
while (scanner.hasNext())
{
rewriter.append(new LazilyCompactedRow(controller, Arrays.asList(scanner.next())));
@@ -684,7 +662,8 @@ public class SSTableRewriterTest extends SchemaLoader
}
try
{
- rewriter.finishAndThrow(earlyException);
+ rewriter.throwDuringPrepare(earlyException);
+ rewriter.prepareToCommit();
}
catch (Throwable t)
{
@@ -749,14 +728,14 @@ public class SSTableRewriterTest extends SchemaLoader
compacting.add(s);
cfs.getDataTracker().markCompacting(compacting);
- SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
+
SSTableRewriter.overrideOpenInterval(1);
- SSTableWriter w = getWriter(cfs, s.descriptor.directory);
- rewriter.switchWriter(w);
int keyCount = 0;
- try (ISSTableScanner scanner = compacting.iterator().next().getScanner();
+ try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
+ ISSTableScanner scanner = compacting.iterator().next().getScanner();
CompactionController controller = new CompactionController(cfs, compacting, 0))
{
+ rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
while (scanner.hasNext())
{
rewriter.append(new LazilyCompactedRow(controller, Arrays.asList(scanner.next())));
@@ -841,11 +820,12 @@ public class SSTableRewriterTest extends SchemaLoader
File dir = cfs.directories.getDirectoryForNewSSTables();
String filename = cfs.getTempSSTablePath(dir);
- SSTableWriter writer = SSTableWriter.create(filename, 0, 0);
-
- for (int i = 0; i < count * 5; i++)
- writer.append(StorageService.getPartitioner().decorateKey(ByteBufferUtil.bytes(i)), cf);
- return writer.closeAndOpenReader();
+ try (SSTableWriter writer = SSTableWriter.create(filename, 0, 0);)
+ {
+ for (int i = 0; i < count * 5; i++)
+ writer.append(StorageService.getPartitioner().decorateKey(ByteBufferUtil.bytes(i)), cf);
+ return writer.finish(true);
+ }
}
private void validateCFS(ColumnFamilyStore cfs)
@@ -899,7 +879,7 @@ public class SSTableRewriterTest extends SchemaLoader
return SSTableWriter.create(filename, 0, 0);
}
- private ByteBuffer random(int i, int size)
+ public static ByteBuffer random(int i, int size)
{
byte[] bytes = new byte[size + 4];
ThreadLocalRandom.current().nextBytes(bytes);
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 d2559cd..a116b84 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
@@ -217,7 +217,7 @@ public class SSTableUtils
File datafile = (dest == null) ? tempSSTableFile(ksname, cfname, generation) : new File(dest.filenameFor(Component.DATA));
SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(datafile.getAbsolutePath()), expectedSize, ActiveRepairService.UNREPAIRED_SSTABLE, 0);
while (appender.append(writer)) { /* pass */ }
- SSTableReader reader = writer.closeAndOpenReader();
+ SSTableReader reader = writer.finish(true);
// mark all components for removal
if (cleanup)
for (Component component : reader.components)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java b/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java
index a40034d..0ff4b01 100644
--- a/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java
+++ b/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java
@@ -128,7 +128,7 @@ public class BufferedRandomAccessFileTest
assert data[i] == 0;
}
- w.close();
+ w.finish();
r.close();
}
@@ -153,7 +153,7 @@ public class BufferedRandomAccessFileTest
assert negone == -1 : "We read past the end of the file, should have gotten EOF -1. Instead, " + negone;
r.close();
- w.close();
+ w.finish();
}
@Test
@@ -178,7 +178,7 @@ public class BufferedRandomAccessFileTest
w.write(biggerThenBuffer);
assertEquals(biggerThenBuffer.length + lessThenBuffer.length, w.length());
- w.close();
+ w.finish();
// will use cachedlength
RandomAccessReader r = RandomAccessReader.open(tmpFile);
@@ -223,7 +223,7 @@ public class BufferedRandomAccessFileTest
}
});
- w.close();
+ w.finish();
r.close();
}
@@ -233,7 +233,7 @@ public class BufferedRandomAccessFileTest
SequentialWriter w = createTempFile("brafSeek");
byte[] data = generateByteArray(RandomAccessReader.DEFAULT_BUFFER_SIZE + 20);
w.write(data);
- w.close();
+ w.finish();
final RandomAccessReader file = RandomAccessReader.open(w);
@@ -272,7 +272,7 @@ public class BufferedRandomAccessFileTest
{
SequentialWriter w = createTempFile("brafSkipBytes");
w.write(generateByteArray(RandomAccessReader.DEFAULT_BUFFER_SIZE * 2));
- w.close();
+ w.finish();
RandomAccessReader file = RandomAccessReader.open(w);
@@ -320,7 +320,7 @@ public class BufferedRandomAccessFileTest
r.read(new byte[4]);
assertEquals(r.getFilePointer(), 20);
- w.close();
+ w.finish();
r.close();
}
@@ -329,7 +329,7 @@ public class BufferedRandomAccessFileTest
{
SequentialWriter file = createTempFile("brafGetPath");
assert file.getPath().contains("brafGetPath");
- file.close();
+ file.finish();
}
@Test
@@ -411,7 +411,7 @@ public class BufferedRandomAccessFileTest
r.skipBytes(10);
assertEquals(r.bytesRemaining(), r.length() - 10);
- w.close();
+ w.finish();
r.close();
}
@@ -443,7 +443,7 @@ public class BufferedRandomAccessFileTest
byte[] data = generateByteArray(RandomAccessReader.DEFAULT_BUFFER_SIZE + 20);
w.write(data);
- w.close(); // will flush
+ w.finish();
final RandomAccessReader r = RandomAccessReader.open(new File(w.getPath()));
@@ -481,7 +481,7 @@ public class BufferedRandomAccessFileTest
SequentialWriter w = createTempFile("brafTestMark");
w.write(new byte[30]);
- w.close();
+ w.finish();
RandomAccessReader file = RandomAccessReader.open(w);
@@ -542,10 +542,10 @@ public class BufferedRandomAccessFileTest
SequentialWriter w2 = createTempFile("fscache2");
w1.write(new byte[30]);
- w1.close();
+ w1.finish();
w2.write(new byte[30]);
- w2.close();
+ w2.finish();
for (int i = 0; i < 20; i++)
{
@@ -652,7 +652,7 @@ public class BufferedRandomAccessFileTest
assertEquals(new String(content), "cccccccccc");
- file.close();
+ file.finish();
copy.close();
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/test/unit/org/apache/cassandra/io/util/ChecksummedSequentialWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/util/ChecksummedSequentialWriterTest.java b/test/unit/org/apache/cassandra/io/util/ChecksummedSequentialWriterTest.java
new file mode 100644
index 0000000..9731a8d
--- /dev/null
+++ b/test/unit/org/apache/cassandra/io/util/ChecksummedSequentialWriterTest.java
@@ -0,0 +1,92 @@
+/*
+* 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.util;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.After;
+
+import junit.framework.Assert;
+
+public class ChecksummedSequentialWriterTest extends SequentialWriterTest
+{
+
+ private final List<TestableCSW> writers = new ArrayList<>();
+
+ @After
+ public void cleanup()
+ {
+ for (TestableSW sw : writers)
+ sw.file.delete();
+ writers.clear();
+ }
+
+ protected TestableTransaction newTest() throws IOException
+ {
+ TestableCSW sw = new TestableCSW();
+ writers.add(sw);
+ return sw;
+ }
+
+ private static class TestableCSW extends TestableSW
+ {
+ final File crcFile;
+
+ private TestableCSW() throws IOException
+ {
+ this(tempFile("compressedsequentialwriter"),
+ tempFile("compressedsequentialwriter.checksum"));
+ }
+
+ private TestableCSW(File file, File crcFile) throws IOException
+ {
+ this(file, crcFile, new ChecksummedSequentialWriter(file, BUFFER_SIZE, crcFile));
+ }
+
+ private TestableCSW(File file, File crcFile, SequentialWriter sw) throws IOException
+ {
+ super(file, sw);
+ this.crcFile = crcFile;
+ }
+
+ protected void assertInProgress() throws Exception
+ {
+ super.assertInProgress();
+ Assert.assertTrue(crcFile.exists());
+ Assert.assertEquals(0, crcFile.length());
+ }
+
+ protected void assertPrepared() throws Exception
+ {
+ super.assertPrepared();
+ Assert.assertTrue(crcFile.exists());
+ Assert.assertFalse(0 == crcFile.length());
+ }
+
+ protected void assertAborted() throws Exception
+ {
+ super.assertAborted();
+ Assert.assertFalse(crcFile.exists());
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/test/unit/org/apache/cassandra/io/util/DataOutputTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/util/DataOutputTest.java b/test/unit/org/apache/cassandra/io/util/DataOutputTest.java
index b15da47..ec280fa 100644
--- a/test/unit/org/apache/cassandra/io/util/DataOutputTest.java
+++ b/test/unit/org/apache/cassandra/io/util/DataOutputTest.java
@@ -180,6 +180,7 @@ public class DataOutputTest
DataOutputStreamPlus write = new WrappedDataOutputStreamPlus(writer);
DataInput canon = testWrite(write);
write.flush();
+ writer.finish();
write.close();
DataInputStream test = new DataInputStream(new FileInputStream(file));
testRead(test, canon);
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/test/unit/org/apache/cassandra/io/util/SequentialWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/util/SequentialWriterTest.java b/test/unit/org/apache/cassandra/io/util/SequentialWriterTest.java
new file mode 100644
index 0000000..ef52030
--- /dev/null
+++ b/test/unit/org/apache/cassandra/io/util/SequentialWriterTest.java
@@ -0,0 +1,117 @@
+/*
+* 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.util;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.ThreadLocalRandom;
+
+import org.junit.After;
+
+import junit.framework.Assert;
+import org.apache.cassandra.utils.concurrent.AbstractTransactionalTest;
+
+import static org.apache.commons.io.FileUtils.*;
+
+public class SequentialWriterTest extends AbstractTransactionalTest
+{
+
+ private final List<TestableSW> writers = new ArrayList<>();
+
+ @After
+ public void cleanup()
+ {
+ for (TestableSW sw : writers)
+ sw.file.delete();
+ writers.clear();
+ }
+
+ protected TestableTransaction newTest() throws IOException
+ {
+ TestableSW sw = new TestableSW();
+ writers.add(sw);
+ return sw;
+ }
+
+ protected static class TestableSW extends TestableTransaction
+ {
+ protected static final int BUFFER_SIZE = 8 << 10;
+ protected final File file;
+ protected final SequentialWriter writer;
+ protected final byte[] fullContents, partialContents;
+
+ protected TestableSW() throws IOException
+ {
+ this(tempFile("sequentialwriter"));
+ }
+
+ protected TestableSW(File file) throws IOException
+ {
+ this(file, new SequentialWriter(file, 8 << 10, true));
+ }
+
+ protected TestableSW(File file, SequentialWriter sw) throws IOException
+ {
+ super(sw);
+ this.file = file;
+ this.writer = sw;
+ fullContents = new byte[BUFFER_SIZE + BUFFER_SIZE / 2];
+ ThreadLocalRandom.current().nextBytes(fullContents);
+ partialContents = Arrays.copyOf(fullContents, BUFFER_SIZE);
+ sw.write(fullContents);
+ }
+
+ protected void assertInProgress() throws Exception
+ {
+ Assert.assertTrue(file.exists());
+ byte[] bytes = readFileToByteArray(file);
+ Assert.assertTrue(Arrays.equals(partialContents, bytes));
+ }
+
+ protected void assertPrepared() throws Exception
+ {
+ Assert.assertTrue(file.exists());
+ byte[] bytes = readFileToByteArray(file);
+ Assert.assertTrue(Arrays.equals(fullContents, bytes));
+ }
+
+ protected void assertAborted() throws Exception
+ {
+ Assert.assertFalse(writer.isOpen());
+ Assert.assertFalse(file.exists());
+ }
+
+ protected void assertCommitted() throws Exception
+ {
+ assertPrepared();
+ Assert.assertFalse(writer.isOpen());
+ }
+
+ protected static File tempFile(String prefix)
+ {
+ File file = FileUtils.createTempFile(prefix, "test");
+ file.delete();
+ return file;
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/test/unit/org/apache/cassandra/streaming/compress/CompressedInputStreamTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/compress/CompressedInputStreamTest.java b/test/unit/org/apache/cassandra/streaming/compress/CompressedInputStreamTest.java
index 016deb3..20371c3 100644
--- a/test/unit/org/apache/cassandra/streaming/compress/CompressedInputStreamTest.java
+++ b/test/unit/org/apache/cassandra/streaming/compress/CompressedInputStreamTest.java
@@ -76,7 +76,7 @@ public class CompressedInputStreamTest
index.put(l, writer.getFilePointer());
writer.stream.writeLong(l);
}
- writer.close();
+ writer.finish();
CompressionMetadata comp = CompressionMetadata.create(tmp.getAbsolutePath());
List<Pair<Long, Long>> sections = new ArrayList<Pair<Long, Long>>();
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/test/unit/org/apache/cassandra/tools/SSTableExportTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/tools/SSTableExportTest.java b/test/unit/org/apache/cassandra/tools/SSTableExportTest.java
index 7e59207..bc73c83 100644
--- a/test/unit/org/apache/cassandra/tools/SSTableExportTest.java
+++ b/test/unit/org/apache/cassandra/tools/SSTableExportTest.java
@@ -109,7 +109,7 @@ public class SSTableExportTest
writer.append(Util.dk("rowB"), cfamily);
cfamily.clear();
- writer.closeAndOpenReader();
+ writer.finish(true);
// Enumerate and verify
File temp = File.createTempFile("Standard1", ".txt");
@@ -153,7 +153,7 @@ public class SSTableExportTest
writer.append(Util.dk("rowExclude"), cfamily);
cfamily.clear();
- SSTableReader reader = writer.closeAndOpenReader();
+ SSTableReader reader = writer.finish(true);
// Export to JSON and verify
File tempJson = File.createTempFile("Standard1", ".json");
@@ -202,7 +202,7 @@ public class SSTableExportTest
writer.append(Util.dk("rowExclude"), cfamily);
cfamily.clear();
- SSTableReader reader = writer.closeAndOpenReader();
+ SSTableReader reader = writer.finish(true);
// Export to JSON and verify
File tempJson = File.createTempFile("Standard1", ".json");
@@ -237,7 +237,7 @@ public class SSTableExportTest
writer.append(Util.dk("rowA"), cfamily);
cfamily.clear();
- SSTableReader reader = writer.closeAndOpenReader();
+ SSTableReader reader = writer.finish(true);
// Export to JSON and verify
File tempJson = File.createTempFile("Counter1", ".json");
@@ -268,7 +268,7 @@ public class SSTableExportTest
writer.append(Util.dk("rowA"), cfamily);
cfamily.clear();
- SSTableReader reader = writer.closeAndOpenReader();
+ SSTableReader reader = writer.finish(true);
// Export to JSON and verify
File tempJson = File.createTempFile("ValuesWithQuotes", ".json");
@@ -300,7 +300,7 @@ public class SSTableExportTest
cfamily.delete(new DeletionInfo(0, 0));
writer.append(Util.dk("rowA"), cfamily);
- SSTableReader reader = writer.closeAndOpenReader();
+ SSTableReader reader = writer.finish(true);
// Export to JSON and verify
File tempJson = File.createTempFile("CFWithDeletionInfo", ".json");
SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[0]);
@@ -359,7 +359,7 @@ public class SSTableExportTest
cfamily.addColumn(column(CFMetaData.DEFAULT_KEY_ALIAS, "not a uuid", 1L));
writer.append(Util.dk(ByteBufferUtil.bytes(UUIDGen.getTimeUUID())), cfamily);
- SSTableReader reader = writer.closeAndOpenReader();
+ SSTableReader reader = writer.finish(true);
// Export to JSON and verify
File tempJson = File.createTempFile("CFWithColumnNameEqualToDefaultKeyAlias", ".json");
SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[0]);
@@ -388,7 +388,7 @@ public class SSTableExportTest
cfamily.addColumn(column("column", "value", 1L));
writer.append(Util.dk("key", AsciiType.instance), cfamily);
- SSTableReader reader = writer.closeAndOpenReader();
+ SSTableReader reader = writer.finish(true);
// Export to JSON and verify
File tempJson = File.createTempFile("CFWithAsciiKeys", ".json");
SSTableExport.export(reader,
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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
new file mode 100644
index 0000000..4e160c2
--- /dev/null
+++ b/test/unit/org/apache/cassandra/utils/concurrent/AbstractTransactionalTest.java
@@ -0,0 +1,136 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+package org.apache.cassandra.utils.concurrent;
+
+import org.junit.Ignore;
+import org.junit.Test;
+
+import junit.framework.Assert;
+
+@Ignore
+public abstract class AbstractTransactionalTest
+{
+
+ protected abstract TestableTransaction newTest() throws Exception;
+
+ @Test
+ public void testNoPrepare() throws Exception
+ {
+ TestableTransaction txn;
+
+ txn = newTest();
+ txn.assertInProgress();
+ txn.testing.close();
+ txn.assertAborted();
+
+ txn = newTest();
+ txn.assertInProgress();
+ Assert.assertNull(txn.testing.abort(null));
+ txn.assertAborted();
+ }
+
+ @Test
+ public void testPrepare() throws Exception
+ {
+ TestableTransaction txn;
+ txn = newTest();
+ txn.assertInProgress();
+ txn.testing.prepareToCommit();
+ txn.assertPrepared();
+ txn.testing.close();
+ txn.assertAborted();
+
+ txn = newTest();
+ txn.assertInProgress();
+ txn.testing.prepareToCommit();
+ txn.assertPrepared();
+ Assert.assertNull(txn.testing.abort(null));
+ txn.assertAborted();
+ }
+
+ @Test
+ public void testCommit() throws Exception
+ {
+ TestableTransaction txn = newTest();
+ txn.assertInProgress();
+ txn.testing.prepareToCommit();
+ txn.assertPrepared();
+ Assert.assertNull(txn.testing.commit(null));
+ txn.assertCommitted();
+ txn.testing.close();
+ txn.assertCommitted();
+ Throwable t = txn.testing.abort(null);
+ Assert.assertTrue(t instanceof IllegalStateException);
+ txn.assertCommitted();
+ }
+
+ @Test
+ public void testThrowableReturn() throws Exception
+ {
+ TestableTransaction txn;
+ 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);
+ }
+
+ @Test
+ public void testBadCommit() throws Exception
+ {
+ TestableTransaction txn;
+ txn = newTest();
+ try
+ {
+ txn.testing.commit(null);
+ Assert.assertTrue(false);
+ }
+ catch (IllegalStateException t)
+ {
+ }
+ txn.assertInProgress();
+ Assert.assertNull(txn.testing.abort(null));
+ txn.assertAborted();
+ try
+ {
+ txn.testing.commit(null);
+ Assert.assertTrue(false);
+ }
+ catch (IllegalStateException t)
+ {
+ }
+ txn.assertAborted();
+ }
+
+
+ public static abstract class TestableTransaction
+ {
+ final Transactional testing;
+ public TestableTransaction(Transactional transactional)
+ {
+ this.testing = transactional;
+ }
+
+ protected abstract void assertInProgress() throws Exception;
+ protected abstract void assertPrepared() throws Exception;
+ protected abstract void assertAborted() throws Exception;
+ protected abstract void assertCommitted() throws Exception;
+ }
+}
[3/7] cassandra git commit: Introduce Transactional API for internal
state changes
Posted by be...@apache.org.
Introduce Transactional API for internal state changes
patch by benedict; reviewed by josh for CASSANDRA-8984
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/8704006b
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/8704006b
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/8704006b
Branch: refs/heads/cassandra-2.2
Commit: 8704006bfa75a78cb904e35662e4c8bafc1f2330
Parents: ff10d63
Author: Benedict Elliott Smith <be...@apache.org>
Authored: Sun May 17 13:50:03 2015 +0100
Committer: Benedict Elliott Smith <be...@apache.org>
Committed: Sun May 17 13:50:03 2015 +0100
----------------------------------------------------------------------
CHANGES.txt | 1 +
src/java/org/apache/cassandra/db/Memtable.java | 14 +-
.../db/compaction/CompactionManager.java | 24 +-
.../cassandra/db/compaction/CompactionTask.java | 31 +--
.../cassandra/db/compaction/Scrubber.java | 21 +-
.../cassandra/db/compaction/Upgrader.java | 11 +-
.../writers/CompactionAwareWriter.java | 42 +++-
.../writers/DefaultCompactionWriter.java | 18 +-
.../writers/MajorLeveledCompactionWriter.java | 28 +--
.../writers/MaxSSTableSizeWriter.java | 16 +-
.../SplittingSizeTieredCompactionWriter.java | 17 +-
.../io/compress/CompressedSequentialWriter.java | 59 ++---
.../io/compress/CompressionMetadata.java | 131 +++++-----
.../cassandra/io/sstable/IndexSummary.java | 11 +-
.../io/sstable/IndexSummaryBuilder.java | 13 +-
.../apache/cassandra/io/sstable/SSTable.java | 2 +-
.../cassandra/io/sstable/SSTableRewriter.java | 250 +++++++++----------
.../io/sstable/SSTableSimpleUnsortedWriter.java | 56 +++--
.../io/sstable/SSTableSimpleWriter.java | 9 +-
.../io/sstable/format/SSTableReader.java | 13 +-
.../io/sstable/format/SSTableWriter.java | 137 +++++++---
.../io/sstable/format/big/BigTableWriter.java | 248 +++++++++---------
.../io/util/BufferedPoolingSegmentedFile.java | 3 +-
.../io/util/BufferedSegmentedFile.java | 3 +-
.../io/util/ChecksummedSequentialWriter.java | 38 ++-
.../io/util/CompressedPoolingSegmentedFile.java | 4 +-
.../io/util/CompressedSegmentedFile.java | 9 +-
.../org/apache/cassandra/io/util/FileUtils.java | 39 ++-
.../cassandra/io/util/MmappedSegmentedFile.java | 3 +-
.../apache/cassandra/io/util/SafeMemory.java | 5 +
.../cassandra/io/util/SafeMemoryWriter.java | 5 +
.../apache/cassandra/io/util/SegmentedFile.java | 23 +-
.../cassandra/io/util/SequentialWriter.java | 116 ++++++---
.../cassandra/streaming/StreamReceiveTask.java | 2 +-
.../apache/cassandra/tools/SSTableImport.java | 97 +++----
.../cassandra/utils/AlwaysPresentFilter.java | 5 +
.../org/apache/cassandra/utils/Throwables.java | 5 +
.../apache/cassandra/utils/concurrent/Ref.java | 28 ++-
.../apache/cassandra/utils/concurrent/Refs.java | 11 +-
.../utils/concurrent/SharedCloseable.java | 1 +
.../utils/concurrent/SharedCloseableImpl.java | 5 +
.../utils/concurrent/Transactional.java | 198 +++++++++++++++
.../unit/org/apache/cassandra/db/ScrubTest.java | 2 +-
.../db/compaction/AntiCompactionTest.java | 11 +-
.../db/compaction/CompactionsTest.java | 31 +--
.../cassandra/io/RandomAccessReaderTest.java | 8 +-
.../CompressedRandomAccessReaderTest.java | 15 +-
.../CompressedSequentialWriterTest.java | 136 ++++++++--
.../io/sstable/BigTableWriterTest.java | 130 ++++++++++
.../io/sstable/SSTableRewriterTest.java | 154 +++++-------
.../cassandra/io/sstable/SSTableUtils.java | 2 +-
.../io/util/BufferedRandomAccessFileTest.java | 28 +--
.../util/ChecksummedSequentialWriterTest.java | 92 +++++++
.../cassandra/io/util/DataOutputTest.java | 1 +
.../cassandra/io/util/SequentialWriterTest.java | 117 +++++++++
.../compress/CompressedInputStreamTest.java | 2 +-
.../cassandra/tools/SSTableExportTest.java | 16 +-
.../concurrent/AbstractTransactionalTest.java | 136 ++++++++++
58 files changed, 1731 insertions(+), 902 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index e3b25b5..9f14fba 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
2.2.0-beta1
+ * Introduce Transactional API for internal state changes (CASSANDRA-8984)
* Add a flag in cassandra.yaml to enable UDFs (CASSANDRA-9404)
* Better support of null for UDF (CASSANDRA-8374)
* Use ecj instead of javassist for UDFs (CASSANDRA-8241)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 ef47aba..3509b27 100644
--- a/src/java/org/apache/cassandra/db/Memtable.java
+++ b/src/java/org/apache/cassandra/db/Memtable.java
@@ -27,7 +27,6 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
-import com.google.common.base.Throwables;
import org.apache.cassandra.io.sstable.Descriptor;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.sstable.format.SSTableWriter;
@@ -341,8 +340,7 @@ public class Memtable
SSTableReader ssTable;
// errors when creating the writer that may leave empty temp files.
- SSTableWriter writer = createFlushWriter(cfs.getTempSSTablePath(sstableDirectory));
- try
+ try (SSTableWriter writer = createFlushWriter(cfs.getTempSSTablePath(sstableDirectory)))
{
boolean trackContention = logger.isDebugEnabled();
int heavilyContendedRowCount = 0;
@@ -372,16 +370,13 @@ public class Memtable
if (writer.getFilePointer() > 0)
{
- writer.isolateReferences();
-
// temp sstables should contain non-repaired data.
- ssTable = writer.closeAndOpenReader();
+ ssTable = writer.finish(true);
logger.info(String.format("Completed flushing %s (%d bytes) for commitlog position %s",
ssTable.getFilename(), new File(ssTable.getFilename()).length(), context));
}
else
{
- writer.abort();
ssTable = null;
logger.info("Completed flushing; nothing needed to be retained. Commitlog position was {}",
context);
@@ -392,11 +387,6 @@ public class Memtable
return ssTable;
}
- catch (Throwable e)
- {
- writer.abort();
- throw Throwables.propagate(e);
- }
}
public SSTableWriter createFlushWriter(String filename)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 35e288d..fc83cc5 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -21,7 +21,6 @@ import java.io.File;
import java.io.IOException;
import java.lang.management.ManagementFactory;
import java.util.ArrayList;
-import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
@@ -42,7 +41,6 @@ import javax.management.openmbean.OpenDataException;
import javax.management.openmbean.TabularData;
import com.google.common.base.Predicate;
-import com.google.common.base.Throwables;
import com.google.common.collect.*;
import com.google.common.util.concurrent.*;
import org.slf4j.Logger;
@@ -787,9 +785,9 @@ public class CompactionManager implements CompactionManagerMBean
metrics.beginCompaction(ci);
Set<SSTableReader> oldSSTable = Sets.newHashSet(sstable);
- SSTableRewriter writer = new SSTableRewriter(cfs, oldSSTable, sstable.maxDataAge, false);
List<SSTableReader> finished;
- try (CompactionController controller = new CompactionController(cfs, sstableSet, getDefaultGcBefore(cfs)))
+ try (SSTableRewriter writer = new SSTableRewriter(cfs, oldSSTable, sstable.maxDataAge, false);
+ CompactionController controller = new CompactionController(cfs, sstableSet, getDefaultGcBefore(cfs)))
{
writer.switchWriter(createWriter(cfs, compactionFileLocation, expectedBloomFilterSize, sstable.getSSTableMetadata().repairedAt, sstable));
@@ -813,11 +811,6 @@ public class CompactionManager implements CompactionManagerMBean
finished = writer.finish();
cfs.getDataTracker().markCompactedSSTablesReplaced(oldSSTable, finished, OperationType.CLEANUP);
}
- catch (Throwable e)
- {
- writer.abort();
- throw Throwables.propagate(e);
- }
finally
{
scanner.close();
@@ -1178,13 +1171,12 @@ public class CompactionManager implements CompactionManagerMBean
Set<SSTableReader> sstableAsSet = new HashSet<>(anticompactionGroup);
File destination = cfs.directories.getWriteableLocationAsFile(cfs.getExpectedCompactedFileSize(sstableAsSet, OperationType.ANTICOMPACTION));
- SSTableRewriter repairedSSTableWriter = new SSTableRewriter(cfs, sstableAsSet, groupMaxDataAge, false);
- SSTableRewriter unRepairedSSTableWriter = new SSTableRewriter(cfs, sstableAsSet, groupMaxDataAge, false);
-
long repairedKeyCount = 0;
long unrepairedKeyCount = 0;
AbstractCompactionStrategy strategy = cfs.getCompactionStrategy();
- try (AbstractCompactionStrategy.ScannerList scanners = strategy.getScanners(anticompactionGroup);
+ try (SSTableRewriter repairedSSTableWriter = new SSTableRewriter(cfs, sstableAsSet, groupMaxDataAge, false);
+ SSTableRewriter unRepairedSSTableWriter = new SSTableRewriter(cfs, sstableAsSet, groupMaxDataAge, false);
+ AbstractCompactionStrategy.ScannerList scanners = strategy.getScanners(anticompactionGroup);
CompactionController controller = new CompactionController(cfs, sstableAsSet, CFMetaData.DEFAULT_GC_GRACE_SECONDS))
{
int expectedBloomFilterSize = Math.max(cfs.metadata.getMinIndexInterval(), (int)(SSTableReader.getApproximateKeyCount(anticompactionGroup)));
@@ -1221,8 +1213,8 @@ public class CompactionManager implements CompactionManagerMBean
// we have the same readers being rewritten by both writers, so we ask the first one NOT to close them
// so that the second one can do so safely, without leaving us with references < 0 or any other ugliness
List<SSTableReader> anticompactedSSTables = new ArrayList<>();
- anticompactedSSTables.addAll(repairedSSTableWriter.finish(repairedAt));
- anticompactedSSTables.addAll(unRepairedSSTableWriter.finish(ActiveRepairService.UNREPAIRED_SSTABLE));
+ anticompactedSSTables.addAll(repairedSSTableWriter.setRepairedAt(repairedAt).finish());
+ anticompactedSSTables.addAll(unRepairedSSTableWriter.setRepairedAt(ActiveRepairService.UNREPAIRED_SSTABLE).finish());
cfs.getDataTracker().markCompactedSSTablesReplaced(sstableAsSet, anticompactedSSTables, OperationType.ANTICOMPACTION);
logger.debug("Repaired {} keys out of {} for {}/{} in {}", repairedKeyCount,
@@ -1236,8 +1228,6 @@ public class CompactionManager implements CompactionManagerMBean
{
JVMStabilityInspector.inspectThrowable(e);
logger.error("Error anticompacting " + anticompactionGroup, e);
- repairedSSTableWriter.abort();
- unRepairedSSTableWriter.abort();
}
return 0;
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 f472711..c397d9a 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@ -167,20 +167,12 @@ public class CompactionTask extends AbstractCompactionTask
if (collector != null)
collector.beginCompaction(ci);
long lastCheckObsoletion = start;
- CompactionAwareWriter writer = null;
- try
+
+ if (!controller.cfs.getCompactionStrategy().isActive)
+ throw new CompactionInterruptedException(ci.getCompactionInfo());
+
+ try (CompactionAwareWriter writer = getCompactionAwareWriter(cfs, sstables, actuallyCompact))
{
- if (!controller.cfs.getCompactionStrategy().isActive)
- throw new CompactionInterruptedException(ci.getCompactionInfo());
- if (!iter.hasNext())
- {
- // don't mark compacted in the finally block, since if there _is_ nondeleted data,
- // we need to sync it (via closeAndOpen) first, so there is no period during which
- // a crash could cause data loss.
- cfs.markObsolete(sstables, compactionType);
- return;
- }
- writer = getCompactionAwareWriter(cfs, sstables, actuallyCompact);
estimatedKeys = writer.estimatedKeys();
while (iter.hasNext())
{
@@ -201,19 +193,6 @@ public class CompactionTask extends AbstractCompactionTask
// don't replace old sstables yet, as we need to mark the compaction finished in the system table
newSStables = writer.finish();
}
- catch (Throwable t)
- {
- try
- {
- if (writer != null)
- writer.abort();
- }
- catch (Throwable t2)
- {
- t.addSuppressed(t2);
- }
- throw t;
- }
finally
{
// point of no return -- the new sstables are live on disk; next we'll start deleting the old ones
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 691566e..29472b3 100644
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@ -127,8 +127,8 @@ public class Scrubber implements Closeable
{
outputHandler.output(String.format("Scrubbing %s (%s bytes)", sstable, dataFile.length()));
Set<SSTableReader> oldSSTable = Sets.newHashSet(sstable);
- SSTableRewriter writer = new SSTableRewriter(cfs, oldSSTable, sstable.maxDataAge, isOffline);
- try
+
+ try (SSTableRewriter writer = new SSTableRewriter(cfs, oldSSTable, sstable.maxDataAge, isOffline);)
{
nextIndexKey = ByteBufferUtil.readWithShortLength(indexFile);
{
@@ -271,26 +271,27 @@ 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;
- SSTableWriter inOrderWriter = CompactionManager.createWriter(cfs, destination, expectedBloomFilterSize, repairedAt, sstable);
- for (Row row : outOfOrderRows)
- inOrderWriter.append(row.key, row.cf);
- newInOrderSstable = inOrderWriter.closeAndOpenReader(sstable.maxDataAge);
+ try (SSTableWriter inOrderWriter = CompactionManager.createWriter(cfs, destination, expectedBloomFilterSize, repairedAt, sstable);)
+ {
+ for (Row row : outOfOrderRows)
+ inOrderWriter.append(row.key, row.cf);
+ newInOrderSstable = inOrderWriter.finish(-1, sstable.maxDataAge, true);
+ }
if (!isOffline)
cfs.getDataTracker().addSSTables(Collections.singleton(newInOrderSstable));
outputHandler.warn(String.format("%d out of order rows found while scrubbing %s; Those have been written (in order) to a new sstable (%s)", outOfOrderRows.size(), sstable, newInOrderSstable));
}
// finish obsoletes the old sstable
- List<SSTableReader> finished = writer.finish(badRows > 0 ? ActiveRepairService.UNREPAIRED_SSTABLE : sstable.getSSTableMetadata().repairedAt);
+ List<SSTableReader> finished = writer.setRepairedAt(badRows > 0 ? ActiveRepairService.UNREPAIRED_SSTABLE : sstable.getSSTableMetadata().repairedAt).finish();
if (!finished.isEmpty())
newSstable = finished.get(0);
if (!isOffline)
cfs.getDataTracker().markCompactedSSTablesReplaced(oldSSTable, finished, OperationType.SCRUB);
}
- catch (Throwable t)
+ catch (IOException e)
{
- writer.abort();
- throw Throwables.propagate(t);
+ throw Throwables.propagate(e);
}
finally
{
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 75964e1..30584fd 100644
--- a/src/java/org/apache/cassandra/db/compaction/Upgrader.java
+++ b/src/java/org/apache/cassandra/db/compaction/Upgrader.java
@@ -81,8 +81,9 @@ public class Upgrader
{
outputHandler.output("Upgrading " + sstable);
Set<SSTableReader> toUpgrade = Sets.newHashSet(sstable);
- SSTableRewriter writer = new SSTableRewriter(cfs, toUpgrade, CompactionTask.getMaxDataAge(toUpgrade), true);
- try (AbstractCompactionStrategy.ScannerList scanners = strategy.getScanners(toUpgrade))
+
+ try (SSTableRewriter writer = new SSTableRewriter(cfs, toUpgrade, CompactionTask.getMaxDataAge(toUpgrade), true);
+ AbstractCompactionStrategy.ScannerList scanners = strategy.getScanners(toUpgrade))
{
Iterator<AbstractCompactedRow> iter = new CompactionIterable(compactionType, scanners.scanners, controller, DatabaseDescriptor.getSSTableFormat()).iterator();
writer.switchWriter(createCompactionWriter(sstable.getSSTableMetadata().repairedAt));
@@ -94,12 +95,6 @@ public class Upgrader
writer.finish();
outputHandler.output("Upgrade of " + sstable + " complete.");
-
- }
- catch (Throwable t)
- {
- writer.abort();
- throw Throwables.propagate(t);
}
finally
{
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 2903ced..fe43186 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java
@@ -25,28 +25,32 @@ import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.Directories;
import org.apache.cassandra.db.compaction.AbstractCompactedRow;
import org.apache.cassandra.db.compaction.CompactionTask;
+import org.apache.cassandra.io.sstable.SSTableRewriter;
import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.utils.concurrent.Transactional;
/**
* Class that abstracts away the actual writing of files to make it possible to use CompactionTask for more
* use cases.
*/
-public abstract class CompactionAwareWriter
+public abstract class CompactionAwareWriter extends Transactional.AbstractTransactional implements Transactional
{
protected final ColumnFamilyStore cfs;
protected final Set<SSTableReader> nonExpiredSSTables;
protected final long estimatedTotalKeys;
protected final long maxAge;
protected final long minRepairedAt;
+ protected final SSTableRewriter sstableWriter;
- public CompactionAwareWriter(ColumnFamilyStore cfs, Set<SSTableReader> nonExpiredSSTables)
+ public CompactionAwareWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables, boolean offline)
{
this.cfs = cfs;
this.nonExpiredSSTables = nonExpiredSSTables;
this.estimatedTotalKeys = SSTableReader.getApproximateKeyCount(nonExpiredSSTables);
this.maxAge = CompactionTask.getMaxDataAge(nonExpiredSSTables);
this.minRepairedAt = CompactionTask.getMinRepairedAt(nonExpiredSSTables);
+ this.sstableWriter = new SSTableRewriter(cfs, allSSTables, maxAge, offline);
}
/**
@@ -56,16 +60,40 @@ public abstract class CompactionAwareWriter
*/
public abstract boolean append(AbstractCompactedRow row);
- /**
- * abort the compaction writer - make sure that all files created are removed etc
- */
- public abstract void abort();
+ @Override
+ protected Throwable doAbort(Throwable accumulate)
+ {
+ return sstableWriter.abort(accumulate);
+ }
+
+ @Override
+ protected Throwable doCleanup(Throwable accumulate)
+ {
+ return accumulate;
+ }
+
+ @Override
+ protected Throwable doCommit(Throwable accumulate)
+ {
+ return sstableWriter.commit(accumulate);
+ }
+
+ @Override
+ protected void doPrepare()
+ {
+ sstableWriter.prepareToCommit();
+ }
/**
* we are done, return the finished sstables so that the caller can mark the old ones as compacted
* @return all the written sstables sstables
*/
- public abstract List<SSTableReader> finish();
+ @Override
+ public List<SSTableReader> finish()
+ {
+ super.finish();
+ return sstableWriter.finished();
+ }
/**
* estimated number of keys we should write
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 d51c82d..3589b54 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java
@@ -34,6 +34,8 @@ import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.sstable.format.SSTableWriter;
import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+import static org.apache.cassandra.utils.Throwables.maybeFail;
+
/**
* The default compaction writer - creates one output file in L0
@@ -41,13 +43,11 @@ import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
public class DefaultCompactionWriter extends CompactionAwareWriter
{
protected static final Logger logger = LoggerFactory.getLogger(DefaultCompactionWriter.class);
- private final SSTableRewriter sstableWriter;
public DefaultCompactionWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables, boolean offline, OperationType compactionType)
{
- super(cfs, nonExpiredSSTables);
+ super(cfs, allSSTables, nonExpiredSSTables, offline);
logger.debug("Expected bloom filter size : {}", estimatedTotalKeys);
- sstableWriter = new SSTableRewriter(cfs, allSSTables, maxAge, offline);
long expectedWriteSize = cfs.getExpectedCompactedFileSize(nonExpiredSSTables, compactionType);
File sstableDirectory = cfs.directories.getLocationForDisk(getWriteDirectory(expectedWriteSize));
SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(sstableDirectory)),
@@ -66,18 +66,6 @@ public class DefaultCompactionWriter extends CompactionAwareWriter
}
@Override
- public void abort()
- {
- sstableWriter.abort();
- }
-
- @Override
- public List<SSTableReader> finish()
- {
- return sstableWriter.finish();
- }
-
- @Override
public long estimatedKeys()
{
return estimatedTotalKeys;
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 b2f8fe1..d48140e 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java
@@ -40,7 +40,6 @@ import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
public class MajorLeveledCompactionWriter extends CompactionAwareWriter
{
private static final Logger logger = LoggerFactory.getLogger(MajorLeveledCompactionWriter.class);
- private final SSTableRewriter rewriter;
private final long maxSSTableSize;
private final long expectedWriteSize;
private final Set<SSTableReader> allSSTables;
@@ -53,10 +52,9 @@ public class MajorLeveledCompactionWriter extends CompactionAwareWriter
public MajorLeveledCompactionWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables, long maxSSTableSize, boolean offline, OperationType compactionType)
{
- super(cfs, nonExpiredSSTables);
+ super(cfs, allSSTables, nonExpiredSSTables, offline);
this.maxSSTableSize = maxSSTableSize;
this.allSSTables = allSSTables;
- rewriter = new SSTableRewriter(cfs, allSSTables, CompactionTask.getMaxDataAge(nonExpiredSSTables), offline);
expectedWriteSize = Math.min(maxSSTableSize, cfs.getExpectedCompactedFileSize(nonExpiredSSTables, compactionType));
long estimatedSSTables = Math.max(1, SSTableReader.getTotalBytes(nonExpiredSSTables) / maxSSTableSize);
long keysPerSSTable = estimatedTotalKeys / estimatedSSTables;
@@ -72,17 +70,17 @@ public class MajorLeveledCompactionWriter extends CompactionAwareWriter
cfs.metadata,
cfs.partitioner,
new MetadataCollector(allSSTables, cfs.metadata.comparator, currentLevel, skipAncestors));
- rewriter.switchWriter(writer);
+ sstableWriter.switchWriter(writer);
}
@Override
public boolean append(AbstractCompactedRow row)
{
- long posBefore = rewriter.currentWriter().getOnDiskFilePointer();
- RowIndexEntry rie = rewriter.append(row);
- totalWrittenInLevel += rewriter.currentWriter().getOnDiskFilePointer() - posBefore;
+ long posBefore = sstableWriter.currentWriter().getOnDiskFilePointer();
+ RowIndexEntry rie = sstableWriter.append(row);
+ totalWrittenInLevel += sstableWriter.currentWriter().getOnDiskFilePointer() - posBefore;
partitionsWritten++;
- if (rewriter.currentWriter().getOnDiskFilePointer() > maxSSTableSize)
+ if (sstableWriter.currentWriter().getOnDiskFilePointer() > maxSSTableSize)
{
if (totalWrittenInLevel > LeveledManifest.maxBytesForLevel(currentLevel, maxSSTableSize))
{
@@ -98,23 +96,11 @@ public class MajorLeveledCompactionWriter extends CompactionAwareWriter
cfs.metadata,
cfs.partitioner,
new MetadataCollector(allSSTables, cfs.metadata.comparator, currentLevel, skipAncestors));
- rewriter.switchWriter(writer);
+ sstableWriter.switchWriter(writer);
partitionsWritten = 0;
sstablesWritten++;
}
return rie != null;
}
-
- @Override
- public void abort()
- {
- rewriter.abort();
- }
-
- @Override
- public List<SSTableReader> finish()
- {
- return rewriter.finish();
- }
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 1a99059..ab24bf8 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java
@@ -36,7 +36,6 @@ public class MaxSSTableSizeWriter extends CompactionAwareWriter
{
private final long estimatedTotalKeys;
private final long expectedWriteSize;
- private final SSTableRewriter sstableWriter;
private final long maxSSTableSize;
private final int level;
private final long estimatedSSTables;
@@ -44,7 +43,7 @@ public class MaxSSTableSizeWriter extends CompactionAwareWriter
public MaxSSTableSizeWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables, long maxSSTableSize, int level, boolean offline, OperationType compactionType)
{
- super(cfs, nonExpiredSSTables);
+ super(cfs, allSSTables, nonExpiredSSTables, offline);
this.allSSTables = allSSTables;
this.level = level;
this.maxSSTableSize = maxSSTableSize;
@@ -52,7 +51,6 @@ public class MaxSSTableSizeWriter extends CompactionAwareWriter
expectedWriteSize = Math.min(maxSSTableSize, totalSize);
estimatedTotalKeys = SSTableReader.getApproximateKeyCount(nonExpiredSSTables);
estimatedSSTables = Math.max(1, estimatedTotalKeys / maxSSTableSize);
- sstableWriter = new SSTableRewriter(cfs, allSSTables, CompactionTask.getMaxDataAge(nonExpiredSSTables), offline);
File sstableDirectory = cfs.directories.getLocationForDisk(getWriteDirectory(expectedWriteSize));
SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(sstableDirectory)),
estimatedTotalKeys / estimatedSSTables,
@@ -83,18 +81,6 @@ public class MaxSSTableSizeWriter extends CompactionAwareWriter
}
@Override
- public void abort()
- {
- sstableWriter.abort();
- }
-
- @Override
- public List<SSTableReader> finish()
- {
- return sstableWriter.finish();
- }
-
- @Override
public long estimatedKeys()
{
return estimatedTotalKeys;
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 c97270c..2a452c7 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
@@ -48,7 +48,6 @@ public class SplittingSizeTieredCompactionWriter extends CompactionAwareWriter
public static final long DEFAULT_SMALLEST_SSTABLE_BYTES = 50_000_000;
private final double[] ratios;
- private final SSTableRewriter sstableWriter;
private final long totalSize;
private final Set<SSTableReader> allSSTables;
private long currentBytesToWrite;
@@ -61,7 +60,7 @@ public class SplittingSizeTieredCompactionWriter extends CompactionAwareWriter
public SplittingSizeTieredCompactionWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables, OperationType compactionType, long smallestSSTable)
{
- super(cfs, nonExpiredSSTables);
+ super(cfs, allSSTables, nonExpiredSSTables, false);
this.allSSTables = allSSTables;
totalSize = cfs.getExpectedCompactedFileSize(nonExpiredSSTables, compactionType);
double[] potentialRatios = new double[20];
@@ -83,7 +82,6 @@ public class SplittingSizeTieredCompactionWriter extends CompactionAwareWriter
}
}
ratios = Arrays.copyOfRange(potentialRatios, 0, noPointIndex);
- sstableWriter = new SSTableRewriter(cfs, allSSTables, CompactionTask.getMaxDataAge(nonExpiredSSTables), false);
File sstableDirectory = cfs.directories.getLocationForDisk(getWriteDirectory(Math.round(totalSize * ratios[currentRatioIndex])));
long currentPartitionsToWrite = Math.round(estimatedTotalKeys * ratios[currentRatioIndex]);
currentBytesToWrite = Math.round(totalSize * ratios[currentRatioIndex]);
@@ -119,17 +117,4 @@ public class SplittingSizeTieredCompactionWriter extends CompactionAwareWriter
}
return rie != null;
}
-
-
- @Override
- public void abort()
- {
- sstableWriter.abort();
- }
-
- @Override
- public List<SSTableReader> finish()
- {
- return sstableWriter.finish();
- }
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
index eb9dcf8..6218526 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
@@ -28,17 +28,12 @@ import java.util.zip.Adler32;
import org.apache.cassandra.io.FSReadError;
import org.apache.cassandra.io.FSWriteError;
import org.apache.cassandra.io.sstable.CorruptSSTableException;
-import org.apache.cassandra.io.sstable.Descriptor;
import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
import org.apache.cassandra.io.util.DataIntegrityMetadata;
import org.apache.cassandra.io.util.FileMark;
import org.apache.cassandra.io.util.SequentialWriter;
import org.apache.cassandra.utils.FBUtilities;
-import static org.apache.cassandra.io.compress.CompressionMetadata.Writer.OpenType.FINAL;
-import static org.apache.cassandra.io.compress.CompressionMetadata.Writer.OpenType.SHARED;
-import static org.apache.cassandra.io.compress.CompressionMetadata.Writer.OpenType.SHARED_FINAL;
-
public class CompressedSequentialWriter extends SequentialWriter
{
private final DataIntegrityMetadata.ChecksumWriter crcMetadata;
@@ -97,12 +92,6 @@ public class CompressedSequentialWriter extends SequentialWriter
}
@Override
- public void sync()
- {
- throw new UnsupportedOperationException();
- }
-
- @Override
public void flush()
{
throw new UnsupportedOperationException();
@@ -163,13 +152,11 @@ public class CompressedSequentialWriter extends SequentialWriter
runPostFlush.run();
}
- public CompressionMetadata open(long overrideLength, boolean isFinal)
+ public CompressionMetadata open(long overrideLength)
{
if (overrideLength <= 0)
- return metadataWriter.open(uncompressedSize, chunkOffset, isFinal ? FINAL : SHARED_FINAL);
- // we are early opening the file, make sure we open metadata with the correct size
- assert !isFinal;
- return metadataWriter.open(overrideLength, chunkOffset, SHARED);
+ overrideLength = uncompressedSize;
+ return metadataWriter.open(overrideLength, chunkOffset);
}
@Override
@@ -279,36 +266,36 @@ public class CompressedSequentialWriter extends SequentialWriter
}
}
- @Override
- public void close()
+ protected class TransactionalProxy extends SequentialWriter.TransactionalProxy
{
- if (buffer == null)
- return;
-
- long finalPosition = current();
-
- super.close();
- sstableMetadataCollector.addCompressionRatio(compressedSize, uncompressedSize);
- try
+ @Override
+ protected Throwable doCommit(Throwable accumulate)
{
- metadataWriter.close(finalPosition, chunkCount);
+ return metadataWriter.commit(accumulate);
}
- catch (IOException e)
+
+ @Override
+ protected Throwable doAbort(Throwable accumulate)
{
- throw new FSWriteError(e, getPath());
+ return super.doAbort(metadataWriter.abort(accumulate));
}
- }
- public void abort()
- {
- super.abort();
- metadataWriter.abort();
+ @Override
+ protected void doPrepare()
+ {
+ syncInternal();
+ if (descriptor != null)
+ crcMetadata.writeFullChecksum(descriptor);
+ releaseFileHandle();
+ sstableMetadataCollector.addCompressionRatio(compressedSize, uncompressedSize);
+ metadataWriter.finalizeLength(current(), chunkCount).prepareToCommit();
+ }
}
@Override
- public void writeFullChecksum(Descriptor descriptor)
+ protected SequentialWriter.TransactionalProxy txnProxy()
{
- crcMetadata.writeFullChecksum(descriptor);
+ return new TransactionalProxy();
}
/**
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java b/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
index 928541a..a6c7a8b 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
@@ -36,9 +36,9 @@ import java.util.SortedSet;
import java.util.TreeSet;
import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Throwables;
import com.google.common.primitives.Longs;
-import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.TypeSizes;
import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.io.FSReadError;
@@ -47,12 +47,12 @@ import org.apache.cassandra.io.IVersionedSerializer;
import org.apache.cassandra.io.sstable.Component;
import org.apache.cassandra.io.sstable.CorruptSSTableException;
import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.format.Version;
import org.apache.cassandra.io.util.DataOutputPlus;
import org.apache.cassandra.io.util.FileUtils;
import org.apache.cassandra.io.util.Memory;
import org.apache.cassandra.io.util.SafeMemory;
import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.concurrent.Transactional;
/**
* Holds metadata about compressed file
@@ -265,7 +265,7 @@ public class CompressionMetadata
chunkOffsets.close();
}
- public static class Writer
+ public static class Writer extends Transactional.AbstractTransactional implements Transactional
{
// path to the file
private final CompressionParameters parameters;
@@ -274,6 +274,8 @@ public class CompressionMetadata
private SafeMemory offsets = new SafeMemory(maxCount * 8L);
private int count = 0;
+ // provided by user when setDescriptor
+ private long dataLength, chunkCount;
private Writer(CompressionParameters parameters, String path)
{
@@ -321,61 +323,60 @@ public class CompressionMetadata
}
}
- static enum OpenType
+ // we've written everything; wire up some final metadata state
+ public Writer finalizeLength(long dataLength, int chunkCount)
{
- // i.e. FinishType == EARLY; we will use the RefCountedMemory in possibly multiple instances
- SHARED,
- // i.e. FinishType == EARLY, but the sstable has been completely written, so we can
- // finalise the contents and size of the memory, but must retain a reference to it
- SHARED_FINAL,
- // i.e. FinishType == NORMAL or FINISH_EARLY, i.e. we have actually finished writing the table
- // and will never need to open the metadata again, so we can release any references to it here
- FINAL
+ this.dataLength = dataLength;
+ this.chunkCount = chunkCount;
+ return this;
}
- public CompressionMetadata open(long dataLength, long compressedLength, OpenType type)
+ public void doPrepare()
{
- SafeMemory offsets;
- int count = this.count;
- switch (type)
+ assert chunkCount == count;
+
+ // finalize the size of memory used if it won't now change;
+ // unnecessary if already correct size
+ if (offsets.size() != count * 8L)
+ {
+ SafeMemory tmp = offsets;
+ offsets = offsets.copy(count * 8L);
+ tmp.free();
+ }
+
+ // flush the data to disk
+ DataOutputStream out = null;
+ try
+ {
+ out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(filePath)));
+ writeHeader(out, dataLength, count);
+ for (int i = 0 ; i < count ; i++)
+ out.writeLong(offsets.getLong(i * 8L));
+ }
+ catch (IOException e)
{
- case FINAL: case SHARED_FINAL:
- if (this.offsets.size() != count * 8L)
- {
- // finalize the size of memory used if it won't now change;
- // unnecessary if already correct size
- SafeMemory tmp = this.offsets.copy(count * 8L);
- this.offsets.free();
- this.offsets = tmp;
- }
-
- if (type == OpenType.SHARED_FINAL)
- {
- offsets = this.offsets.sharedCopy();
- }
- else
- {
- offsets = this.offsets;
- // null out our reference to the original shared data to catch accidental reuse
- // note that since noone is writing to this Writer while we open it, null:ing out this.offsets is safe
- this.offsets = null;
- }
- break;
-
- case SHARED:
- offsets = this.offsets.sharedCopy();
- // we should only be opened on a compression data boundary; truncate our size to this boundary
- count = (int) (dataLength / parameters.chunkLength());
- if (dataLength % parameters.chunkLength() != 0)
- count++;
- // grab our actual compressed length from the next offset from our the position we're opened to
- if (count < this.count)
- compressedLength = offsets.getLong(count * 8L);
- break;
-
- default:
- throw new AssertionError();
+ throw Throwables.propagate(e);
}
+ finally
+ {
+ FileUtils.closeQuietly(out);
+ }
+ }
+
+ public CompressionMetadata open(long dataLength, long compressedLength)
+ {
+ SafeMemory offsets = this.offsets.sharedCopy();
+
+ // calculate how many entries we need, if our dataLength is truncated
+ int count = (int) (dataLength / parameters.chunkLength());
+ if (dataLength % parameters.chunkLength() != 0)
+ count++;
+
+ assert count > 0;
+ // grab our actual compressed length from the next offset from our the position we're opened to
+ if (count < this.count)
+ compressedLength = offsets.getLong(count * 8L);
+
return new CompressionMetadata(filePath, parameters, offsets, count * 8L, dataLength, compressedLength);
}
@@ -402,27 +403,19 @@ public class CompressionMetadata
count = chunkIndex;
}
- public void close(long dataLength, int chunks) throws IOException
+ protected Throwable doCleanup(Throwable failed)
{
- DataOutputStream out = null;
- try
- {
- out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(filePath)));
- assert chunks == count;
- writeHeader(out, dataLength, chunks);
- for (int i = 0 ; i < count ; i++)
- out.writeLong(offsets.getLong(i * 8L));
- }
- finally
- {
- FileUtils.closeQuietly(out);
- }
+ return offsets.close(failed);
}
- public void abort()
+ protected Throwable doCommit(Throwable accumulate)
{
- if (offsets != null)
- offsets.close();
+ return accumulate;
+ }
+
+ protected Throwable doAbort(Throwable accumulate)
+ {
+ return FileUtils.deleteWithConfirm(filePath, false, accumulate);
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/io/sstable/IndexSummary.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummary.java b/src/java/org/apache/cassandra/io/sstable/IndexSummary.java
index fbefe13..59c5eef 100644
--- a/src/java/org/apache/cassandra/io/sstable/IndexSummary.java
+++ b/src/java/org/apache/cassandra/io/sstable/IndexSummary.java
@@ -18,16 +18,20 @@
package org.apache.cassandra.io.sstable;
import java.io.DataInputStream;
+import java.io.File;
+import java.io.FileOutputStream;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
import org.apache.cassandra.db.DecoratedKey;
import org.apache.cassandra.db.RowPosition;
import org.apache.cassandra.dht.IPartitioner;
-import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.io.util.Memory;
-import org.apache.cassandra.io.util.MemoryOutputStream;
+import org.apache.cassandra.io.util.*;
+import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.concurrent.WrappedSharedCloseable;
import org.apache.cassandra.utils.memory.MemoryUtil;
@@ -46,6 +50,7 @@ import static org.apache.cassandra.io.sstable.Downsampling.BASE_SAMPLING_LEVEL;
*/
public class IndexSummary extends WrappedSharedCloseable
{
+ private static final Logger logger = LoggerFactory.getLogger(IndexSummary.class);
public static final IndexSummarySerializer serializer = new IndexSummarySerializer();
/**
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java b/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java
index c7c51e5..12e41c8 100644
--- a/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java
+++ b/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java
@@ -203,12 +203,16 @@ public class IndexSummaryBuilder implements AutoCloseable
}
}
- public IndexSummary build(IPartitioner partitioner)
+ public void prepareToCommit()
{
// this method should only be called when we've finished appending records, so we truncate the
// memory we're using to the exact amount required to represent it before building our summary
entries.setCapacity(entries.length());
offsets.setCapacity(offsets.length());
+ }
+
+ public IndexSummary build(IPartitioner partitioner)
+ {
return build(partitioner, null);
}
@@ -240,6 +244,13 @@ public class IndexSummaryBuilder implements AutoCloseable
offsets.close();
}
+ public Throwable close(Throwable accumulate)
+ {
+ accumulate = entries.close(accumulate);
+ accumulate = offsets.close(accumulate);
+ return accumulate;
+ }
+
public static int entriesAtSamplingLevel(int samplingLevel, int maxSummarySize)
{
return (int) Math.ceil((samplingLevel * maxSummarySize) / (double) BASE_SAMPLING_LEVEL);
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 f486b78..bc3486a 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTable.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTable.java
@@ -195,7 +195,7 @@ public abstract class SSTable
}
}
- private static Set<Component> discoverComponentsFor(Descriptor desc)
+ public static Set<Component> discoverComponentsFor(Descriptor desc)
{
Set<Component.Type> knownTypes = Sets.difference(Component.TYPES, Collections.singleton(Component.Type.CUSTOM));
Set<Component> components = Sets.newHashSetWithExpectedSize(knownTypes.size());
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 8890659..a526ec9 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.io.sstable;
import java.util.*;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import org.apache.cassandra.config.DatabaseDescriptor;
@@ -33,6 +32,8 @@ import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.sstable.format.SSTableWriter;
import org.apache.cassandra.utils.CLibrary;
import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.concurrent.Refs;
+import org.apache.cassandra.utils.concurrent.Transactional;
import static org.apache.cassandra.utils.Throwables.merge;
@@ -51,7 +52,7 @@ import static org.apache.cassandra.utils.Throwables.merge;
* but leave any hard-links in place for the readers we opened to cleanup when they're finished as we would had we finished
* successfully.
*/
-public class SSTableRewriter
+public class SSTableRewriter extends Transactional.AbstractTransactional implements Transactional
{
private static long preemptiveOpenInterval;
static
@@ -77,7 +78,9 @@ public class SSTableRewriter
private final ColumnFamilyStore cfs;
private final long maxAge;
- private final List<SSTableReader> finished = new ArrayList<>();
+ private long repairedAt = -1;
+ // the set of final readers we will expose on commit
+ private final List<SSTableReader> preparedForCommit = new ArrayList<>();
private final Set<SSTableReader> rewriting; // the readers we are rewriting (updated as they are replaced)
private final Map<Descriptor, DecoratedKey> originalStarts = new HashMap<>(); // the start key for each reader we are rewriting
private final Map<Descriptor, Integer> fileDescriptors = new HashMap<>(); // the file descriptors for each reader descriptor we are rewriting
@@ -85,21 +88,18 @@ public class SSTableRewriter
private SSTableReader currentlyOpenedEarly; // the reader for the most recent (re)opening of the target file
private long currentlyOpenedEarlyAt; // the position (in MB) in the target file we last (re)opened at
- private final List<SSTableReader> finishedReaders = new ArrayList<>();
- private final Queue<Finished> finishedEarly = new ArrayDeque<>();
- // as writers are closed from finishedEarly, their last readers are moved
- // into discard, so that abort can cleanup after us safely
- private final List<SSTableReader> discard = new ArrayList<>();
- private final boolean isOffline; // true for operations that are performed without Cassandra running (prevents updates of DataTracker)
+ private final List<Finished> finishedWriters = new ArrayList<>();
+ // as writers are closed from finishedWriters, their last readers are moved into discard, so that abort can cleanup
+ // after us safely; we use a set so we can add in both prepareToCommit and abort
+ private final Set<SSTableReader> discard = new HashSet<>();
+ // true for operations that are performed without Cassandra running (prevents updates of DataTracker)
+ private final boolean isOffline;
private SSTableWriter writer;
private Map<DecoratedKey, RowIndexEntry> cachedKeys = new HashMap<>();
- private State state = State.WORKING;
- private static enum State
- {
- WORKING, FINISHED, ABORTED
- }
+ // for testing (TODO: remove when have byteman setup)
+ private boolean throwEarly, throwLate;
public SSTableRewriter(ColumnFamilyStore cfs, Set<SSTableReader> rewriting, long maxAge, boolean isOffline)
{
@@ -178,7 +178,7 @@ public class SSTableRewriter
}
else
{
- SSTableReader reader = writer.openEarly(maxAge);
+ SSTableReader reader = writer.setMaxDataAge(maxAge).openEarly();
if (reader != null)
{
replaceEarlyOpenedFile(currentlyOpenedEarly, reader);
@@ -190,29 +190,19 @@ public class SSTableRewriter
}
}
- public void abort()
+ protected Throwable doAbort(Throwable accumulate)
{
- switch (state)
- {
- case ABORTED:
- return;
- case FINISHED:
- throw new IllegalStateException("Cannot abort - changes have already been committed");
- }
- state = State.ABORTED;
-
- Throwable fail = null;
try
{
moveStarts(null, null, true);
}
catch (Throwable t)
{
- fail = merge(fail, t);
+ accumulate = merge(accumulate, t);
}
- // remove already completed SSTables
- for (SSTableReader sstable : finished)
+ // cleanup any sstables we prepared for commit
+ for (SSTableReader sstable : preparedForCommit)
{
try
{
@@ -221,50 +211,41 @@ public class SSTableRewriter
}
catch (Throwable t)
{
- fail = merge(fail, t);
+ accumulate = merge(accumulate , t);
}
}
+ // abort the writers, and add the early opened readers to our discard pile
+
if (writer != null)
- finishedEarly.add(new Finished(writer, currentlyOpenedEarly));
+ finishedWriters.add(new Finished(writer, currentlyOpenedEarly));
- // abort the writers
- for (Finished finished : finishedEarly)
+ for (Finished finished : finishedWriters)
{
- try
- {
- finished.writer.abort();
- }
- catch (Throwable t)
- {
- fail = merge(fail, t);
- }
- try
- {
- if (finished.reader != null)
- {
- // if we've already been opened, add ourselves to the discard pile
- discard.add(finished.reader);
- finished.reader.markObsolete();
- }
- }
- catch (Throwable t)
- {
- fail = merge(fail, t);
- }
- }
+ accumulate = finished.writer.abort(accumulate);
- try
- {
- replaceWithFinishedReaders(Collections.<SSTableReader>emptyList());
- }
- catch (Throwable t)
- {
- fail = merge(fail, t);
+ // if we've already been opened, add ourselves to the discard pile
+ if (finished.reader != null)
+ discard.add(finished.reader);
}
- if (fail != null)
- throw Throwables.propagate(fail);
+ accumulate = replaceWithFinishedReaders(Collections.<SSTableReader>emptyList(), accumulate);
+ return accumulate;
+ }
+
+ protected Throwable doCommit(Throwable accumulate)
+ {
+ for (Finished f : finishedWriters)
+ accumulate = f.writer.commit(accumulate);
+ accumulate = replaceWithFinishedReaders(preparedForCommit, accumulate);
+
+ return accumulate;
+ }
+
+ protected Throwable doCleanup(Throwable accumulate)
+ {
+ // we have no state of our own to cleanup; Transactional objects cleanup their own state in abort or commit
+ return accumulate;
}
/**
@@ -369,41 +350,38 @@ public class SSTableRewriter
public void switchWriter(SSTableWriter newWriter)
{
- if (writer == null)
+ if (writer == null || writer.getFilePointer() == 0)
{
+ if (writer != null)
+ writer.abort();
writer = newWriter;
return;
}
- if (writer.getFilePointer() != 0)
- {
- // If early re-open is disabled, simply finalize the writer and store it
- if (preemptiveOpenInterval == Long.MAX_VALUE)
- {
- SSTableReader reader = writer.finish(SSTableWriter.FinishType.NORMAL, maxAge, -1);
- finishedReaders.add(reader);
- }
- else
- {
- // we leave it as a tmp file, but we open it and add it to the dataTracker
- SSTableReader reader = writer.finish(SSTableWriter.FinishType.EARLY, maxAge, -1);
- replaceEarlyOpenedFile(currentlyOpenedEarly, reader);
- moveStarts(reader, reader.last, false);
- finishedEarly.add(new Finished(writer, reader));
- }
- }
- else
+ SSTableReader reader = null;
+ if (preemptiveOpenInterval != Long.MAX_VALUE)
{
- writer.abort();
+ // we leave it as a tmp file, but we open it and add it to the dataTracker
+ reader = writer.setMaxDataAge(maxAge).openFinalEarly();
+ replaceEarlyOpenedFile(currentlyOpenedEarly, reader);
+ moveStarts(reader, reader.last, false);
}
+ finishedWriters.add(new Finished(writer, reader));
+
currentlyOpenedEarly = null;
currentlyOpenedEarlyAt = 0;
writer = newWriter;
}
- public List<SSTableReader> finish()
+ /**
+ * @param repairedAt the repair time, -1 if we should use the time we supplied when we created
+ * the SSTableWriter (and called rewriter.switchWriter(..)), actual time if we want to override the
+ * repair time.
+ */
+ public SSTableRewriter setRepairedAt(long repairedAt)
{
- return finish(-1);
+ this.repairedAt = repairedAt;
+ return this;
}
/**
@@ -417,94 +395,92 @@ public class SSTableRewriter
* gymnastics (ie, call DataTracker#markCompactedSSTablesReplaced(..))
*
*
- * @param repairedAt the repair time, -1 if we should use the time we supplied when we created
- * the SSTableWriter (and called rewriter.switchWriter(..)), actual time if we want to override the
- * repair time.
*/
- public List<SSTableReader> finish(long repairedAt)
+ public List<SSTableReader> finish()
{
- return finishAndMaybeThrow(repairedAt, false, false);
+ super.finish();
+ return finished();
}
- @VisibleForTesting
- void finishAndThrow(boolean throwEarly)
+ public List<SSTableReader> finished()
{
- finishAndMaybeThrow(-1, throwEarly, !throwEarly);
+ assert state() == State.COMMITTED || state() == State.READY_TO_COMMIT;
+ return preparedForCommit;
}
- private List<SSTableReader> finishAndMaybeThrow(long repairedAt, boolean throwEarly, boolean throwLate)
+ protected void doPrepare()
{
- switch (state)
- {
- case FINISHED: case ABORTED:
- throw new IllegalStateException("Cannot finish - changes have already been " + state.toString().toLowerCase());
- }
-
- List<SSTableReader> newReaders = new ArrayList<>();
switchWriter(null);
if (throwEarly)
throw new RuntimeException("exception thrown early in finish, for testing");
// No early open to finalize and replace
- if (preemptiveOpenInterval == Long.MAX_VALUE)
- {
- replaceWithFinishedReaders(finishedReaders);
- if (throwLate)
- throw new RuntimeException("exception thrown after all sstables finished, for testing");
- return finishedReaders;
- }
-
- while (!finishedEarly.isEmpty())
+ for (Finished f : finishedWriters)
{
- Finished f = finishedEarly.peek();
- if (f.writer.getFilePointer() > 0)
- {
- if (f.reader != null)
- discard.add(f.reader);
+ if (f.reader != null)
+ discard.add(f.reader);
- SSTableReader newReader = f.writer.finish(SSTableWriter.FinishType.FINISH_EARLY, maxAge, repairedAt);
+ f.writer.setRepairedAt(repairedAt).setMaxDataAge(maxAge).setOpenResult(true).prepareToCommit();
+ SSTableReader newReader = f.writer.finished();
- if (f.reader != null)
- f.reader.setReplacedBy(newReader);
+ if (f.reader != null)
+ f.reader.setReplacedBy(newReader);
- finished.add(newReader);
- newReaders.add(newReader);
- }
- else
- {
- f.writer.abort();
- assert f.reader == null;
- }
- finishedEarly.poll();
+ preparedForCommit.add(newReader);
}
if (throwLate)
throw new RuntimeException("exception thrown after all sstables finished, for testing");
+ }
- replaceWithFinishedReaders(newReaders);
- state = State.FINISHED;
- return finished;
+ @VisibleForTesting
+ void throwDuringPrepare(boolean throwEarly)
+ {
+ this.throwEarly = throwEarly;
+ this.throwLate = !throwEarly;
}
// cleanup all our temporary readers and swap in our new ones
- private void replaceWithFinishedReaders(List<SSTableReader> finished)
+ private Throwable replaceWithFinishedReaders(List<SSTableReader> finished, Throwable accumulate)
{
if (isOffline)
{
for (SSTableReader reader : discard)
{
- if (reader.getCurrentReplacement() == reader)
- reader.markObsolete();
- reader.selfRef().release();
+ try
+ {
+ if (reader.getCurrentReplacement() == reader)
+ reader.markObsolete();
+ }
+ catch (Throwable t)
+ {
+ accumulate = merge(accumulate, t);
+ }
}
+ accumulate = Refs.release(Refs.selfRefs(discard), accumulate);
}
else
{
- dataTracker.replaceEarlyOpenedFiles(discard, finished);
- dataTracker.unmarkCompacting(discard);
+ try
+ {
+ dataTracker.replaceEarlyOpenedFiles(discard, finished);
+ }
+ catch (Throwable t)
+ {
+ accumulate = merge(accumulate, t);
+ }
+ try
+ {
+ dataTracker.unmarkCompacting(discard);
+ }
+ catch (Throwable t)
+ {
+ accumulate = merge(accumulate, t);
+ }
}
discard.clear();
+ return accumulate;
}
private static final class Finished
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 5998044..d6ab940 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
@@ -157,6 +157,7 @@ public class SSTableSimpleUnsortedWriter extends AbstractSSTableSimpleWriter
{
throw new RuntimeException(e);
}
+ checkForWriterException();
}
// This is overridden by CQLSSTableWriter to hold off replacing column family until the next iteration through
@@ -215,39 +216,40 @@ public class SSTableSimpleUnsortedWriter extends AbstractSSTableSimpleWriter
public void run()
{
- SSTableWriter writer = null;
-
- while (true)
{
- try
+ while (true)
{
- Buffer b = writeQueue.take();
- if (b == SENTINEL)
- return;
-
- writer = getWriter();
- boolean first = true;
- for (Map.Entry<DecoratedKey, ColumnFamily> entry : b.entrySet())
+ try
{
- if (entry.getValue().getColumnCount() > 0)
- writer.append(entry.getKey(), entry.getValue());
- else if (!first)
- throw new AssertionError("Empty partition");
- first = false;
+ Buffer b = writeQueue.take();
+ if (b == SENTINEL)
+ return;
+
+ try (SSTableWriter writer = getWriter();)
+ {
+ boolean first = true;
+ for (Map.Entry<DecoratedKey, ColumnFamily> entry : b.entrySet())
+ {
+ if (entry.getValue().getColumnCount() > 0)
+ writer.append(entry.getKey(), entry.getValue());
+ else if (!first)
+ throw new AssertionError("Empty partition");
+ first = false;
+ }
+
+ writer.finish(false);
+ }
+ }
+ catch (Throwable e)
+ {
+ JVMStabilityInspector.inspectThrowable(e);
+ // Keep only the first exception
+ if (exception == null)
+ exception = e;
}
- writer.close();
- }
- catch (Throwable e)
- {
- JVMStabilityInspector.inspectThrowable(e);
- if (writer != null)
- writer.abort();
- // Keep only the first exception
- if (exception == null)
- exception = e;
}
- }
+ }
}
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 3417d68..f206969 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
@@ -19,6 +19,8 @@ package org.apache.cassandra.io.sstable;
import java.io.File;
+import com.google.common.base.Throwables;
+
import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.db.*;
import org.apache.cassandra.db.marshal.AbstractType;
@@ -72,12 +74,11 @@ public class SSTableSimpleWriter extends AbstractSSTableSimpleWriter
{
if (currentKey != null)
writeRow(currentKey, columnFamily);
- writer.close();
+ writer.finish(false);
}
- catch (FSError e)
+ catch (Throwable t)
{
- writer.abort();
- throw e;
+ throw Throwables.propagate(writer.abort(t));
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 4411ca7..23c27b0 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -864,12 +864,21 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
* @param ibuilder
* @param dbuilder
*/
+
public void saveSummary(SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder)
{
- saveSummary(ibuilder, dbuilder, indexSummary);
+ saveSummary(this.descriptor, this.first, this.last, ibuilder, dbuilder, indexSummary);
}
- private void saveSummary(SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder, IndexSummary summary)
+ private void saveSummary(SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder, IndexSummary newSummary)
+ {
+ saveSummary(this.descriptor, this.first, this.last, ibuilder, dbuilder, newSummary);
+ }
+ /**
+ * Save index summary to Summary.db file.
+ */
+ public static void saveSummary(Descriptor descriptor, DecoratedKey first, DecoratedKey last,
+ SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder, IndexSummary summary)
{
File summariesFile = new File(descriptor.filenameFor(Component.SUMMARY));
if (summariesFile.exists())
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 baacb5a..f99292e 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
@@ -31,16 +31,17 @@ 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.metadata.MetadataCollector;
+import org.apache.cassandra.io.sstable.metadata.MetadataComponent;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.utils.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.cassandra.utils.concurrent.Transactional;
import java.io.DataInput;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashSet;
+import java.util.Map;
import java.util.Set;
/**
@@ -49,30 +50,24 @@ import java.util.Set;
* TableWriter.create() is the primary way to create a writer for a particular format.
* The format information is part of the Descriptor.
*/
-public abstract class SSTableWriter extends SSTable
+public abstract class SSTableWriter extends SSTable implements Transactional
{
- private static final Logger logger = LoggerFactory.getLogger(SSTableWriter.class);
-
- public static enum FinishType
- {
- CLOSE(null, true),
- NORMAL(SSTableReader.OpenReason.NORMAL, true),
- EARLY(SSTableReader.OpenReason.EARLY, false), // no renaming
- FINISH_EARLY(SSTableReader.OpenReason.NORMAL, true); // tidy up an EARLY finish
- public final SSTableReader.OpenReason openReason;
-
- public final boolean isFinal;
- FinishType(SSTableReader.OpenReason openReason, boolean isFinal)
- {
- this.openReason = openReason;
- this.isFinal = isFinal;
- }
- }
-
- protected final long repairedAt;
+ protected long repairedAt;
+ protected long maxDataAge = -1;
protected final long keyCount;
protected final MetadataCollector metadataCollector;
protected final RowIndexEntry.IndexSerializer rowIndexEntrySerializer;
+ protected final TransactionalProxy txnProxy = txnProxy();
+
+ protected abstract TransactionalProxy txnProxy();
+
+ // due to lack of multiple inheritance, we use an inner class to proxy our Transactional implementation details
+ protected abstract class TransactionalProxy extends AbstractTransactional
+ {
+ // should be set during doPrepare()
+ protected SSTableReader finalReader;
+ protected boolean openResult;
+ }
protected SSTableWriter(Descriptor descriptor, long keyCount, long repairedAt, CFMetaData metadata, IPartitioner partitioner, MetadataCollector metadataCollector)
{
@@ -164,28 +159,98 @@ public abstract class SSTableWriter extends SSTable
public abstract long getOnDiskFilePointer();
- public abstract void isolateReferences();
-
public abstract void resetAndTruncate();
- public SSTableReader closeAndOpenReader()
+ public SSTableWriter setRepairedAt(long repairedAt)
+ {
+ if (repairedAt > 0)
+ this.repairedAt = repairedAt;
+ return this;
+ }
+
+ public SSTableWriter setMaxDataAge(long maxDataAge)
+ {
+ this.maxDataAge = maxDataAge;
+ return this;
+ }
+
+ public SSTableWriter setOpenResult(boolean openResult)
+ {
+ txnProxy.openResult = openResult;
+ return this;
+ }
+
+ /**
+ * Open the resultant SSTableReader before it has been fully written
+ */
+ public abstract SSTableReader openEarly();
+
+ /**
+ * Open the resultant SSTableReader once it has been fully written, but before the
+ * _set_ of tables that are being written together as one atomic operation are all ready
+ */
+ public abstract SSTableReader openFinalEarly();
+
+ public SSTableReader finish(long repairedAt, long maxDataAge, boolean openResult)
{
- return closeAndOpenReader(System.currentTimeMillis());
+ if (repairedAt > 0)
+ this.repairedAt = repairedAt;
+ this.maxDataAge = maxDataAge;
+ return finish(openResult);
}
- public SSTableReader closeAndOpenReader(long maxDataAge)
+ public SSTableReader finish(boolean openResult)
{
- return finish(FinishType.NORMAL, maxDataAge, repairedAt);
+ txnProxy.openResult = openResult;
+ txnProxy.finish();
+ return finished();
}
- public abstract SSTableReader finish(FinishType finishType, long maxDataAge, long repairedAt);
+ /**
+ * Open the resultant SSTableReader once it has been fully written, and all related state
+ * is ready to be finalised including other sstables being written involved in the same operation
+ */
+ public SSTableReader finished()
+ {
+ return txnProxy.finalReader;
+ }
+
+ // finalise our state on disk, including renaming
+ public final void prepareToCommit()
+ {
+ txnProxy.prepareToCommit();
+ }
+
+ public final Throwable commit(Throwable accumulate)
+ {
+ return txnProxy.commit(accumulate);
+ }
+
+ public final Throwable abort(Throwable accumulate)
+ {
+ return txnProxy.abort(accumulate);
+ }
- public abstract SSTableReader openEarly(long maxDataAge);
+ public final void close()
+ {
+ txnProxy.close();
+ }
- // Close the writer and return the descriptor to the new sstable and it's metadata
- public abstract Pair<Descriptor, StatsMetadata> close();
+ public final void abort()
+ {
+ txnProxy.abort();
+ }
+ protected Map<MetadataType, MetadataComponent> finalizeMetadata()
+ {
+ return metadataCollector.finalizeMetadata(partitioner.getClass().getCanonicalName(),
+ metadata.getBloomFilterFpChance(), repairedAt);
+ }
+ protected StatsMetadata statsMetadata()
+ {
+ return (StatsMetadata) finalizeMetadata().get(MetadataType.STATS);
+ }
public static Descriptor rename(Descriptor tmpdesc, Set<Component> components)
{
@@ -209,12 +274,6 @@ public abstract class SSTableWriter extends SSTable
}
- /**
- * After failure, attempt to close the index writer and data file before deleting all temp components for the sstable
- */
- public abstract void abort();
-
-
public static abstract class Factory
{
public abstract SSTableWriter open(Descriptor descriptor, long keyCount, long repairedAt, CFMetaData metadata, IPartitioner partitioner, MetadataCollector metadataCollector);
[2/7] cassandra git commit: Introduce Transactional API for internal
state changes
Posted by be...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 087e57a..fa17c20 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
@@ -47,8 +47,10 @@ import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.FilterFactory;
import org.apache.cassandra.utils.IFilter;
-import org.apache.cassandra.utils.Pair;
import org.apache.cassandra.utils.StreamingHistogram;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import static org.apache.cassandra.utils.Throwables.merge;
public class BigTableWriter extends SSTableWriter
{
@@ -57,7 +59,7 @@ public class BigTableWriter extends SSTableWriter
// not very random, but the only value that can't be mistaken for a legal column-name length
public static final int END_OF_ROW = 0x0000;
- private IndexWriter iwriter;
+ private final IndexWriter iwriter;
private SegmentedFile.Builder dbuilder;
private final SequentialWriter dataFile;
private DecoratedKey lastWrittenKey;
@@ -270,47 +272,6 @@ public class BigTableWriter extends SSTableWriter
return currentPosition;
}
- /**
- * After failure, attempt to close the index writer and data file before deleting all temp components for the sstable
- */
- public void abort()
- {
- assert descriptor.type.isTemporary;
- if (iwriter == null && dataFile == null)
- return;
-
- if (iwriter != null)
- iwriter.abort();
-
- if (dataFile!= null)
- dataFile.abort();
-
- if (dbuilder != null)
- dbuilder.close();
-
- Set<Component> components = SSTable.componentsFor(descriptor);
- try
- {
- if (!components.isEmpty())
- SSTable.delete(descriptor, components);
- }
- catch (FSWriteError e)
- {
- logger.error(String.format("Failed deleting temp components for %s", descriptor), e);
- throw e;
- }
- }
-
- // we use this method to ensure any managed data we may have retained references to during the write are no
- // longer referenced, so that we do not need to enclose the expensive call to closeAndOpenReader() in a transaction
- public void isolateReferences()
- {
- // currently we only maintain references to first/last/lastWrittenKey from the data provided; all other
- // data retention is done through copying
- first = getMinimalKey(first);
- last = lastWrittenKey = getMinimalKey(last);
- }
-
private Descriptor makeTmpLinks()
{
// create temp links if they don't already exist
@@ -323,17 +284,14 @@ public class BigTableWriter extends SSTableWriter
return link;
}
- public SSTableReader openEarly(long maxDataAge)
+ public SSTableReader openEarly()
{
- StatsMetadata sstableMetadata = (StatsMetadata) metadataCollector.finalizeMetadata(partitioner.getClass().getCanonicalName(),
- metadata.getBloomFilterFpChance(),
- repairedAt).get(MetadataType.STATS);
-
// find the max (exclusive) readable key
IndexSummaryBuilder.ReadableBoundary boundary = iwriter.getMaxReadable();
if (boundary == null)
return null;
+ 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
@@ -343,7 +301,7 @@ public class BigTableWriter extends SSTableWriter
components, metadata,
partitioner, ifile,
dfile, iwriter.summary.build(partitioner, boundary),
- iwriter.bf.sharedCopy(), maxDataAge, sstableMetadata, SSTableReader.OpenReason.EARLY);
+ iwriter.bf.sharedCopy(), maxDataAge, stats, SSTableReader.OpenReason.EARLY);
// now it's open, find the ACTUAL last readable key (i.e. for which the data file has also been flushed)
sstable.first = getMinimalKey(first);
@@ -351,31 +309,23 @@ public class BigTableWriter extends SSTableWriter
return sstable;
}
- public SSTableReader closeAndOpenReader()
+ public SSTableReader openFinalEarly()
{
- return closeAndOpenReader(System.currentTimeMillis());
+ // we must ensure the data is completely flushed to disk
+ dataFile.sync();
+ iwriter.indexFile.sync();
+ return openFinal(makeTmpLinks(), SSTableReader.OpenReason.EARLY);
}
- public SSTableReader closeAndOpenReader(long maxDataAge)
+ private SSTableReader openFinal(Descriptor desc, SSTableReader.OpenReason openReason)
{
- return finish(FinishType.NORMAL, maxDataAge, this.repairedAt);
- }
-
- public SSTableReader finish(FinishType finishType, long maxDataAge, long repairedAt)
- {
- assert finishType != FinishType.CLOSE;
- Pair<Descriptor, StatsMetadata> p;
-
- p = close(finishType, repairedAt < 0 ? this.repairedAt : repairedAt);
- Descriptor desc = p.left;
- StatsMetadata metadata = p.right;
-
- if (finishType == FinishType.EARLY)
- desc = makeTmpLinks();
+ if (maxDataAge < 0)
+ maxDataAge = System.currentTimeMillis();
+ StatsMetadata stats = statsMetadata();
// finalize in-memory state for the reader
- SegmentedFile ifile = iwriter.builder.complete(desc.filenameFor(Component.PRIMARY_INDEX), finishType.isFinal);
- SegmentedFile dfile = dbuilder.complete(desc.filenameFor(Component.DATA), finishType.isFinal);
+ 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),
components,
this.metadata,
@@ -385,81 +335,93 @@ public class BigTableWriter extends SSTableWriter
iwriter.summary.build(partitioner),
iwriter.bf.sharedCopy(),
maxDataAge,
- metadata,
- finishType.openReason);
+ stats,
+ openReason);
sstable.first = getMinimalKey(first);
sstable.last = getMinimalKey(last);
-
- if (finishType.isFinal)
- {
- iwriter.bf.close();
- iwriter.summary.close();
- // try to save the summaries to disk
- sstable.saveSummary(iwriter.builder, dbuilder);
- iwriter.builder.close();
- iwriter = null;
- dbuilder.close();
- dbuilder = null;
- }
return sstable;
}
- // Close the writer and return the descriptor to the new sstable and it's metadata
- public Pair<Descriptor, StatsMetadata> close()
+ protected SSTableWriter.TransactionalProxy txnProxy()
{
- Pair<Descriptor, StatsMetadata> ret = close(FinishType.CLOSE, this.repairedAt);
- if (dbuilder != null)
- dbuilder.close();
- if (iwriter != null)
- iwriter.builder.close();
- return ret;
+ return new TransactionalProxy();
}
- private Pair<Descriptor, StatsMetadata> close(FinishType type, long repairedAt)
+ class TransactionalProxy extends SSTableWriter.TransactionalProxy
{
- switch (type)
+ // finalise our state on disk, including renaming
+ protected void doPrepare()
{
- case EARLY: case CLOSE: case NORMAL:
- iwriter.close();
- dataFile.close();
- if (type == FinishType.CLOSE)
- iwriter.bf.close();
- }
+ Map<MetadataType, MetadataComponent> metadataComponents = finalizeMetadata();
- // write sstable statistics
- Map<MetadataType, MetadataComponent> metadataComponents;
- metadataComponents = metadataCollector
- .finalizeMetadata(partitioner.getClass().getCanonicalName(),
- metadata.getBloomFilterFpChance(),repairedAt);
+ iwriter.prepareToCommit();
- // remove the 'tmp' marker from all components
- Descriptor descriptor = this.descriptor;
- if (type.isFinal)
- {
- dataFile.writeFullChecksum(descriptor);
+ // write sstable statistics
+ dataFile.setDescriptor(descriptor).prepareToCommit();
writeMetadata(descriptor, metadataComponents);
+
// save the table of components
SSTable.appendTOC(descriptor, components);
- descriptor = rename(descriptor, components);
+
+ // rename to final
+ rename(descriptor, components);
+
+ if (openResult)
+ finalReader = openFinal(descriptor.asType(Descriptor.Type.FINAL), SSTableReader.OpenReason.NORMAL);
}
- return Pair.create(descriptor, (StatsMetadata) metadataComponents.get(MetadataType.STATS));
+ protected Throwable doCommit(Throwable accumulate)
+ {
+ accumulate = dataFile.commit(accumulate);
+ accumulate = iwriter.commit(accumulate);
+ return accumulate;
+ }
+
+ protected Throwable doCleanup(Throwable accumulate)
+ {
+ accumulate = dbuilder.close(accumulate);
+ return accumulate;
+ }
+
+ protected Throwable doAbort(Throwable accumulate)
+ {
+ 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;
+ }
}
private static void writeMetadata(Descriptor desc, Map<MetadataType, MetadataComponent> components)
{
- SequentialWriter out = SequentialWriter.open(new File(desc.filenameFor(Component.STATS)));
- try
+ File file = new File(desc.filenameFor(Component.STATS));
+ try (SequentialWriter out = SequentialWriter.open(file);)
{
desc.getMetadataSerializer().serialize(components, out.stream);
+ out.setDescriptor(desc).finish();
}
catch (IOException e)
{
- throw new FSWriteError(e, out.getPath());
- }
- finally
- {
- out.close();
+ throw new FSWriteError(e, file.getPath());
}
}
@@ -476,7 +438,7 @@ public class BigTableWriter extends SSTableWriter
/**
* Encapsulates writing the index and filter for an SSTable. The state of this object is not valid until it has been closed.
*/
- class IndexWriter
+ class IndexWriter extends AbstractTransactional implements Transactional
{
private final SequentialWriter indexFile;
public final SegmentedFile.Builder builder;
@@ -535,18 +497,10 @@ public class BigTableWriter extends SSTableWriter
builder.addPotentialBoundary(indexStart);
}
- public void abort()
- {
- summary.close();
- indexFile.abort();
- bf.close();
- builder.close();
- }
-
/**
* Closes the index and bloomfilter, making the public state of this writer valid for consumption.
*/
- public void close()
+ void flushBf()
{
if (components.contains(Component.FILTER))
{
@@ -566,11 +520,6 @@ public class BigTableWriter extends SSTableWriter
throw new FSWriteError(e, path);
}
}
-
- // index
- long position = indexFile.getFilePointer();
- indexFile.close(); // calls force
- FileUtils.truncate(indexFile.getPath(), position);
}
public void mark()
@@ -585,5 +534,40 @@ public class BigTableWriter extends SSTableWriter
// we assume that if that worked then we won't be trying to reset.
indexFile.resetAndTruncate(mark);
}
+
+ protected void doPrepare()
+ {
+ flushBf();
+
+ // truncate index file
+ long position = iwriter.indexFile.getFilePointer();
+ iwriter.indexFile.setDescriptor(descriptor).prepareToCommit();
+ FileUtils.truncate(iwriter.indexFile.getPath(), position);
+
+ // save summary
+ summary.prepareToCommit();
+ try (IndexSummary summary = iwriter.summary.build(partitioner))
+ {
+ SSTableReader.saveSummary(descriptor, first, last, iwriter.builder, dbuilder, summary);
+ }
+ }
+
+ protected Throwable doCommit(Throwable accumulate)
+ {
+ return indexFile.commit(accumulate);
+ }
+
+ protected Throwable doAbort(Throwable accumulate)
+ {
+ return indexFile.abort(accumulate);
+ }
+
+ protected Throwable doCleanup(Throwable accumulate)
+ {
+ accumulate = summary.close(accumulate);
+ accumulate = bf.close(accumulate);
+ accumulate = builder.close(accumulate);
+ return accumulate;
+ }
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/io/util/BufferedPoolingSegmentedFile.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/BufferedPoolingSegmentedFile.java b/src/java/org/apache/cassandra/io/util/BufferedPoolingSegmentedFile.java
index e8b719e..b623e54 100644
--- a/src/java/org/apache/cassandra/io/util/BufferedPoolingSegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/BufferedPoolingSegmentedFile.java
@@ -41,9 +41,8 @@ public class BufferedPoolingSegmentedFile extends PoolingSegmentedFile
// only one segment in a standard-io file
}
- public SegmentedFile complete(ChannelProxy channel, long overrideLength, boolean isFinal)
+ public SegmentedFile complete(ChannelProxy channel, long overrideLength)
{
- assert !isFinal || overrideLength <= 0;
long length = overrideLength > 0 ? overrideLength : channel.size();
return new BufferedPoolingSegmentedFile(channel, length);
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/io/util/BufferedSegmentedFile.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/BufferedSegmentedFile.java b/src/java/org/apache/cassandra/io/util/BufferedSegmentedFile.java
index b4d966a..2c59def 100644
--- a/src/java/org/apache/cassandra/io/util/BufferedSegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/BufferedSegmentedFile.java
@@ -48,9 +48,8 @@ public class BufferedSegmentedFile extends SegmentedFile
// only one segment in a standard-io file
}
- public SegmentedFile complete(ChannelProxy channel, long overrideLength, boolean isFinal)
+ public SegmentedFile complete(ChannelProxy channel, long overrideLength)
{
- assert !isFinal || overrideLength <= 0;
long length = overrideLength > 0 ? overrideLength : channel.size();
return new BufferedSegmentedFile(channel, length);
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/io/util/ChecksummedSequentialWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/ChecksummedSequentialWriter.java b/src/java/org/apache/cassandra/io/util/ChecksummedSequentialWriter.java
index d28a14d..ec68c2d 100644
--- a/src/java/org/apache/cassandra/io/util/ChecksummedSequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/util/ChecksummedSequentialWriter.java
@@ -20,8 +20,6 @@ package org.apache.cassandra.io.util;
import java.io.File;
import java.nio.ByteBuffer;
-import org.apache.cassandra.io.sstable.Descriptor;
-
public class ChecksummedSequentialWriter extends SequentialWriter
{
private final SequentialWriter crcWriter;
@@ -44,20 +42,36 @@ public class ChecksummedSequentialWriter extends SequentialWriter
crcMetadata.appendDirect(toAppend, false);
}
- public void writeFullChecksum(Descriptor descriptor)
+ protected class TransactionalProxy extends SequentialWriter.TransactionalProxy
{
- crcMetadata.writeFullChecksum(descriptor);
- }
+ @Override
+ protected Throwable doCommit(Throwable accumulate)
+ {
+ return crcWriter.commit(accumulate);
+ }
- public void close()
- {
- super.close();
- crcWriter.close();
+ @Override
+ protected Throwable doAbort(Throwable accumulate)
+ {
+ return super.doAbort(crcWriter.abort(accumulate));
+ }
+
+ @Override
+ protected void doPrepare()
+ {
+ syncInternal();
+ if (descriptor != null)
+ crcMetadata.writeFullChecksum(descriptor);
+ crcWriter.setDescriptor(descriptor).prepareToCommit();
+ // we must cleanup our file handles during prepareCommit for Windows compatibility as we cannot rename an open file;
+ // TODO: once we stop file renaming, remove this for clarity
+ releaseFileHandle();
+ }
}
- public void abort()
+ @Override
+ protected SequentialWriter.TransactionalProxy txnProxy()
{
- super.abort();
- crcWriter.abort();
+ return new TransactionalProxy();
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/io/util/CompressedPoolingSegmentedFile.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/CompressedPoolingSegmentedFile.java b/src/java/org/apache/cassandra/io/util/CompressedPoolingSegmentedFile.java
index cb30131..fdc4f61 100644
--- a/src/java/org/apache/cassandra/io/util/CompressedPoolingSegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/CompressedPoolingSegmentedFile.java
@@ -96,9 +96,9 @@ public class CompressedPoolingSegmentedFile extends PoolingSegmentedFile impleme
// only one segment in a standard-io file
}
- public SegmentedFile complete(ChannelProxy channel, long overrideLength, boolean isFinal)
+ public SegmentedFile complete(ChannelProxy channel, long overrideLength)
{
- return new CompressedPoolingSegmentedFile(channel, metadata(channel.filePath(), overrideLength, isFinal));
+ return new CompressedPoolingSegmentedFile(channel, metadata(channel.filePath(), overrideLength));
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/io/util/CompressedSegmentedFile.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/CompressedSegmentedFile.java b/src/java/org/apache/cassandra/io/util/CompressedSegmentedFile.java
index caf4c22..ceff7ba 100644
--- a/src/java/org/apache/cassandra/io/util/CompressedSegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/CompressedSegmentedFile.java
@@ -136,18 +136,17 @@ public class CompressedSegmentedFile extends SegmentedFile implements ICompresse
// only one segment in a standard-io file
}
- protected CompressionMetadata metadata(String path, long overrideLength, boolean isFinal)
+ protected CompressionMetadata metadata(String path, long overrideLength)
{
if (writer == null)
return CompressionMetadata.create(path);
- return writer.open(overrideLength, isFinal);
+ return writer.open(overrideLength);
}
- public SegmentedFile complete(ChannelProxy channel, long overrideLength, boolean isFinal)
+ public SegmentedFile complete(ChannelProxy channel, long overrideLength)
{
- assert !isFinal || overrideLength <= 0;
- return new CompressedSegmentedFile(channel, metadata(channel.filePath(), overrideLength, isFinal));
+ return new CompressedSegmentedFile(channel, metadata(channel.filePath(), overrideLength));
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 8007039..2566952 100644
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@ -40,6 +40,9 @@ import org.apache.cassandra.io.sstable.CorruptSSTableException;
import org.apache.cassandra.service.StorageService;
import org.apache.cassandra.utils.JVMStabilityInspector;
+import static org.apache.cassandra.utils.Throwables.maybeFail;
+import static org.apache.cassandra.utils.Throwables.merge;
+
public class FileUtils
{
private static final Logger logger = LoggerFactory.getLogger(FileUtils.class);
@@ -107,24 +110,42 @@ public class FileUtils
return createTempFile(prefix, suffix, new File(System.getProperty("java.io.tmpdir")));
}
- public static void deleteWithConfirm(String file)
+ public static Throwable deleteWithConfirm(String filePath, boolean expect, Throwable accumulate)
{
- deleteWithConfirm(new File(file));
+ return deleteWithConfirm(new File(filePath), expect, accumulate);
}
- public static void deleteWithConfirm(File file)
+ public static Throwable deleteWithConfirm(File file, boolean expect, Throwable accumulate)
{
- assert file.exists() : "attempted to delete non-existing file " + file.getName();
- if (logger.isDebugEnabled())
- logger.debug("Deleting {}", file.getName());
+ boolean exists = file.exists();
+ assert exists || !expect : "attempted to delete non-existing file " + file.getName();
try
{
- Files.delete(file.toPath());
+ if (exists)
+ Files.delete(file.toPath());
}
- catch (IOException e)
+ catch (Throwable t)
{
- throw new FSWriteError(e, file);
+ try
+ {
+ throw new FSWriteError(t, file);
+ }
+ catch (Throwable t2)
+ {
+ accumulate = merge(accumulate, t2);
+ }
}
+ return accumulate;
+ }
+
+ public static void deleteWithConfirm(String file)
+ {
+ deleteWithConfirm(new File(file));
+ }
+
+ public static void deleteWithConfirm(File file)
+ {
+ maybeFail(deleteWithConfirm(file, true, null));
}
public static void renameWithOutConfirm(String from, String to)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java b/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
index 57295fe..91908c9 100644
--- a/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
@@ -183,9 +183,8 @@ public class MmappedSegmentedFile extends SegmentedFile
}
}
- public SegmentedFile complete(ChannelProxy channel, long overrideLength, boolean isFinal)
+ public SegmentedFile complete(ChannelProxy channel, long overrideLength)
{
- assert !isFinal || overrideLength <= 0;
long length = overrideLength > 0 ? overrideLength : channel.size();
// create the segments
return new MmappedSegmentedFile(channel, length, createSegments(channel, length));
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/io/util/SafeMemory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/SafeMemory.java b/src/java/org/apache/cassandra/io/util/SafeMemory.java
index f96afcc..ad11472 100644
--- a/src/java/org/apache/cassandra/io/util/SafeMemory.java
+++ b/src/java/org/apache/cassandra/io/util/SafeMemory.java
@@ -62,6 +62,11 @@ public class SafeMemory extends Memory implements SharedCloseable
peer = 0;
}
+ public Throwable close(Throwable accumulate)
+ {
+ return ref.ensureReleased(accumulate);
+ }
+
public SafeMemory copy(long newSize)
{
SafeMemory copy = new SafeMemory(newSize);
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/io/util/SafeMemoryWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/SafeMemoryWriter.java b/src/java/org/apache/cassandra/io/util/SafeMemoryWriter.java
index 1fc374f..1096b5f 100644
--- a/src/java/org/apache/cassandra/io/util/SafeMemoryWriter.java
+++ b/src/java/org/apache/cassandra/io/util/SafeMemoryWriter.java
@@ -71,6 +71,11 @@ public class SafeMemoryWriter extends DataOutputBuffer
memory.close();
}
+ public Throwable close(Throwable accumulate)
+ {
+ return memory.close(accumulate);
+ }
+
public long length()
{
return tailOffset(memory) + buffer.position();
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/io/util/SegmentedFile.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/SegmentedFile.java b/src/java/org/apache/cassandra/io/util/SegmentedFile.java
index cb4d132..edbd742 100644
--- a/src/java/org/apache/cassandra/io/util/SegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/SegmentedFile.java
@@ -37,6 +37,8 @@ import org.apache.cassandra.utils.Pair;
import org.apache.cassandra.utils.concurrent.RefCounted;
import org.apache.cassandra.utils.concurrent.SharedCloseableImpl;
+import static org.apache.cassandra.utils.Throwables.maybeFail;
+
/**
* Abstracts a read-only file that has been split into segments, each of which can be represented by an independent
* FileDataInput. Allows for iteration over the FileDataInputs, or random access to the FileDataInput for a given
@@ -169,21 +171,16 @@ public abstract class SegmentedFile extends SharedCloseableImpl
* Called after all potential boundaries have been added to apply this Builder to a concrete file on disk.
* @param channel The channel to the file on disk.
*/
- protected abstract SegmentedFile complete(ChannelProxy channel, long overrideLength, boolean isFinal);
+ protected abstract SegmentedFile complete(ChannelProxy channel, long overrideLength);
public SegmentedFile complete(String path)
{
- return complete(getChannel(path), -1, true);
- }
-
- public SegmentedFile complete(String path, boolean isFinal)
- {
- return complete(getChannel(path), -1, isFinal);
+ return complete(getChannel(path), -1);
}
public SegmentedFile complete(String path, long overrideLength)
{
- return complete(getChannel(path), overrideLength, false);
+ return complete(getChannel(path), overrideLength);
}
public void serializeBounds(DataOutput out) throws IOException
@@ -197,10 +194,16 @@ public abstract class SegmentedFile extends SharedCloseableImpl
throw new IOException("Cannot deserialize SSTable Summary component because the DiskAccessMode was changed!");
}
- public void close()
+ public Throwable close(Throwable accumulate)
{
if (channel != null)
- channel.close();
+ return channel.close(accumulate);
+ return accumulate;
+ }
+
+ public void close()
+ {
+ maybeFail(close(null));
}
private ChannelProxy getChannel(String path)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 c4fef07..d63be31 100644
--- a/src/java/org/apache/cassandra/io/util/SequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/util/SequentialWriter.java
@@ -24,8 +24,6 @@ import java.nio.channels.FileChannel;
import java.nio.channels.WritableByteChannel;
import java.nio.file.StandardOpenOption;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.io.FSReadError;
import org.apache.cassandra.io.FSWriteError;
@@ -34,15 +32,16 @@ import org.apache.cassandra.io.compress.CompressionParameters;
import org.apache.cassandra.io.sstable.Descriptor;
import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
import org.apache.cassandra.utils.CLibrary;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import static org.apache.cassandra.utils.Throwables.merge;
/**
* Adds buffering, mark, and fsyncing to OutputStream. We always fsync on close; we may also
* fsync incrementally if Config.trickle_fsync is enabled.
*/
-public class SequentialWriter extends OutputStream implements WritableByteChannel
+public class SequentialWriter extends OutputStream implements WritableByteChannel, Transactional
{
- private static final Logger logger = LoggerFactory.getLogger(SequentialWriter.class);
-
// isDirty - true if this.buffer contains any un-synced bytes
protected boolean isDirty = false, syncNeeded = false;
@@ -71,6 +70,55 @@ public class SequentialWriter extends OutputStream implements WritableByteChanne
protected Runnable runPostFlush;
+ private final TransactionalProxy txnProxy = txnProxy();
+ protected Descriptor descriptor;
+
+ // due to lack of multiple-inheritance, we proxy our transactional implementation
+ protected class TransactionalProxy extends AbstractTransactional
+ {
+ @Override
+ protected Throwable doCleanup(Throwable accumulate)
+ {
+ if (directoryFD >= 0)
+ {
+ try { CLibrary.tryCloseFD(directoryFD); }
+ catch (Throwable t) { accumulate = merge(accumulate, t); }
+ directoryFD = -1;
+ }
+
+ // close is idempotent
+ try { channel.close(); }
+ catch (Throwable t) { accumulate = merge(accumulate, t); }
+
+ if (buffer != null)
+ {
+ try { FileUtils.clean(buffer); }
+ catch (Throwable t) { accumulate = merge(accumulate, t); }
+ buffer = null;
+ }
+
+ return accumulate;
+ }
+
+ protected void doPrepare()
+ {
+ syncInternal();
+ // we must cleanup our file handles during prepareCommit for Windows compatibility as we cannot rename an open file;
+ // TODO: once we stop file renaming, remove this for clarity
+ releaseFileHandle();
+ }
+
+ protected Throwable doCommit(Throwable accumulate)
+ {
+ return accumulate;
+ }
+
+ protected Throwable doAbort(Throwable accumulate)
+ {
+ return FileUtils.deleteWithConfirm(filePath, false, accumulate);
+ }
+ }
+
public SequentialWriter(File file, int bufferSize, boolean offheap)
{
try
@@ -383,49 +431,53 @@ public class SequentialWriter extends OutputStream implements WritableByteChanne
return channel.isOpen();
}
- @Override
- public void close()
+ public SequentialWriter setDescriptor(Descriptor descriptor)
{
- if (buffer == null)
- return; // already closed
-
- syncInternal();
+ this.descriptor = descriptor;
+ return this;
+ }
- buffer = null;
+ public final void prepareToCommit()
+ {
+ txnProxy.prepareToCommit();
+ }
- cleanup(true);
+ public final Throwable commit(Throwable accumulate)
+ {
+ return txnProxy.commit(accumulate);
}
- public void abort()
+ public final Throwable abort(Throwable accumulate)
{
- cleanup(false);
+ return txnProxy.abort(accumulate);
}
- private void cleanup(boolean throwExceptions)
+ @Override
+ public final void close()
{
- if (directoryFD >= 0)
- {
- try { CLibrary.tryCloseFD(directoryFD); }
- catch (Throwable t) { handle(t, throwExceptions); }
- directoryFD = -1;
- }
+ txnProxy.close();
+ }
- // close is idempotent
- try { channel.close(); }
- catch (Throwable t) { handle(t, throwExceptions); }
+ public final void finish()
+ {
+ txnProxy.finish();
}
- private void handle(Throwable t, boolean throwExceptions)
+ protected TransactionalProxy txnProxy()
{
- if (!throwExceptions)
- logger.warn("Suppressing exception thrown while aborting writer", t);
- else
- throw new FSWriteError(t, getPath());
+ return new TransactionalProxy();
}
- // hack to make life easier for subclasses
- public void writeFullChecksum(Descriptor descriptor)
+ public void releaseFileHandle()
{
+ try
+ {
+ channel.close();
+ }
+ catch (IOException e)
+ {
+ throw new FSWriteError(e, filePath);
+ }
}
/**
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 1049d43..d4d49b3 100644
--- a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
@@ -122,7 +122,7 @@ public class StreamReceiveTask extends StreamTask
lockfile.create(task.sstables);
List<SSTableReader> readers = new ArrayList<>();
for (SSTableWriter writer : task.sstables)
- readers.add(writer.closeAndOpenReader());
+ readers.add(writer.finish(true));
lockfile.delete();
task.sstables.clear();
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/tools/SSTableImport.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/SSTableImport.java b/src/java/org/apache/cassandra/tools/SSTableImport.java
index d3a2683..7b187ac 100644
--- a/src/java/org/apache/cassandra/tools/SSTableImport.java
+++ b/src/java/org/apache/cassandra/tools/SSTableImport.java
@@ -71,7 +71,7 @@ public class SSTableImport
private final boolean isSorted;
private static final JsonFactory factory = new MappingJsonFactory().configure(
- JsonParser.Feature.INTERN_FIELD_NAMES, false);
+ JsonParser.Feature.INTERN_FIELD_NAMES, false);
static
{
@@ -143,10 +143,10 @@ public class SSTableImport
else
{
assert meta.isCQL3Table() || name.hasRemaining() : "Cell name should not be empty";
- value = stringAsType((String) fields.get(1),
- meta.getValueValidator(name.hasRemaining()
- ? comparator.cellFromByteBuffer(name)
- : meta.comparator.rowMarker(Composites.EMPTY)));
+ value = stringAsType((String) fields.get(1),
+ meta.getValueValidator(name.hasRemaining()
+ ? comparator.cellFromByteBuffer(name)
+ : meta.comparator.rowMarker(Composites.EMPTY)));
}
}
}
@@ -219,10 +219,10 @@ public class SSTableImport
cfamily.addAtom(new RangeTombstone(start, end, col.timestamp, col.localExpirationTime));
continue;
}
-
+
assert cfm.isCQL3Table() || col.getName().hasRemaining() : "Cell name should not be empty";
- CellName cname = col.getName().hasRemaining() ? cfm.comparator.cellFromByteBuffer(col.getName())
- : cfm.comparator.rowMarker(Composites.EMPTY);
+ CellName cname = col.getName().hasRemaining() ? cfm.comparator.cellFromByteBuffer(col.getName())
+ : cfm.comparator.rowMarker(Composites.EMPTY);
if (col.isExpiring())
{
@@ -345,13 +345,13 @@ public class SSTableImport
break;
}
- writer.closeAndOpenReader();
+ writer.finish(true);
return importedKeys;
}
private int importSorted(String jsonFile, ColumnFamily columnFamily, String ssTablePath,
- IPartitioner partitioner) throws IOException
+ IPartitioner partitioner) throws IOException
{
int importedKeys = 0; // already imported keys count
long start = System.nanoTime();
@@ -377,55 +377,56 @@ public class SSTableImport
System.out.printf("Importing %s keys...%n", keyCountToImport);
parser = getParser(jsonFile); // renewing parser
- SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(ssTablePath), keyCountToImport, ActiveRepairService.UNREPAIRED_SSTABLE);
+ try (SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(ssTablePath), keyCountToImport, ActiveRepairService.UNREPAIRED_SSTABLE);)
+ {
+ int lineNumber = 1;
+ DecoratedKey prevStoredKey = null;
- int lineNumber = 1;
- DecoratedKey prevStoredKey = null;
+ parser.nextToken(); // START_ARRAY
+ while (parser.nextToken() != null)
+ {
+ String key = parser.getCurrentName();
+ Map<?, ?> row = parser.readValueAs(new TypeReference<Map<?, ?>>(){});
+ DecoratedKey currentKey = partitioner.decorateKey(getKeyValidator(columnFamily).fromString((String) row.get("key")));
- parser.nextToken(); // START_ARRAY
- while (parser.nextToken() != null)
- {
- String key = parser.getCurrentName();
- Map<?, ?> row = parser.readValueAs(new TypeReference<Map<?, ?>>(){});
- DecoratedKey currentKey = partitioner.decorateKey(getKeyValidator(columnFamily).fromString((String) row.get("key")));
+ if (row.containsKey("metadata"))
+ parseMeta((Map<?, ?>) row.get("metadata"), columnFamily, null);
- if (row.containsKey("metadata"))
- parseMeta((Map<?, ?>) row.get("metadata"), columnFamily, null);
+ addColumnsToCF((List<?>) row.get("cells"), columnFamily);
- addColumnsToCF((List<?>) row.get("cells"), columnFamily);
+ if (prevStoredKey != null && prevStoredKey.compareTo(currentKey) != -1)
+ {
+ System.err
+ .printf("Line %d: Key %s is greater than previous, collection is not sorted properly. Aborting import. You might need to delete SSTables manually.%n",
+ lineNumber, key);
+ return -1;
+ }
- if (prevStoredKey != null && prevStoredKey.compareTo(currentKey) != -1)
- {
- System.err
- .printf("Line %d: Key %s is greater than previous, collection is not sorted properly. Aborting import. You might need to delete SSTables manually.%n",
- lineNumber, key);
- return -1;
- }
+ // saving decorated key
+ writer.append(currentKey, columnFamily);
+ columnFamily.clear();
- // saving decorated key
- writer.append(currentKey, columnFamily);
- columnFamily.clear();
+ prevStoredKey = currentKey;
+ importedKeys++;
+ lineNumber++;
- prevStoredKey = currentKey;
- importedKeys++;
- lineNumber++;
+ long current = System.nanoTime();
- long current = System.nanoTime();
+ if (TimeUnit.NANOSECONDS.toSeconds(current - start) >= 5) // 5 secs.
+ {
+ System.out.printf("Currently imported %d keys.%n", importedKeys);
+ start = current;
+ }
+
+ if (keyCountToImport == importedKeys)
+ break;
- if (TimeUnit.NANOSECONDS.toSeconds(current - start) >= 5) // 5 secs.
- {
- System.out.printf("Currently imported %d keys.%n", importedKeys);
- start = current;
}
- if (keyCountToImport == importedKeys)
- break;
+ writer.finish(true);
+ return importedKeys;
}
-
- writer.closeAndOpenReader();
-
- return importedKeys;
}
/**
@@ -511,7 +512,7 @@ public class SSTableImport
try
{
- new SSTableImport(keyCountToImport, isSorted).importJson(json, keyspace, cfamily, ssTable);
+ new SSTableImport(keyCountToImport, isSorted).importJson(json, keyspace, cfamily, ssTable);
}
catch (Exception e)
{
@@ -527,7 +528,7 @@ public class SSTableImport
private static void printProgramUsage()
{
System.out.printf("Usage: %s -s -K <keyspace> -c <column_family> -n <num_keys> <json> <sstable>%n%n",
- SSTableImport.class.getName());
+ SSTableImport.class.getName());
System.out.println("Options:");
for (Object o : options.getOptions())
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/utils/AlwaysPresentFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/AlwaysPresentFilter.java b/src/java/org/apache/cassandra/utils/AlwaysPresentFilter.java
index a7f6fce..44d8f24 100644
--- a/src/java/org/apache/cassandra/utils/AlwaysPresentFilter.java
+++ b/src/java/org/apache/cassandra/utils/AlwaysPresentFilter.java
@@ -35,6 +35,11 @@ public class AlwaysPresentFilter implements IFilter
return this;
}
+ public Throwable close(Throwable accumulate)
+ {
+ return accumulate;
+ }
+
public long serializedSize() { return 0; }
@Override
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/utils/Throwables.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/Throwables.java b/src/java/org/apache/cassandra/utils/Throwables.java
index 552ca87..0a2bd28 100644
--- a/src/java/org/apache/cassandra/utils/Throwables.java
+++ b/src/java/org/apache/cassandra/utils/Throwables.java
@@ -29,4 +29,9 @@ public class Throwables
return existingFail;
}
+ public static void maybeFail(Throwable fail)
+ {
+ if (fail != null)
+ com.google.common.base.Throwables.propagate(fail);
+ }
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/utils/concurrent/Ref.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/concurrent/Ref.java b/src/java/org/apache/cassandra/utils/concurrent/Ref.java
index 25f8510..ebabd79 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/Ref.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/Ref.java
@@ -11,12 +11,14 @@ import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
-import com.google.common.base.Throwables;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.cassandra.concurrent.NamedThreadFactory;
+import static org.apache.cassandra.utils.Throwables.maybeFail;
+import static org.apache.cassandra.utils.Throwables.merge;
+
/**
* An object that needs ref counting does the two following:
* - defines a Tidy object that will cleanup once it's gone,
@@ -77,14 +79,19 @@ public final class Ref<T> implements RefCounted<T>, AutoCloseable
state.release(false);
}
+ public Throwable ensureReleased(Throwable accumulate)
+ {
+ return state.ensureReleased(accumulate);
+ }
+
public void ensureReleased()
{
- state.ensureReleased();
+ maybeFail(state.ensureReleased(null));
}
public void close()
{
- state.ensureReleased();
+ ensureReleased();
}
public T get()
@@ -150,14 +157,15 @@ public final class Ref<T> implements RefCounted<T>, AutoCloseable
assert released == 0;
}
- void ensureReleased()
+ Throwable ensureReleased(Throwable accumulate)
{
if (releasedUpdater.getAndSet(this, 1) == 0)
{
- globalState.release(this);
+ accumulate = globalState.release(this, accumulate);
if (DEBUG_ENABLED)
debug.deallocate();
}
+ return accumulate;
}
void release(boolean leak)
@@ -174,7 +182,7 @@ public final class Ref<T> implements RefCounted<T>, AutoCloseable
}
return;
}
- globalState.release(this);
+ Throwable fail = globalState.release(this, null);
if (leak)
{
String id = this.toString();
@@ -186,6 +194,8 @@ public final class Ref<T> implements RefCounted<T>, AutoCloseable
{
debug.deallocate();
}
+ if (fail != null)
+ logger.error("Error when closing {}", globalState, fail);
}
}
@@ -264,7 +274,7 @@ public final class Ref<T> implements RefCounted<T>, AutoCloseable
}
// release a single reference, and cleanup if no more are extant
- void release(Ref.State ref)
+ Throwable release(Ref.State ref, Throwable accumulate)
{
locallyExtant.remove(ref);
if (-1 == counts.decrementAndGet())
@@ -276,10 +286,10 @@ public final class Ref<T> implements RefCounted<T>, AutoCloseable
}
catch (Throwable t)
{
- logger.error("Error when closing {}", this, t);
- Throwables.propagate(t);
+ accumulate = merge(accumulate, t);
}
}
+ return accumulate;
}
int count()
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/utils/concurrent/Refs.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/concurrent/Refs.java b/src/java/org/apache/cassandra/utils/concurrent/Refs.java
index 1c6486e..dd65971 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/Refs.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/Refs.java
@@ -9,6 +9,7 @@ import com.google.common.base.Throwables;
import com.google.common.collect.Iterables;
import com.google.common.collect.Iterators;
+import static org.apache.cassandra.utils.Throwables.maybeFail;
import static org.apache.cassandra.utils.Throwables.merge;
/**
@@ -204,7 +205,10 @@ public final class Refs<T extends RefCounted<T>> extends AbstractCollection<T> i
public static void release(Iterable<? extends Ref<?>> refs)
{
- Throwable fail = null;
+ maybeFail(release(refs, null));
+ }
+ public static Throwable release(Iterable<? extends Ref<?>> refs, Throwable accumulate)
+ {
for (Ref ref : refs)
{
try
@@ -213,11 +217,10 @@ public final class Refs<T extends RefCounted<T>> extends AbstractCollection<T> i
}
catch (Throwable t)
{
- fail = merge(fail, t);
+ accumulate = merge(accumulate, t);
}
}
- if (fail != null)
- throw Throwables.propagate(fail);
+ return accumulate;
}
public static <T extends SelfRefCounted<T>> Iterable<Ref<T>> selfRefs(Iterable<T> refs)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/utils/concurrent/SharedCloseable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/concurrent/SharedCloseable.java b/src/java/org/apache/cassandra/utils/concurrent/SharedCloseable.java
index 1e5a026..a3a1863 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/SharedCloseable.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/SharedCloseable.java
@@ -31,5 +31,6 @@ public interface SharedCloseable extends AutoCloseable
* Throws an exception if the shared resource has already been closed.
*/
public SharedCloseable sharedCopy();
+ public Throwable close(Throwable accumulate);
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/utils/concurrent/SharedCloseableImpl.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/concurrent/SharedCloseableImpl.java b/src/java/org/apache/cassandra/utils/concurrent/SharedCloseableImpl.java
index 0d3a843..d85fd54 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/SharedCloseableImpl.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/SharedCloseableImpl.java
@@ -44,4 +44,9 @@ public abstract class SharedCloseableImpl implements SharedCloseable
{
ref.ensureReleased();
}
+
+ public Throwable close(Throwable accumulate)
+ {
+ return ref.ensureReleased(accumulate);
+ }
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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
new file mode 100644
index 0000000..bcf5095
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/concurrent/Transactional.java
@@ -0,0 +1,198 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+package org.apache.cassandra.utils.concurrent;
+
+import java.util.Set;
+
+import com.google.common.collect.ImmutableSet;
+
+import static org.apache.cassandra.utils.Throwables.maybeFail;
+import static org.apache.cassandra.utils.Throwables.merge;
+
+/**
+ * An abstraction for Transactional behaviour. An object implementing this interface has a lifetime
+ * of the following pattern:
+ *
+ * Throwable failure = null;
+ * try (Transactional t1, t2 = ...)
+ * {
+ * // do work with t1 and t2
+ * t1.prepareToCommit();
+ * t2.prepareToCommit();
+ * failure = t1.commit(failure);
+ * failure = t2.commit(failure);
+ * }
+ * logger.error(failure);
+ *
+ * If something goes wrong before commit() is called on any transaction, then on exiting the try block
+ * the auto close method should invoke cleanup() and then abort() to reset any state.
+ * 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
+ * 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.
+ */
+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.
+ *
+ * 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
+ {
+ public static enum State
+ {
+ IN_PROGRESS,
+ READY_TO_COMMIT,
+ COMMITTED,
+ ABORTED;
+ }
+
+ private State state = State.IN_PROGRESS;
+
+ // the methods for actually performing the necessary behaviours, that are themselves protected against
+ // improper use by the external implementations provided by this class. empty default implementations
+ // could be provided, but we consider it safer to force implementers to consider explicitly their presence
+
+ protected abstract Throwable doCommit(Throwable accumulate);
+ protected abstract Throwable doAbort(Throwable accumulate);
+
+ // this only needs to perform cleanup of state unique to this instance; any internal
+ // Transactional objects will perform cleanup in the commit() or abort() calls
+ protected abstract Throwable doCleanup(Throwable accumulate);
+
+ /**
+ * Do any preparatory work prior to commit. This method should throw any exceptions that can be encountered
+ * during the finalization of the behaviour.
+ */
+ protected abstract void doPrepare();
+
+ /**
+ * commit any effects of this transaction object graph, then cleanup; delegates first to doCommit, then to doCleanup
+ */
+ public final Throwable commit(Throwable accumulate)
+ {
+ if (state != State.READY_TO_COMMIT)
+ throw new IllegalStateException("Commit attempted before prepared to commit");
+ accumulate = doCommit(accumulate);
+ accumulate = doCleanup(accumulate);
+ state = State.COMMITTED;
+ return accumulate;
+ }
+
+ /**
+ * rollback any effects of this transaction object graph; delegates first to doCleanup, then to doAbort
+ */
+ public final Throwable abort(Throwable accumulate)
+ {
+ if (state == State.ABORTED)
+ return accumulate;
+ if (state == State.COMMITTED)
+ {
+ try
+ {
+ throw new IllegalStateException("Attempted to abort a committed operation");
+ }
+ catch (Throwable t)
+ {
+ accumulate = merge(accumulate, t);
+ }
+ return accumulate;
+ }
+ state = State.ABORTED;
+ // we cleanup first so that, e.g., file handles can be released prior to deletion
+ accumulate = doCleanup(accumulate);
+ accumulate = doAbort(accumulate);
+ return accumulate;
+ }
+
+ // if we are committed or aborted, then we are done; otherwise abort
+ public final void close()
+ {
+ switch (state)
+ {
+ case COMMITTED:
+ case ABORTED:
+ break;
+ default:
+ abort();
+ }
+ }
+
+ /**
+ * The first phase of commit: delegates to doPrepare(), with valid state transition enforcement.
+ * This call should be propagated onto any child objects participating in the transaction
+ */
+ public final void prepareToCommit()
+ {
+ if (state != State.IN_PROGRESS)
+ throw new IllegalStateException("Cannot prepare to commit unless IN_PROGRESS; state is " + state);
+
+ doPrepare();
+ state = State.READY_TO_COMMIT;
+ }
+
+ /**
+ * convenience method to both prepareToCommit() and commit() in one operation;
+ * only of use to outer-most transactional object of an object graph
+ */
+ public Object finish()
+ {
+ prepareToCommit();
+ commit();
+ return this;
+ }
+
+ // convenience method wrapping abort, and throwing any exception encountered
+ // only of use to (and to be used by) outer-most object in a transactional graph
+ public final void abort()
+ {
+ maybeFail(abort(null));
+ }
+
+ // convenience method wrapping commit, and throwing any exception encountered
+ // only of use to (and to be used by) outer-most object in a transactional graph
+ public final void commit()
+ {
+ maybeFail(commit(null));
+ }
+
+ public final State state()
+ {
+ return state;
+ }
+ }
+
+ // commit should generally never throw an exception, and preferably never generate one,
+ // 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);
+
+ // release any resources, then rollback all state changes (unless commit() has already been invoked)
+ public Throwable abort(Throwable accumulate);
+
+ public void prepareToCommit();
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 128d1b0..09121f4 100644
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@ -336,7 +336,7 @@ public class ScrubTest
writer.append(Util.dk("c"), cf);
writer.append(Util.dk("y"), cf);
writer.append(Util.dk("d"), cf);
- writer.closeAndOpenReader();
+ writer.finish();
*/
String root = System.getProperty("corrupt-sstable-root");
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 231b3f3..1dc72ae 100644
--- a/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
@@ -156,11 +156,12 @@ public class AntiCompactionTest
File dir = cfs.directories.getDirectoryForNewSSTables();
String filename = cfs.getTempSSTablePath(dir);
- SSTableWriter writer = SSTableWriter.create(filename, 0, 0);
-
- for (int i = 0; i < count * 5; i++)
- writer.append(StorageService.getPartitioner().decorateKey(ByteBufferUtil.bytes(i)), cf);
- return writer.closeAndOpenReader();
+ try (SSTableWriter writer = SSTableWriter.create(filename, 0, 0);)
+ {
+ for (int i = 0; i < count * 5; i++)
+ writer.append(StorageService.getPartitioner().decorateKey(ByteBufferUtil.bytes(i)), cf);
+ return writer.finish(true);
+ }
}
public void generateSStable(ColumnFamilyStore store, String Suffix)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 42ea0c7..18418e8 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
@@ -240,9 +240,9 @@ public class CompactionsTest
long newSize1 = it.next().uncompressedLength();
long newSize2 = it.next().uncompressedLength();
assertEquals("candidate sstable should not be tombstone-compacted because its key range overlap with other sstable",
- originalSize1, newSize1);
+ originalSize1, newSize1);
assertEquals("candidate sstable should not be tombstone-compacted because its key range overlap with other sstable",
- originalSize2, newSize2);
+ originalSize2, newSize2);
// now let's enable the magic property
store.metadata.compactionStrategyOptions.put("unchecked_tombstone_compaction", "true");
@@ -401,21 +401,24 @@ 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);
-
+ try (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);
- writer.append(Util.dk("0"), cf);
- writer.append(Util.dk("1"), cf);
- writer.append(Util.dk("3"), cf);
+ cfs.addSSTable(writer.finish(true));
+ }
- cfs.addSSTable(writer.closeAndOpenReader());
- writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(dir.getDirectoryForNewSSTables())), 0, 0, 0);
+ try (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("2"), cf);
+ writer.append(Util.dk("3"), cf);
+ cfs.addSSTable(writer.finish(true));
+ }
- 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;
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/test/unit/org/apache/cassandra/io/RandomAccessReaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/RandomAccessReaderTest.java b/test/unit/org/apache/cassandra/io/RandomAccessReaderTest.java
index 678b926..fe04096 100644
--- a/test/unit/org/apache/cassandra/io/RandomAccessReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/RandomAccessReaderTest.java
@@ -30,7 +30,7 @@ public class RandomAccessReaderTest
SequentialWriter writer = new SequentialWriter(f, CompressionParameters.DEFAULT_CHUNK_LENGTH, false);
writer.write(expected.getBytes());
- writer.close();
+ writer.finish();
assert f.exists();
@@ -58,7 +58,7 @@ public class RandomAccessReaderTest
SequentialWriter writer = new SequentialWriter(f, CompressionParameters.DEFAULT_CHUNK_LENGTH, false);
writer.write(expected.getBytes());
- writer.close();
+ writer.finish();
assert f.exists();
@@ -87,7 +87,7 @@ public class RandomAccessReaderTest
SequentialWriter writer = new SequentialWriter(f, CompressionParameters.DEFAULT_CHUNK_LENGTH, false);
for (int i = 0; i < numIterations; i++)
writer.write(expected.getBytes());
- writer.close();
+ writer.finish();
assert f.exists();
@@ -166,7 +166,7 @@ public class RandomAccessReaderTest
SequentialWriter writer = new SequentialWriter(f, CompressionParameters.DEFAULT_CHUNK_LENGTH, false);
for (int i = 0; i < expected.length; i++)
writer.write(expected[i].getBytes());
- writer.close();
+ writer.finish();
assert f.exists();
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
index f4d3e87..cfc4bb8 100644
--- a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
@@ -81,7 +81,7 @@ public class CompressedRandomAccessReaderTest
for (int i = 0; i < 20; i++)
writer.write("x".getBytes());
- writer.close();
+ writer.finish();
CompressedRandomAccessReader reader = CompressedRandomAccessReader.open(channel, new CompressionMetadata(filename + ".metadata", f.length()));
String res = reader.readLine();
@@ -124,7 +124,7 @@ public class CompressedRandomAccessReaderTest
writer.resetAndTruncate(mark);
writer.write("brown fox jumps over the lazy dog".getBytes());
- writer.close();
+ writer.finish();
assert f.exists();
RandomAccessReader reader = compressed
@@ -161,10 +161,11 @@ public class CompressedRandomAccessReaderTest
metadata.deleteOnExit();
MetadataCollector sstableMetadataCollector = new MetadataCollector(new SimpleDenseCellNameType(BytesType.instance)).replayPosition(null);
- SequentialWriter writer = new CompressedSequentialWriter(file, metadata.getPath(), new CompressionParameters(SnappyCompressor.instance), sstableMetadataCollector);
-
- writer.write(CONTENT.getBytes());
- writer.close();
+ try (SequentialWriter writer = new CompressedSequentialWriter(file, metadata.getPath(), new CompressionParameters(SnappyCompressor.instance), sstableMetadataCollector))
+ {
+ writer.write(CONTENT.getBytes());
+ writer.finish();
+ }
ChannelProxy channel = new ChannelProxy(file);
@@ -175,8 +176,6 @@ public class CompressedRandomAccessReaderTest
RandomAccessReader reader = CompressedRandomAccessReader.open(channel, meta);
// read and verify compressed data
assertEquals(CONTENT, reader.readLine());
- // close reader
- reader.close();
Random random = new Random();
RandomAccessFile checksumModifier = null;
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java b/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
index 46da343..184319f 100644
--- a/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
@@ -17,23 +17,31 @@
*/
package org.apache.cassandra.io.compress;
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
import java.io.File;
import java.io.IOException;
import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.Random;
+import java.util.*;
+import static org.apache.commons.io.FileUtils.readFileToByteArray;
import static org.junit.Assert.assertEquals;
+
+import org.junit.After;
import org.junit.Test;
+import junit.framework.Assert;
+import org.apache.cassandra.db.composites.CellNames;
import org.apache.cassandra.db.composites.SimpleDenseCellNameType;
import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.UTF8Type;
import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
import org.apache.cassandra.io.util.ChannelProxy;
import org.apache.cassandra.io.util.FileMark;
import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.io.util.SequentialWriterTest;
-public class CompressedSequentialWriterTest
+public class CompressedSequentialWriterTest extends SequentialWriterTest
{
private ICompressor compressor;
@@ -78,30 +86,31 @@ public class CompressedSequentialWriterTest
try
{
MetadataCollector sstableMetadataCollector = new MetadataCollector(new SimpleDenseCellNameType(BytesType.instance)).replayPosition(null);
- CompressedSequentialWriter writer = new CompressedSequentialWriter(f, filename + ".metadata", new CompressionParameters(compressor), sstableMetadataCollector);
-
byte[] dataPre = new byte[bytesToTest];
byte[] rawPost = new byte[bytesToTest];
- Random r = new Random();
-
- // Test both write with byte[] and ByteBuffer
- r.nextBytes(dataPre);
- r.nextBytes(rawPost);
- ByteBuffer dataPost = makeBB(bytesToTest);
- dataPost.put(rawPost);
- dataPost.flip();
-
- writer.write(dataPre);
- FileMark mark = writer.mark();
-
- // Write enough garbage to transition chunk
- for (int i = 0; i < CompressionParameters.DEFAULT_CHUNK_LENGTH; i++)
+ try (CompressedSequentialWriter writer = new CompressedSequentialWriter(f, filename + ".metadata", new CompressionParameters(compressor), sstableMetadataCollector);)
{
- writer.write((byte)i);
+ Random r = new Random();
+
+ // Test both write with byte[] and ByteBuffer
+ r.nextBytes(dataPre);
+ r.nextBytes(rawPost);
+ ByteBuffer dataPost = makeBB(bytesToTest);
+ dataPost.put(rawPost);
+ dataPost.flip();
+
+ writer.write(dataPre);
+ FileMark mark = writer.mark();
+
+ // Write enough garbage to transition chunk
+ for (int i = 0; i < CompressionParameters.DEFAULT_CHUNK_LENGTH; i++)
+ {
+ writer.write((byte)i);
+ }
+ writer.resetAndTruncate(mark);
+ writer.write(dataPost);
+ writer.finish();
}
- writer.resetAndTruncate(mark);
- writer.write(dataPost);
- writer.close();
assert f.exists();
RandomAccessReader reader = CompressedRandomAccessReader.open(channel, new CompressionMetadata(filename + ".metadata", f.length()));
@@ -137,4 +146,85 @@ public class CompressedSequentialWriterTest
? ByteBuffer.allocateDirect(size)
: ByteBuffer.allocate(size);
}
+
+ private final List<TestableCSW> writers = new ArrayList<>();
+
+ @After
+ public void cleanup()
+ {
+ for (TestableCSW sw : writers)
+ sw.cleanup();
+ writers.clear();
+ }
+
+ protected TestableTransaction newTest() throws IOException
+ {
+ TestableCSW sw = new TestableCSW();
+ writers.add(sw);
+ return sw;
+ }
+
+ private static class TestableCSW extends TestableSW
+ {
+ final File offsetsFile;
+
+ private TestableCSW() throws IOException
+ {
+ this(tempFile("compressedsequentialwriter"),
+ tempFile("compressedsequentialwriter.offsets"));
+ }
+
+ private TestableCSW(File file, File offsetsFile) throws IOException
+ {
+ this(file, offsetsFile, new CompressedSequentialWriter(file, offsetsFile.getPath(), new CompressionParameters(LZ4Compressor.instance, BUFFER_SIZE, new HashMap<String, String>()), new MetadataCollector(CellNames.fromAbstractType(UTF8Type.instance, false))));
+ }
+
+ private TestableCSW(File file, File offsetsFile, CompressedSequentialWriter sw) throws IOException
+ {
+ super(file, sw);
+ this.offsetsFile = offsetsFile;
+ }
+
+ protected void assertInProgress() throws Exception
+ {
+ Assert.assertTrue(file.exists());
+ Assert.assertFalse(offsetsFile.exists());
+ byte[] compressed = readFileToByteArray(file);
+ byte[] uncompressed = new byte[partialContents.length];
+ LZ4Compressor.instance.uncompress(compressed, 0, compressed.length - 4, uncompressed, 0);
+ Assert.assertTrue(Arrays.equals(partialContents, uncompressed));
+ }
+
+ protected void assertPrepared() throws Exception
+ {
+ Assert.assertTrue(file.exists());
+ Assert.assertTrue(offsetsFile.exists());
+ DataInputStream offsets = new DataInputStream(new ByteArrayInputStream(readFileToByteArray(offsetsFile)));
+ Assert.assertTrue(offsets.readUTF().endsWith("LZ4Compressor"));
+ Assert.assertEquals(0, offsets.readInt());
+ Assert.assertEquals(BUFFER_SIZE, offsets.readInt());
+ Assert.assertEquals(fullContents.length, offsets.readLong());
+ Assert.assertEquals(2, offsets.readInt());
+ Assert.assertEquals(0, offsets.readLong());
+ int offset = (int) offsets.readLong();
+ byte[] compressed = readFileToByteArray(file);
+ byte[] uncompressed = new byte[fullContents.length];
+ LZ4Compressor.instance.uncompress(compressed, 0, offset - 4, uncompressed, 0);
+ LZ4Compressor.instance.uncompress(compressed, offset, compressed.length - (4 + offset), uncompressed, partialContents.length);
+ Assert.assertTrue(Arrays.equals(fullContents, uncompressed));
+ }
+
+ protected void assertAborted() throws Exception
+ {
+ super.assertAborted();
+ Assert.assertFalse(offsetsFile.exists());
+ }
+
+ void cleanup()
+ {
+ file.delete();
+ offsetsFile.delete();
+ }
+ }
+
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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
new file mode 100644
index 0000000..dfb55a1
--- /dev/null
+++ b/test/unit/org/apache/cassandra/io/sstable/BigTableWriterTest.java
@@ -0,0 +1,130 @@
+/*
+* 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.io.IOException;
+
+import org.junit.After;
+import org.junit.BeforeClass;
+
+import junit.framework.Assert;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.db.ArrayBackedSortedColumns;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.concurrent.AbstractTransactionalTest;
+
+public class BigTableWriterTest extends AbstractTransactionalTest
+{
+ public static final String KEYSPACE1 = "BigTableWriterTest";
+ public static final String CF_STANDARD = "Standard1";
+
+ private static ColumnFamilyStore cfs;
+
+ @BeforeClass
+ public static void defineSchema() throws Exception
+ {
+ SchemaLoader.prepareServer();
+ SchemaLoader.createKeyspace(KEYSPACE1,
+ SimpleStrategy.class,
+ KSMetaData.optsWithRF(1),
+ SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD));
+ cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD);
+ }
+
+ protected TestableTransaction newTest() throws IOException
+ {
+ return new TestableBTW();
+ }
+
+ private static class TestableBTW extends TestableTransaction
+ {
+ final File file;
+ final Descriptor descriptor;
+ final SSTableWriter writer;
+
+ private TestableBTW() throws IOException
+ {
+ this(cfs.getTempSSTablePath(cfs.directories.getDirectoryForNewSSTables()));
+ }
+
+ private TestableBTW(String file) throws IOException
+ {
+ this(file, SSTableWriter.create(file, 0, 0));
+ }
+
+ private TestableBTW(String file, SSTableWriter sw) throws IOException
+ {
+ super(sw);
+ this.file = new File(file);
+ this.descriptor = Descriptor.fromFilename(file);
+ this.writer = sw;
+ ArrayBackedSortedColumns cf = ArrayBackedSortedColumns.factory.create(cfs.metadata);
+ for (int i = 0; i < 10; i++)
+ cf.addColumn(Util.cellname(i), SSTableRewriterTest.random(0, 1000), 1);
+ for (int i = 0; i < 100; i++)
+ writer.append(StorageService.getPartitioner().decorateKey(ByteBufferUtil.bytes(i)), cf);
+ }
+
+ 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);
+ 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);
+ }
+
+ 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);
+ Assert.assertFalse(file.exists());
+ }
+
+ protected void assertCommitted() throws Exception
+ {
+ assertPrepared();
+ }
+
+ private void assertExists(Descriptor.Type type, Component ... components)
+ {
+ for (Component component : components)
+ Assert.assertTrue(new File(descriptor.asType(type).filenameFor(component)).exists());
+ }
+ private void assertNotExists(Descriptor.Type type, Component ... components)
+ {
+ for (Component component : components)
+ Assert.assertFalse(type.toString() + " " + component.toString(), new File(descriptor.asType(type).filenameFor(component)).exists());
+ }
+ }
+
+}
[4/7] cassandra git commit: Introduce Transactional API for internal
state changes
Posted by be...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 3104a96..d39da61 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
@@ -52,6 +52,7 @@ import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.Pair;
import static org.junit.Assert.*;
+import static org.apache.cassandra.utils.Throwables.maybeFail;
public class SSTableRewriterTest extends SchemaLoader
{
@@ -93,8 +94,8 @@ public class SSTableRewriterTest extends SchemaLoader
cfs.forceBlockingFlush();
Set<SSTableReader> sstables = new HashSet<>(cfs.getSSTables());
assertEquals(1, sstables.size());
- SSTableRewriter writer = new SSTableRewriter(cfs, sstables, 1000, false);
- try (AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables);)
+ try (SSTableRewriter writer = new SSTableRewriter(cfs, sstables, 1000, false);
+ AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables);)
{
ISSTableScanner scanner = scanners.scanners.get(0);
CompactionController controller = new CompactionController(cfs, sstables, cfs.gcBefore(System.currentTimeMillis()));
@@ -104,9 +105,9 @@ public class SSTableRewriterTest extends SchemaLoader
AbstractCompactedRow row = new LazilyCompactedRow(controller, Arrays.asList(scanner.next()));
writer.append(row);
}
+ Collection<SSTableReader> newsstables = writer.finish();
+ cfs.getDataTracker().markCompactedSSTablesReplaced(sstables, newsstables , OperationType.COMPACTION);
}
- Collection<SSTableReader> newsstables = writer.finish();
- cfs.getDataTracker().markCompactedSSTablesReplaced(sstables, newsstables, OperationType.COMPACTION);
SSTableDeletingTask.waitForDeletions();
validateCFS(cfs);
@@ -126,8 +127,8 @@ public class SSTableRewriterTest extends SchemaLoader
Set<SSTableReader> sstables = new HashSet<>(cfs.getSSTables());
assertEquals(1, sstables.size());
SSTableRewriter.overrideOpenInterval(10000000);
- SSTableRewriter writer = new SSTableRewriter(cfs, sstables, 1000, false);
- try (AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables);)
+ try (SSTableRewriter writer = new SSTableRewriter(cfs, sstables, 1000, false);
+ AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables);)
{
ISSTableScanner scanner = scanners.scanners.get(0);
CompactionController controller = new CompactionController(cfs, sstables, cfs.gcBefore(System.currentTimeMillis()));
@@ -137,9 +138,9 @@ public class SSTableRewriterTest extends SchemaLoader
AbstractCompactedRow row = new LazilyCompactedRow(controller, Arrays.asList(scanner.next()));
writer.append(row);
}
+ Collection<SSTableReader> newsstables = writer.finish();
+ cfs.getDataTracker().markCompactedSSTablesReplaced(sstables, newsstables, OperationType.COMPACTION);
}
- Collection<SSTableReader> newsstables = writer.finish();
- cfs.getDataTracker().markCompactedSSTablesReplaced(sstables, newsstables, OperationType.COMPACTION);
SSTableDeletingTask.waitForDeletions();
validateCFS(cfs);
@@ -159,9 +160,9 @@ public class SSTableRewriterTest extends SchemaLoader
Set<SSTableReader> sstables = new HashSet<>(cfs.getSSTables());
assertEquals(1, sstables.size());
SSTableRewriter.overrideOpenInterval(10000000);
- SSTableRewriter writer = new SSTableRewriter(cfs, sstables, 1000, false);
boolean checked = false;
- try (AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables);)
+ try (SSTableRewriter writer = new SSTableRewriter(cfs, sstables, 1000, false);
+ AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables);)
{
ISSTableScanner scanner = scanners.scanners.get(0);
CompactionController controller = new CompactionController(cfs, sstables, cfs.gcBefore(System.currentTimeMillis()));
@@ -191,10 +192,10 @@ public class SSTableRewriterTest extends SchemaLoader
}
}
}
+ assertTrue(checked);
+ Collection<SSTableReader> newsstables = writer.finish();
+ cfs.getDataTracker().markCompactedSSTablesReplaced(sstables, newsstables, OperationType.COMPACTION);
}
- assertTrue(checked);
- Collection<SSTableReader> newsstables = writer.finish();
- cfs.getDataTracker().markCompactedSSTablesReplaced(sstables, newsstables, OperationType.COMPACTION);
SSTableDeletingTask.waitForDeletions();
validateCFS(cfs);
@@ -216,17 +217,17 @@ public class SSTableRewriterTest extends SchemaLoader
for (int i = 0; i < 100; i++)
cf.addColumn(Util.cellname(i), ByteBuffer.allocate(1000), 1);
File dir = cfs.directories.getDirectoryForNewSSTables();
- SSTableWriter writer = getWriter(cfs, dir);
- try
+
+ try (SSTableWriter writer = getWriter(cfs, dir);)
{
for (int i = 0; i < 10000; i++)
writer.append(StorageService.getPartitioner().decorateKey(random(i, 10)), cf);
- SSTableReader s = writer.openEarly(1000);
+ SSTableReader s = writer.setMaxDataAge(1000).openEarly();
assert s != null;
assertFileCounts(dir.list(), 2, 2);
for (int i = 10000; i < 20000; i++)
writer.append(StorageService.getPartitioner().decorateKey(random(i, 10)), cf);
- SSTableReader s2 = writer.openEarly(1000);
+ SSTableReader s2 = writer.setMaxDataAge(1000).openEarly();
assertTrue(s.last.compareTo(s2.last) < 0);
assertFileCounts(dir.list(), 2, 2);
s.markObsolete();
@@ -245,11 +246,6 @@ public class SSTableRewriterTest extends SchemaLoader
assertEquals(datafiles, 0);
validateCFS(cfs);
}
- catch (Throwable t)
- {
- writer.abort();
- throw t;
- }
}
@Test
@@ -264,13 +260,14 @@ public class SSTableRewriterTest extends SchemaLoader
long startStorageMetricsLoad = StorageMetrics.load.getCount();
Set<SSTableReader> compacting = Sets.newHashSet(s);
SSTableRewriter.overrideOpenInterval(10000000);
- SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
- rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
+ List<SSTableReader> sstables;
int files = 1;
- try (ISSTableScanner scanner = s.getScanner();
+ try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
+ ISSTableScanner scanner = s.getScanner();
CompactionController controller = new CompactionController(cfs, compacting, 0))
{
+ rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
while(scanner.hasNext())
{
rewriter.append(new LazilyCompactedRow(controller, Arrays.asList(scanner.next())));
@@ -284,9 +281,10 @@ public class SSTableRewriterTest extends SchemaLoader
}
}
+ sstables = rewriter.finish();
+ cfs.getDataTracker().markCompactedSSTablesReplaced(compacting, sstables, OperationType.COMPACTION);
+ assertEquals(files, sstables.size());
}
- List<SSTableReader> sstables = rewriter.finish();
- cfs.getDataTracker().markCompactedSSTablesReplaced(compacting, sstables, OperationType.COMPACTION);
long sum = 0;
for (SSTableReader x : cfs.getSSTables())
sum += x.bytesOnDisk();
@@ -314,13 +312,14 @@ public class SSTableRewriterTest extends SchemaLoader
Set<SSTableReader> compacting = Sets.newHashSet(s);
SSTableRewriter.overrideOpenInterval(10000000);
- SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
- rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
+ List<SSTableReader> sstables;
int files = 1;
- try (ISSTableScanner scanner = s.getScanner();
+ try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
+ ISSTableScanner scanner = s.getScanner();
CompactionController controller = new CompactionController(cfs, compacting, 0))
{
+ rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
while(scanner.hasNext())
{
rewriter.append(new LazilyCompactedRow(controller, Arrays.asList(scanner.next())));
@@ -331,14 +330,9 @@ public class SSTableRewriterTest extends SchemaLoader
assertEquals(cfs.getSSTables().size(), files); // we have one original file plus the ones we have switched out.
}
}
- }
- catch (Throwable t)
- {
- rewriter.abort();
- throw t;
+ sstables = rewriter.finish();
}
- List<SSTableReader> sstables = rewriter.finish();
assertEquals(files, sstables.size());
assertEquals(files, cfs.getSSTables().size());
assertEquals(1, cfs.getDataTracker().getView().shadowed.size());
@@ -445,19 +439,14 @@ public class SSTableRewriterTest extends SchemaLoader
long startSize = cfs.metric.liveDiskSpaceUsed.getCount();
Set<SSTableReader> compacting = Sets.newHashSet(s);
SSTableRewriter.overrideOpenInterval(10000000);
- SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
- rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
- try (ISSTableScanner scanner = s.getScanner();
+ try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
+ ISSTableScanner scanner = s.getScanner();
CompactionController controller = new CompactionController(cfs, compacting, 0))
{
+ rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
test.run(scanner, controller, s, cfs, rewriter);
}
- catch (Throwable t)
- {
- rewriter.abort();
- throw t;
- }
SSTableDeletingTask.waitForDeletions();
@@ -481,13 +470,13 @@ public class SSTableRewriterTest extends SchemaLoader
Set<SSTableReader> compacting = Sets.newHashSet(s);
SSTableRewriter.overrideOpenInterval(10000000);
- SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
- rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
int files = 1;
- try (ISSTableScanner scanner = s.getScanner();
+ try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
+ ISSTableScanner scanner = s.getScanner();
CompactionController controller = new CompactionController(cfs, compacting, 0))
{
+ rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
while(scanner.hasNext())
{
rewriter.append(new LazilyCompactedRow(controller, Arrays.asList(scanner.next())));
@@ -506,11 +495,6 @@ public class SSTableRewriterTest extends SchemaLoader
}
}
}
- catch (Throwable t)
- {
- rewriter.abort();
- throw t;
- }
SSTableDeletingTask.waitForDeletions();
@@ -531,13 +515,14 @@ public class SSTableRewriterTest extends SchemaLoader
cfs.addSSTable(s);
Set<SSTableReader> compacting = Sets.newHashSet(s);
SSTableRewriter.overrideOpenInterval(10000000);
- SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
- rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
+ List<SSTableReader> sstables;
int files = 1;
- try (ISSTableScanner scanner = s.getScanner();
+ try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
+ ISSTableScanner scanner = s.getScanner();
CompactionController controller = new CompactionController(cfs, compacting, 0))
{
+ rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
while(scanner.hasNext())
{
rewriter.append(new LazilyCompactedRow(controller, Arrays.asList(scanner.next())));
@@ -548,15 +533,11 @@ public class SSTableRewriterTest extends SchemaLoader
assertEquals(cfs.getSSTables().size(), files); // we have one original file plus the ones we have switched out.
}
}
- }
- catch (Throwable t)
- {
- rewriter.abort();
- throw t;
+
+ sstables = rewriter.finish();
+ cfs.getDataTracker().markCompactedSSTablesReplaced(compacting, sstables, OperationType.COMPACTION);
}
- List<SSTableReader> sstables = rewriter.finish();
- cfs.getDataTracker().markCompactedSSTablesReplaced(compacting, sstables, OperationType.COMPACTION);
SSTableDeletingTask.waitForDeletions();
assertFileCounts(s.descriptor.directory.list(), 0, 0);
cfs.truncateBlocking();
@@ -576,13 +557,14 @@ public class SSTableRewriterTest extends SchemaLoader
cfs.addSSTable(s);
Set<SSTableReader> compacting = Sets.newHashSet(s);
SSTableRewriter.overrideOpenInterval(1000000);
- SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
- rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
+ List<SSTableReader> sstables;
int files = 1;
- try (ISSTableScanner scanner = s.getScanner();
+ try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
+ ISSTableScanner scanner = s.getScanner();
CompactionController controller = new CompactionController(cfs, compacting, 0))
{
+ rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
while(scanner.hasNext())
{
rewriter.append(new LazilyCompactedRow(controller, Arrays.asList(scanner.next())));
@@ -593,15 +575,11 @@ public class SSTableRewriterTest extends SchemaLoader
files++;
}
}
- }
- catch (Throwable t)
- {
- rewriter.abort();
- throw t;
+
+ sstables = rewriter.finish();
+ cfs.getDataTracker().markCompactedSSTablesReplaced(compacting, sstables, OperationType.COMPACTION);
}
- List<SSTableReader> sstables = rewriter.finish();
- cfs.getDataTracker().markCompactedSSTablesReplaced(compacting, sstables, OperationType.COMPACTION);
assertEquals(files, sstables.size());
assertEquals(files, cfs.getSSTables().size());
SSTableDeletingTask.waitForDeletions();
@@ -668,12 +646,12 @@ public class SSTableRewriterTest extends SchemaLoader
Set<SSTableReader> compacting = Sets.newHashSet(s);
cfs.getDataTracker().markCompacting(compacting);
SSTableRewriter.overrideOpenInterval(10000000);
- SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, offline);
- SSTableWriter w = getWriter(cfs, s.descriptor.directory);
- rewriter.switchWriter(w);
- try (ISSTableScanner scanner = compacting.iterator().next().getScanner();
+
+ try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, offline);
+ ISSTableScanner scanner = compacting.iterator().next().getScanner();
CompactionController controller = new CompactionController(cfs, compacting, 0))
{
+ rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
while (scanner.hasNext())
{
rewriter.append(new LazilyCompactedRow(controller, Arrays.asList(scanner.next())));
@@ -684,7 +662,8 @@ public class SSTableRewriterTest extends SchemaLoader
}
try
{
- rewriter.finishAndThrow(earlyException);
+ rewriter.throwDuringPrepare(earlyException);
+ rewriter.prepareToCommit();
}
catch (Throwable t)
{
@@ -749,14 +728,14 @@ public class SSTableRewriterTest extends SchemaLoader
compacting.add(s);
cfs.getDataTracker().markCompacting(compacting);
- SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
+
SSTableRewriter.overrideOpenInterval(1);
- SSTableWriter w = getWriter(cfs, s.descriptor.directory);
- rewriter.switchWriter(w);
int keyCount = 0;
- try (ISSTableScanner scanner = compacting.iterator().next().getScanner();
+ try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
+ ISSTableScanner scanner = compacting.iterator().next().getScanner();
CompactionController controller = new CompactionController(cfs, compacting, 0))
{
+ rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
while (scanner.hasNext())
{
rewriter.append(new LazilyCompactedRow(controller, Arrays.asList(scanner.next())));
@@ -841,11 +820,12 @@ public class SSTableRewriterTest extends SchemaLoader
File dir = cfs.directories.getDirectoryForNewSSTables();
String filename = cfs.getTempSSTablePath(dir);
- SSTableWriter writer = SSTableWriter.create(filename, 0, 0);
-
- for (int i = 0; i < count * 5; i++)
- writer.append(StorageService.getPartitioner().decorateKey(ByteBufferUtil.bytes(i)), cf);
- return writer.closeAndOpenReader();
+ try (SSTableWriter writer = SSTableWriter.create(filename, 0, 0);)
+ {
+ for (int i = 0; i < count * 5; i++)
+ writer.append(StorageService.getPartitioner().decorateKey(ByteBufferUtil.bytes(i)), cf);
+ return writer.finish(true);
+ }
}
private void validateCFS(ColumnFamilyStore cfs)
@@ -899,7 +879,7 @@ public class SSTableRewriterTest extends SchemaLoader
return SSTableWriter.create(filename, 0, 0);
}
- private ByteBuffer random(int i, int size)
+ public static ByteBuffer random(int i, int size)
{
byte[] bytes = new byte[size + 4];
ThreadLocalRandom.current().nextBytes(bytes);
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 d2559cd..a116b84 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
@@ -217,7 +217,7 @@ public class SSTableUtils
File datafile = (dest == null) ? tempSSTableFile(ksname, cfname, generation) : new File(dest.filenameFor(Component.DATA));
SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(datafile.getAbsolutePath()), expectedSize, ActiveRepairService.UNREPAIRED_SSTABLE, 0);
while (appender.append(writer)) { /* pass */ }
- SSTableReader reader = writer.closeAndOpenReader();
+ SSTableReader reader = writer.finish(true);
// mark all components for removal
if (cleanup)
for (Component component : reader.components)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java b/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java
index a40034d..0ff4b01 100644
--- a/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java
+++ b/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java
@@ -128,7 +128,7 @@ public class BufferedRandomAccessFileTest
assert data[i] == 0;
}
- w.close();
+ w.finish();
r.close();
}
@@ -153,7 +153,7 @@ public class BufferedRandomAccessFileTest
assert negone == -1 : "We read past the end of the file, should have gotten EOF -1. Instead, " + negone;
r.close();
- w.close();
+ w.finish();
}
@Test
@@ -178,7 +178,7 @@ public class BufferedRandomAccessFileTest
w.write(biggerThenBuffer);
assertEquals(biggerThenBuffer.length + lessThenBuffer.length, w.length());
- w.close();
+ w.finish();
// will use cachedlength
RandomAccessReader r = RandomAccessReader.open(tmpFile);
@@ -223,7 +223,7 @@ public class BufferedRandomAccessFileTest
}
});
- w.close();
+ w.finish();
r.close();
}
@@ -233,7 +233,7 @@ public class BufferedRandomAccessFileTest
SequentialWriter w = createTempFile("brafSeek");
byte[] data = generateByteArray(RandomAccessReader.DEFAULT_BUFFER_SIZE + 20);
w.write(data);
- w.close();
+ w.finish();
final RandomAccessReader file = RandomAccessReader.open(w);
@@ -272,7 +272,7 @@ public class BufferedRandomAccessFileTest
{
SequentialWriter w = createTempFile("brafSkipBytes");
w.write(generateByteArray(RandomAccessReader.DEFAULT_BUFFER_SIZE * 2));
- w.close();
+ w.finish();
RandomAccessReader file = RandomAccessReader.open(w);
@@ -320,7 +320,7 @@ public class BufferedRandomAccessFileTest
r.read(new byte[4]);
assertEquals(r.getFilePointer(), 20);
- w.close();
+ w.finish();
r.close();
}
@@ -329,7 +329,7 @@ public class BufferedRandomAccessFileTest
{
SequentialWriter file = createTempFile("brafGetPath");
assert file.getPath().contains("brafGetPath");
- file.close();
+ file.finish();
}
@Test
@@ -411,7 +411,7 @@ public class BufferedRandomAccessFileTest
r.skipBytes(10);
assertEquals(r.bytesRemaining(), r.length() - 10);
- w.close();
+ w.finish();
r.close();
}
@@ -443,7 +443,7 @@ public class BufferedRandomAccessFileTest
byte[] data = generateByteArray(RandomAccessReader.DEFAULT_BUFFER_SIZE + 20);
w.write(data);
- w.close(); // will flush
+ w.finish();
final RandomAccessReader r = RandomAccessReader.open(new File(w.getPath()));
@@ -481,7 +481,7 @@ public class BufferedRandomAccessFileTest
SequentialWriter w = createTempFile("brafTestMark");
w.write(new byte[30]);
- w.close();
+ w.finish();
RandomAccessReader file = RandomAccessReader.open(w);
@@ -542,10 +542,10 @@ public class BufferedRandomAccessFileTest
SequentialWriter w2 = createTempFile("fscache2");
w1.write(new byte[30]);
- w1.close();
+ w1.finish();
w2.write(new byte[30]);
- w2.close();
+ w2.finish();
for (int i = 0; i < 20; i++)
{
@@ -652,7 +652,7 @@ public class BufferedRandomAccessFileTest
assertEquals(new String(content), "cccccccccc");
- file.close();
+ file.finish();
copy.close();
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/test/unit/org/apache/cassandra/io/util/ChecksummedSequentialWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/util/ChecksummedSequentialWriterTest.java b/test/unit/org/apache/cassandra/io/util/ChecksummedSequentialWriterTest.java
new file mode 100644
index 0000000..9731a8d
--- /dev/null
+++ b/test/unit/org/apache/cassandra/io/util/ChecksummedSequentialWriterTest.java
@@ -0,0 +1,92 @@
+/*
+* 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.util;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.After;
+
+import junit.framework.Assert;
+
+public class ChecksummedSequentialWriterTest extends SequentialWriterTest
+{
+
+ private final List<TestableCSW> writers = new ArrayList<>();
+
+ @After
+ public void cleanup()
+ {
+ for (TestableSW sw : writers)
+ sw.file.delete();
+ writers.clear();
+ }
+
+ protected TestableTransaction newTest() throws IOException
+ {
+ TestableCSW sw = new TestableCSW();
+ writers.add(sw);
+ return sw;
+ }
+
+ private static class TestableCSW extends TestableSW
+ {
+ final File crcFile;
+
+ private TestableCSW() throws IOException
+ {
+ this(tempFile("compressedsequentialwriter"),
+ tempFile("compressedsequentialwriter.checksum"));
+ }
+
+ private TestableCSW(File file, File crcFile) throws IOException
+ {
+ this(file, crcFile, new ChecksummedSequentialWriter(file, BUFFER_SIZE, crcFile));
+ }
+
+ private TestableCSW(File file, File crcFile, SequentialWriter sw) throws IOException
+ {
+ super(file, sw);
+ this.crcFile = crcFile;
+ }
+
+ protected void assertInProgress() throws Exception
+ {
+ super.assertInProgress();
+ Assert.assertTrue(crcFile.exists());
+ Assert.assertEquals(0, crcFile.length());
+ }
+
+ protected void assertPrepared() throws Exception
+ {
+ super.assertPrepared();
+ Assert.assertTrue(crcFile.exists());
+ Assert.assertFalse(0 == crcFile.length());
+ }
+
+ protected void assertAborted() throws Exception
+ {
+ super.assertAborted();
+ Assert.assertFalse(crcFile.exists());
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/test/unit/org/apache/cassandra/io/util/DataOutputTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/util/DataOutputTest.java b/test/unit/org/apache/cassandra/io/util/DataOutputTest.java
index b15da47..ec280fa 100644
--- a/test/unit/org/apache/cassandra/io/util/DataOutputTest.java
+++ b/test/unit/org/apache/cassandra/io/util/DataOutputTest.java
@@ -180,6 +180,7 @@ public class DataOutputTest
DataOutputStreamPlus write = new WrappedDataOutputStreamPlus(writer);
DataInput canon = testWrite(write);
write.flush();
+ writer.finish();
write.close();
DataInputStream test = new DataInputStream(new FileInputStream(file));
testRead(test, canon);
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/test/unit/org/apache/cassandra/io/util/SequentialWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/util/SequentialWriterTest.java b/test/unit/org/apache/cassandra/io/util/SequentialWriterTest.java
new file mode 100644
index 0000000..ef52030
--- /dev/null
+++ b/test/unit/org/apache/cassandra/io/util/SequentialWriterTest.java
@@ -0,0 +1,117 @@
+/*
+* 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.util;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.ThreadLocalRandom;
+
+import org.junit.After;
+
+import junit.framework.Assert;
+import org.apache.cassandra.utils.concurrent.AbstractTransactionalTest;
+
+import static org.apache.commons.io.FileUtils.*;
+
+public class SequentialWriterTest extends AbstractTransactionalTest
+{
+
+ private final List<TestableSW> writers = new ArrayList<>();
+
+ @After
+ public void cleanup()
+ {
+ for (TestableSW sw : writers)
+ sw.file.delete();
+ writers.clear();
+ }
+
+ protected TestableTransaction newTest() throws IOException
+ {
+ TestableSW sw = new TestableSW();
+ writers.add(sw);
+ return sw;
+ }
+
+ protected static class TestableSW extends TestableTransaction
+ {
+ protected static final int BUFFER_SIZE = 8 << 10;
+ protected final File file;
+ protected final SequentialWriter writer;
+ protected final byte[] fullContents, partialContents;
+
+ protected TestableSW() throws IOException
+ {
+ this(tempFile("sequentialwriter"));
+ }
+
+ protected TestableSW(File file) throws IOException
+ {
+ this(file, new SequentialWriter(file, 8 << 10, true));
+ }
+
+ protected TestableSW(File file, SequentialWriter sw) throws IOException
+ {
+ super(sw);
+ this.file = file;
+ this.writer = sw;
+ fullContents = new byte[BUFFER_SIZE + BUFFER_SIZE / 2];
+ ThreadLocalRandom.current().nextBytes(fullContents);
+ partialContents = Arrays.copyOf(fullContents, BUFFER_SIZE);
+ sw.write(fullContents);
+ }
+
+ protected void assertInProgress() throws Exception
+ {
+ Assert.assertTrue(file.exists());
+ byte[] bytes = readFileToByteArray(file);
+ Assert.assertTrue(Arrays.equals(partialContents, bytes));
+ }
+
+ protected void assertPrepared() throws Exception
+ {
+ Assert.assertTrue(file.exists());
+ byte[] bytes = readFileToByteArray(file);
+ Assert.assertTrue(Arrays.equals(fullContents, bytes));
+ }
+
+ protected void assertAborted() throws Exception
+ {
+ Assert.assertFalse(writer.isOpen());
+ Assert.assertFalse(file.exists());
+ }
+
+ protected void assertCommitted() throws Exception
+ {
+ assertPrepared();
+ Assert.assertFalse(writer.isOpen());
+ }
+
+ protected static File tempFile(String prefix)
+ {
+ File file = FileUtils.createTempFile(prefix, "test");
+ file.delete();
+ return file;
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/test/unit/org/apache/cassandra/streaming/compress/CompressedInputStreamTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/compress/CompressedInputStreamTest.java b/test/unit/org/apache/cassandra/streaming/compress/CompressedInputStreamTest.java
index 016deb3..20371c3 100644
--- a/test/unit/org/apache/cassandra/streaming/compress/CompressedInputStreamTest.java
+++ b/test/unit/org/apache/cassandra/streaming/compress/CompressedInputStreamTest.java
@@ -76,7 +76,7 @@ public class CompressedInputStreamTest
index.put(l, writer.getFilePointer());
writer.stream.writeLong(l);
}
- writer.close();
+ writer.finish();
CompressionMetadata comp = CompressionMetadata.create(tmp.getAbsolutePath());
List<Pair<Long, Long>> sections = new ArrayList<Pair<Long, Long>>();
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/test/unit/org/apache/cassandra/tools/SSTableExportTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/tools/SSTableExportTest.java b/test/unit/org/apache/cassandra/tools/SSTableExportTest.java
index 7e59207..bc73c83 100644
--- a/test/unit/org/apache/cassandra/tools/SSTableExportTest.java
+++ b/test/unit/org/apache/cassandra/tools/SSTableExportTest.java
@@ -109,7 +109,7 @@ public class SSTableExportTest
writer.append(Util.dk("rowB"), cfamily);
cfamily.clear();
- writer.closeAndOpenReader();
+ writer.finish(true);
// Enumerate and verify
File temp = File.createTempFile("Standard1", ".txt");
@@ -153,7 +153,7 @@ public class SSTableExportTest
writer.append(Util.dk("rowExclude"), cfamily);
cfamily.clear();
- SSTableReader reader = writer.closeAndOpenReader();
+ SSTableReader reader = writer.finish(true);
// Export to JSON and verify
File tempJson = File.createTempFile("Standard1", ".json");
@@ -202,7 +202,7 @@ public class SSTableExportTest
writer.append(Util.dk("rowExclude"), cfamily);
cfamily.clear();
- SSTableReader reader = writer.closeAndOpenReader();
+ SSTableReader reader = writer.finish(true);
// Export to JSON and verify
File tempJson = File.createTempFile("Standard1", ".json");
@@ -237,7 +237,7 @@ public class SSTableExportTest
writer.append(Util.dk("rowA"), cfamily);
cfamily.clear();
- SSTableReader reader = writer.closeAndOpenReader();
+ SSTableReader reader = writer.finish(true);
// Export to JSON and verify
File tempJson = File.createTempFile("Counter1", ".json");
@@ -268,7 +268,7 @@ public class SSTableExportTest
writer.append(Util.dk("rowA"), cfamily);
cfamily.clear();
- SSTableReader reader = writer.closeAndOpenReader();
+ SSTableReader reader = writer.finish(true);
// Export to JSON and verify
File tempJson = File.createTempFile("ValuesWithQuotes", ".json");
@@ -300,7 +300,7 @@ public class SSTableExportTest
cfamily.delete(new DeletionInfo(0, 0));
writer.append(Util.dk("rowA"), cfamily);
- SSTableReader reader = writer.closeAndOpenReader();
+ SSTableReader reader = writer.finish(true);
// Export to JSON and verify
File tempJson = File.createTempFile("CFWithDeletionInfo", ".json");
SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[0]);
@@ -359,7 +359,7 @@ public class SSTableExportTest
cfamily.addColumn(column(CFMetaData.DEFAULT_KEY_ALIAS, "not a uuid", 1L));
writer.append(Util.dk(ByteBufferUtil.bytes(UUIDGen.getTimeUUID())), cfamily);
- SSTableReader reader = writer.closeAndOpenReader();
+ SSTableReader reader = writer.finish(true);
// Export to JSON and verify
File tempJson = File.createTempFile("CFWithColumnNameEqualToDefaultKeyAlias", ".json");
SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[0]);
@@ -388,7 +388,7 @@ public class SSTableExportTest
cfamily.addColumn(column("column", "value", 1L));
writer.append(Util.dk("key", AsciiType.instance), cfamily);
- SSTableReader reader = writer.closeAndOpenReader();
+ SSTableReader reader = writer.finish(true);
// Export to JSON and verify
File tempJson = File.createTempFile("CFWithAsciiKeys", ".json");
SSTableExport.export(reader,
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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
new file mode 100644
index 0000000..4e160c2
--- /dev/null
+++ b/test/unit/org/apache/cassandra/utils/concurrent/AbstractTransactionalTest.java
@@ -0,0 +1,136 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+package org.apache.cassandra.utils.concurrent;
+
+import org.junit.Ignore;
+import org.junit.Test;
+
+import junit.framework.Assert;
+
+@Ignore
+public abstract class AbstractTransactionalTest
+{
+
+ protected abstract TestableTransaction newTest() throws Exception;
+
+ @Test
+ public void testNoPrepare() throws Exception
+ {
+ TestableTransaction txn;
+
+ txn = newTest();
+ txn.assertInProgress();
+ txn.testing.close();
+ txn.assertAborted();
+
+ txn = newTest();
+ txn.assertInProgress();
+ Assert.assertNull(txn.testing.abort(null));
+ txn.assertAborted();
+ }
+
+ @Test
+ public void testPrepare() throws Exception
+ {
+ TestableTransaction txn;
+ txn = newTest();
+ txn.assertInProgress();
+ txn.testing.prepareToCommit();
+ txn.assertPrepared();
+ txn.testing.close();
+ txn.assertAborted();
+
+ txn = newTest();
+ txn.assertInProgress();
+ txn.testing.prepareToCommit();
+ txn.assertPrepared();
+ Assert.assertNull(txn.testing.abort(null));
+ txn.assertAborted();
+ }
+
+ @Test
+ public void testCommit() throws Exception
+ {
+ TestableTransaction txn = newTest();
+ txn.assertInProgress();
+ txn.testing.prepareToCommit();
+ txn.assertPrepared();
+ Assert.assertNull(txn.testing.commit(null));
+ txn.assertCommitted();
+ txn.testing.close();
+ txn.assertCommitted();
+ Throwable t = txn.testing.abort(null);
+ Assert.assertTrue(t instanceof IllegalStateException);
+ txn.assertCommitted();
+ }
+
+ @Test
+ public void testThrowableReturn() throws Exception
+ {
+ TestableTransaction txn;
+ 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);
+ }
+
+ @Test
+ public void testBadCommit() throws Exception
+ {
+ TestableTransaction txn;
+ txn = newTest();
+ try
+ {
+ txn.testing.commit(null);
+ Assert.assertTrue(false);
+ }
+ catch (IllegalStateException t)
+ {
+ }
+ txn.assertInProgress();
+ Assert.assertNull(txn.testing.abort(null));
+ txn.assertAborted();
+ try
+ {
+ txn.testing.commit(null);
+ Assert.assertTrue(false);
+ }
+ catch (IllegalStateException t)
+ {
+ }
+ txn.assertAborted();
+ }
+
+
+ public static abstract class TestableTransaction
+ {
+ final Transactional testing;
+ public TestableTransaction(Transactional transactional)
+ {
+ this.testing = transactional;
+ }
+
+ protected abstract void assertInProgress() throws Exception;
+ protected abstract void assertPrepared() throws Exception;
+ protected abstract void assertAborted() throws Exception;
+ protected abstract void assertCommitted() throws Exception;
+ }
+}
[7/7] cassandra git commit: Merge branch 'cassandra-2.2' into trunk
Posted by be...@apache.org.
Merge branch 'cassandra-2.2' into trunk
Conflicts:
CHANGES.txt
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/75a34879
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/75a34879
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/75a34879
Branch: refs/heads/trunk
Commit: 75a34879bb4aa01a6a65858c303f2a058af512a4
Parents: c2a1cd4 8704006
Author: Benedict Elliott Smith <be...@apache.org>
Authored: Sun May 17 13:50:44 2015 +0100
Committer: Benedict Elliott Smith <be...@apache.org>
Committed: Sun May 17 13:50:44 2015 +0100
----------------------------------------------------------------------
CHANGES.txt | 1 +
src/java/org/apache/cassandra/db/Memtable.java | 14 +-
.../db/compaction/CompactionManager.java | 24 +-
.../cassandra/db/compaction/CompactionTask.java | 31 +--
.../cassandra/db/compaction/Scrubber.java | 21 +-
.../cassandra/db/compaction/Upgrader.java | 11 +-
.../writers/CompactionAwareWriter.java | 42 +++-
.../writers/DefaultCompactionWriter.java | 18 +-
.../writers/MajorLeveledCompactionWriter.java | 28 +--
.../writers/MaxSSTableSizeWriter.java | 16 +-
.../SplittingSizeTieredCompactionWriter.java | 17 +-
.../io/compress/CompressedSequentialWriter.java | 59 ++---
.../io/compress/CompressionMetadata.java | 131 +++++-----
.../cassandra/io/sstable/IndexSummary.java | 11 +-
.../io/sstable/IndexSummaryBuilder.java | 13 +-
.../apache/cassandra/io/sstable/SSTable.java | 2 +-
.../cassandra/io/sstable/SSTableRewriter.java | 250 +++++++++----------
.../io/sstable/SSTableSimpleUnsortedWriter.java | 56 +++--
.../io/sstable/SSTableSimpleWriter.java | 9 +-
.../io/sstable/format/SSTableReader.java | 13 +-
.../io/sstable/format/SSTableWriter.java | 137 +++++++---
.../io/sstable/format/big/BigTableWriter.java | 248 +++++++++---------
.../io/util/BufferedPoolingSegmentedFile.java | 3 +-
.../io/util/BufferedSegmentedFile.java | 3 +-
.../io/util/ChecksummedSequentialWriter.java | 38 ++-
.../io/util/CompressedPoolingSegmentedFile.java | 4 +-
.../io/util/CompressedSegmentedFile.java | 9 +-
.../org/apache/cassandra/io/util/FileUtils.java | 39 ++-
.../cassandra/io/util/MmappedSegmentedFile.java | 3 +-
.../apache/cassandra/io/util/SafeMemory.java | 5 +
.../cassandra/io/util/SafeMemoryWriter.java | 5 +
.../apache/cassandra/io/util/SegmentedFile.java | 23 +-
.../cassandra/io/util/SequentialWriter.java | 116 ++++++---
.../cassandra/streaming/StreamReceiveTask.java | 2 +-
.../apache/cassandra/tools/SSTableImport.java | 97 +++----
.../cassandra/utils/AlwaysPresentFilter.java | 5 +
.../org/apache/cassandra/utils/Throwables.java | 5 +
.../apache/cassandra/utils/concurrent/Ref.java | 28 ++-
.../apache/cassandra/utils/concurrent/Refs.java | 11 +-
.../utils/concurrent/SharedCloseable.java | 1 +
.../utils/concurrent/SharedCloseableImpl.java | 5 +
.../utils/concurrent/Transactional.java | 198 +++++++++++++++
.../unit/org/apache/cassandra/db/ScrubTest.java | 2 +-
.../db/compaction/AntiCompactionTest.java | 11 +-
.../db/compaction/CompactionsTest.java | 31 +--
.../cassandra/io/RandomAccessReaderTest.java | 8 +-
.../CompressedRandomAccessReaderTest.java | 15 +-
.../CompressedSequentialWriterTest.java | 136 ++++++++--
.../io/sstable/BigTableWriterTest.java | 130 ++++++++++
.../io/sstable/SSTableRewriterTest.java | 154 +++++-------
.../cassandra/io/sstable/SSTableUtils.java | 2 +-
.../io/util/BufferedRandomAccessFileTest.java | 28 +--
.../util/ChecksummedSequentialWriterTest.java | 92 +++++++
.../cassandra/io/util/DataOutputTest.java | 1 +
.../cassandra/io/util/SequentialWriterTest.java | 117 +++++++++
.../compress/CompressedInputStreamTest.java | 2 +-
.../cassandra/tools/SSTableExportTest.java | 16 +-
.../concurrent/AbstractTransactionalTest.java | 136 ++++++++++
58 files changed, 1731 insertions(+), 902 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/75a34879/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 2350e62,9f14fba..2f75ac2
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,4 -1,5 +1,5 @@@
-2.2.0-beta1
+2.2
+ * Introduce Transactional API for internal state changes (CASSANDRA-8984)
* Add a flag in cassandra.yaml to enable UDFs (CASSANDRA-9404)
* Better support of null for UDF (CASSANDRA-8374)
* Use ecj instead of javassist for UDFs (CASSANDRA-8241)
[5/7] cassandra git commit: Introduce Transactional API for internal
state changes
Posted by be...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 087e57a..fa17c20 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
@@ -47,8 +47,10 @@ import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.FilterFactory;
import org.apache.cassandra.utils.IFilter;
-import org.apache.cassandra.utils.Pair;
import org.apache.cassandra.utils.StreamingHistogram;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import static org.apache.cassandra.utils.Throwables.merge;
public class BigTableWriter extends SSTableWriter
{
@@ -57,7 +59,7 @@ public class BigTableWriter extends SSTableWriter
// not very random, but the only value that can't be mistaken for a legal column-name length
public static final int END_OF_ROW = 0x0000;
- private IndexWriter iwriter;
+ private final IndexWriter iwriter;
private SegmentedFile.Builder dbuilder;
private final SequentialWriter dataFile;
private DecoratedKey lastWrittenKey;
@@ -270,47 +272,6 @@ public class BigTableWriter extends SSTableWriter
return currentPosition;
}
- /**
- * After failure, attempt to close the index writer and data file before deleting all temp components for the sstable
- */
- public void abort()
- {
- assert descriptor.type.isTemporary;
- if (iwriter == null && dataFile == null)
- return;
-
- if (iwriter != null)
- iwriter.abort();
-
- if (dataFile!= null)
- dataFile.abort();
-
- if (dbuilder != null)
- dbuilder.close();
-
- Set<Component> components = SSTable.componentsFor(descriptor);
- try
- {
- if (!components.isEmpty())
- SSTable.delete(descriptor, components);
- }
- catch (FSWriteError e)
- {
- logger.error(String.format("Failed deleting temp components for %s", descriptor), e);
- throw e;
- }
- }
-
- // we use this method to ensure any managed data we may have retained references to during the write are no
- // longer referenced, so that we do not need to enclose the expensive call to closeAndOpenReader() in a transaction
- public void isolateReferences()
- {
- // currently we only maintain references to first/last/lastWrittenKey from the data provided; all other
- // data retention is done through copying
- first = getMinimalKey(first);
- last = lastWrittenKey = getMinimalKey(last);
- }
-
private Descriptor makeTmpLinks()
{
// create temp links if they don't already exist
@@ -323,17 +284,14 @@ public class BigTableWriter extends SSTableWriter
return link;
}
- public SSTableReader openEarly(long maxDataAge)
+ public SSTableReader openEarly()
{
- StatsMetadata sstableMetadata = (StatsMetadata) metadataCollector.finalizeMetadata(partitioner.getClass().getCanonicalName(),
- metadata.getBloomFilterFpChance(),
- repairedAt).get(MetadataType.STATS);
-
// find the max (exclusive) readable key
IndexSummaryBuilder.ReadableBoundary boundary = iwriter.getMaxReadable();
if (boundary == null)
return null;
+ 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
@@ -343,7 +301,7 @@ public class BigTableWriter extends SSTableWriter
components, metadata,
partitioner, ifile,
dfile, iwriter.summary.build(partitioner, boundary),
- iwriter.bf.sharedCopy(), maxDataAge, sstableMetadata, SSTableReader.OpenReason.EARLY);
+ iwriter.bf.sharedCopy(), maxDataAge, stats, SSTableReader.OpenReason.EARLY);
// now it's open, find the ACTUAL last readable key (i.e. for which the data file has also been flushed)
sstable.first = getMinimalKey(first);
@@ -351,31 +309,23 @@ public class BigTableWriter extends SSTableWriter
return sstable;
}
- public SSTableReader closeAndOpenReader()
+ public SSTableReader openFinalEarly()
{
- return closeAndOpenReader(System.currentTimeMillis());
+ // we must ensure the data is completely flushed to disk
+ dataFile.sync();
+ iwriter.indexFile.sync();
+ return openFinal(makeTmpLinks(), SSTableReader.OpenReason.EARLY);
}
- public SSTableReader closeAndOpenReader(long maxDataAge)
+ private SSTableReader openFinal(Descriptor desc, SSTableReader.OpenReason openReason)
{
- return finish(FinishType.NORMAL, maxDataAge, this.repairedAt);
- }
-
- public SSTableReader finish(FinishType finishType, long maxDataAge, long repairedAt)
- {
- assert finishType != FinishType.CLOSE;
- Pair<Descriptor, StatsMetadata> p;
-
- p = close(finishType, repairedAt < 0 ? this.repairedAt : repairedAt);
- Descriptor desc = p.left;
- StatsMetadata metadata = p.right;
-
- if (finishType == FinishType.EARLY)
- desc = makeTmpLinks();
+ if (maxDataAge < 0)
+ maxDataAge = System.currentTimeMillis();
+ StatsMetadata stats = statsMetadata();
// finalize in-memory state for the reader
- SegmentedFile ifile = iwriter.builder.complete(desc.filenameFor(Component.PRIMARY_INDEX), finishType.isFinal);
- SegmentedFile dfile = dbuilder.complete(desc.filenameFor(Component.DATA), finishType.isFinal);
+ 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),
components,
this.metadata,
@@ -385,81 +335,93 @@ public class BigTableWriter extends SSTableWriter
iwriter.summary.build(partitioner),
iwriter.bf.sharedCopy(),
maxDataAge,
- metadata,
- finishType.openReason);
+ stats,
+ openReason);
sstable.first = getMinimalKey(first);
sstable.last = getMinimalKey(last);
-
- if (finishType.isFinal)
- {
- iwriter.bf.close();
- iwriter.summary.close();
- // try to save the summaries to disk
- sstable.saveSummary(iwriter.builder, dbuilder);
- iwriter.builder.close();
- iwriter = null;
- dbuilder.close();
- dbuilder = null;
- }
return sstable;
}
- // Close the writer and return the descriptor to the new sstable and it's metadata
- public Pair<Descriptor, StatsMetadata> close()
+ protected SSTableWriter.TransactionalProxy txnProxy()
{
- Pair<Descriptor, StatsMetadata> ret = close(FinishType.CLOSE, this.repairedAt);
- if (dbuilder != null)
- dbuilder.close();
- if (iwriter != null)
- iwriter.builder.close();
- return ret;
+ return new TransactionalProxy();
}
- private Pair<Descriptor, StatsMetadata> close(FinishType type, long repairedAt)
+ class TransactionalProxy extends SSTableWriter.TransactionalProxy
{
- switch (type)
+ // finalise our state on disk, including renaming
+ protected void doPrepare()
{
- case EARLY: case CLOSE: case NORMAL:
- iwriter.close();
- dataFile.close();
- if (type == FinishType.CLOSE)
- iwriter.bf.close();
- }
+ Map<MetadataType, MetadataComponent> metadataComponents = finalizeMetadata();
- // write sstable statistics
- Map<MetadataType, MetadataComponent> metadataComponents;
- metadataComponents = metadataCollector
- .finalizeMetadata(partitioner.getClass().getCanonicalName(),
- metadata.getBloomFilterFpChance(),repairedAt);
+ iwriter.prepareToCommit();
- // remove the 'tmp' marker from all components
- Descriptor descriptor = this.descriptor;
- if (type.isFinal)
- {
- dataFile.writeFullChecksum(descriptor);
+ // write sstable statistics
+ dataFile.setDescriptor(descriptor).prepareToCommit();
writeMetadata(descriptor, metadataComponents);
+
// save the table of components
SSTable.appendTOC(descriptor, components);
- descriptor = rename(descriptor, components);
+
+ // rename to final
+ rename(descriptor, components);
+
+ if (openResult)
+ finalReader = openFinal(descriptor.asType(Descriptor.Type.FINAL), SSTableReader.OpenReason.NORMAL);
}
- return Pair.create(descriptor, (StatsMetadata) metadataComponents.get(MetadataType.STATS));
+ protected Throwable doCommit(Throwable accumulate)
+ {
+ accumulate = dataFile.commit(accumulate);
+ accumulate = iwriter.commit(accumulate);
+ return accumulate;
+ }
+
+ protected Throwable doCleanup(Throwable accumulate)
+ {
+ accumulate = dbuilder.close(accumulate);
+ return accumulate;
+ }
+
+ protected Throwable doAbort(Throwable accumulate)
+ {
+ 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;
+ }
}
private static void writeMetadata(Descriptor desc, Map<MetadataType, MetadataComponent> components)
{
- SequentialWriter out = SequentialWriter.open(new File(desc.filenameFor(Component.STATS)));
- try
+ File file = new File(desc.filenameFor(Component.STATS));
+ try (SequentialWriter out = SequentialWriter.open(file);)
{
desc.getMetadataSerializer().serialize(components, out.stream);
+ out.setDescriptor(desc).finish();
}
catch (IOException e)
{
- throw new FSWriteError(e, out.getPath());
- }
- finally
- {
- out.close();
+ throw new FSWriteError(e, file.getPath());
}
}
@@ -476,7 +438,7 @@ public class BigTableWriter extends SSTableWriter
/**
* Encapsulates writing the index and filter for an SSTable. The state of this object is not valid until it has been closed.
*/
- class IndexWriter
+ class IndexWriter extends AbstractTransactional implements Transactional
{
private final SequentialWriter indexFile;
public final SegmentedFile.Builder builder;
@@ -535,18 +497,10 @@ public class BigTableWriter extends SSTableWriter
builder.addPotentialBoundary(indexStart);
}
- public void abort()
- {
- summary.close();
- indexFile.abort();
- bf.close();
- builder.close();
- }
-
/**
* Closes the index and bloomfilter, making the public state of this writer valid for consumption.
*/
- public void close()
+ void flushBf()
{
if (components.contains(Component.FILTER))
{
@@ -566,11 +520,6 @@ public class BigTableWriter extends SSTableWriter
throw new FSWriteError(e, path);
}
}
-
- // index
- long position = indexFile.getFilePointer();
- indexFile.close(); // calls force
- FileUtils.truncate(indexFile.getPath(), position);
}
public void mark()
@@ -585,5 +534,40 @@ public class BigTableWriter extends SSTableWriter
// we assume that if that worked then we won't be trying to reset.
indexFile.resetAndTruncate(mark);
}
+
+ protected void doPrepare()
+ {
+ flushBf();
+
+ // truncate index file
+ long position = iwriter.indexFile.getFilePointer();
+ iwriter.indexFile.setDescriptor(descriptor).prepareToCommit();
+ FileUtils.truncate(iwriter.indexFile.getPath(), position);
+
+ // save summary
+ summary.prepareToCommit();
+ try (IndexSummary summary = iwriter.summary.build(partitioner))
+ {
+ SSTableReader.saveSummary(descriptor, first, last, iwriter.builder, dbuilder, summary);
+ }
+ }
+
+ protected Throwable doCommit(Throwable accumulate)
+ {
+ return indexFile.commit(accumulate);
+ }
+
+ protected Throwable doAbort(Throwable accumulate)
+ {
+ return indexFile.abort(accumulate);
+ }
+
+ protected Throwable doCleanup(Throwable accumulate)
+ {
+ accumulate = summary.close(accumulate);
+ accumulate = bf.close(accumulate);
+ accumulate = builder.close(accumulate);
+ return accumulate;
+ }
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/io/util/BufferedPoolingSegmentedFile.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/BufferedPoolingSegmentedFile.java b/src/java/org/apache/cassandra/io/util/BufferedPoolingSegmentedFile.java
index e8b719e..b623e54 100644
--- a/src/java/org/apache/cassandra/io/util/BufferedPoolingSegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/BufferedPoolingSegmentedFile.java
@@ -41,9 +41,8 @@ public class BufferedPoolingSegmentedFile extends PoolingSegmentedFile
// only one segment in a standard-io file
}
- public SegmentedFile complete(ChannelProxy channel, long overrideLength, boolean isFinal)
+ public SegmentedFile complete(ChannelProxy channel, long overrideLength)
{
- assert !isFinal || overrideLength <= 0;
long length = overrideLength > 0 ? overrideLength : channel.size();
return new BufferedPoolingSegmentedFile(channel, length);
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/io/util/BufferedSegmentedFile.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/BufferedSegmentedFile.java b/src/java/org/apache/cassandra/io/util/BufferedSegmentedFile.java
index b4d966a..2c59def 100644
--- a/src/java/org/apache/cassandra/io/util/BufferedSegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/BufferedSegmentedFile.java
@@ -48,9 +48,8 @@ public class BufferedSegmentedFile extends SegmentedFile
// only one segment in a standard-io file
}
- public SegmentedFile complete(ChannelProxy channel, long overrideLength, boolean isFinal)
+ public SegmentedFile complete(ChannelProxy channel, long overrideLength)
{
- assert !isFinal || overrideLength <= 0;
long length = overrideLength > 0 ? overrideLength : channel.size();
return new BufferedSegmentedFile(channel, length);
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/io/util/ChecksummedSequentialWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/ChecksummedSequentialWriter.java b/src/java/org/apache/cassandra/io/util/ChecksummedSequentialWriter.java
index d28a14d..ec68c2d 100644
--- a/src/java/org/apache/cassandra/io/util/ChecksummedSequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/util/ChecksummedSequentialWriter.java
@@ -20,8 +20,6 @@ package org.apache.cassandra.io.util;
import java.io.File;
import java.nio.ByteBuffer;
-import org.apache.cassandra.io.sstable.Descriptor;
-
public class ChecksummedSequentialWriter extends SequentialWriter
{
private final SequentialWriter crcWriter;
@@ -44,20 +42,36 @@ public class ChecksummedSequentialWriter extends SequentialWriter
crcMetadata.appendDirect(toAppend, false);
}
- public void writeFullChecksum(Descriptor descriptor)
+ protected class TransactionalProxy extends SequentialWriter.TransactionalProxy
{
- crcMetadata.writeFullChecksum(descriptor);
- }
+ @Override
+ protected Throwable doCommit(Throwable accumulate)
+ {
+ return crcWriter.commit(accumulate);
+ }
- public void close()
- {
- super.close();
- crcWriter.close();
+ @Override
+ protected Throwable doAbort(Throwable accumulate)
+ {
+ return super.doAbort(crcWriter.abort(accumulate));
+ }
+
+ @Override
+ protected void doPrepare()
+ {
+ syncInternal();
+ if (descriptor != null)
+ crcMetadata.writeFullChecksum(descriptor);
+ crcWriter.setDescriptor(descriptor).prepareToCommit();
+ // we must cleanup our file handles during prepareCommit for Windows compatibility as we cannot rename an open file;
+ // TODO: once we stop file renaming, remove this for clarity
+ releaseFileHandle();
+ }
}
- public void abort()
+ @Override
+ protected SequentialWriter.TransactionalProxy txnProxy()
{
- super.abort();
- crcWriter.abort();
+ return new TransactionalProxy();
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/io/util/CompressedPoolingSegmentedFile.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/CompressedPoolingSegmentedFile.java b/src/java/org/apache/cassandra/io/util/CompressedPoolingSegmentedFile.java
index cb30131..fdc4f61 100644
--- a/src/java/org/apache/cassandra/io/util/CompressedPoolingSegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/CompressedPoolingSegmentedFile.java
@@ -96,9 +96,9 @@ public class CompressedPoolingSegmentedFile extends PoolingSegmentedFile impleme
// only one segment in a standard-io file
}
- public SegmentedFile complete(ChannelProxy channel, long overrideLength, boolean isFinal)
+ public SegmentedFile complete(ChannelProxy channel, long overrideLength)
{
- return new CompressedPoolingSegmentedFile(channel, metadata(channel.filePath(), overrideLength, isFinal));
+ return new CompressedPoolingSegmentedFile(channel, metadata(channel.filePath(), overrideLength));
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/io/util/CompressedSegmentedFile.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/CompressedSegmentedFile.java b/src/java/org/apache/cassandra/io/util/CompressedSegmentedFile.java
index caf4c22..ceff7ba 100644
--- a/src/java/org/apache/cassandra/io/util/CompressedSegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/CompressedSegmentedFile.java
@@ -136,18 +136,17 @@ public class CompressedSegmentedFile extends SegmentedFile implements ICompresse
// only one segment in a standard-io file
}
- protected CompressionMetadata metadata(String path, long overrideLength, boolean isFinal)
+ protected CompressionMetadata metadata(String path, long overrideLength)
{
if (writer == null)
return CompressionMetadata.create(path);
- return writer.open(overrideLength, isFinal);
+ return writer.open(overrideLength);
}
- public SegmentedFile complete(ChannelProxy channel, long overrideLength, boolean isFinal)
+ public SegmentedFile complete(ChannelProxy channel, long overrideLength)
{
- assert !isFinal || overrideLength <= 0;
- return new CompressedSegmentedFile(channel, metadata(channel.filePath(), overrideLength, isFinal));
+ return new CompressedSegmentedFile(channel, metadata(channel.filePath(), overrideLength));
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 8007039..2566952 100644
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@ -40,6 +40,9 @@ import org.apache.cassandra.io.sstable.CorruptSSTableException;
import org.apache.cassandra.service.StorageService;
import org.apache.cassandra.utils.JVMStabilityInspector;
+import static org.apache.cassandra.utils.Throwables.maybeFail;
+import static org.apache.cassandra.utils.Throwables.merge;
+
public class FileUtils
{
private static final Logger logger = LoggerFactory.getLogger(FileUtils.class);
@@ -107,24 +110,42 @@ public class FileUtils
return createTempFile(prefix, suffix, new File(System.getProperty("java.io.tmpdir")));
}
- public static void deleteWithConfirm(String file)
+ public static Throwable deleteWithConfirm(String filePath, boolean expect, Throwable accumulate)
{
- deleteWithConfirm(new File(file));
+ return deleteWithConfirm(new File(filePath), expect, accumulate);
}
- public static void deleteWithConfirm(File file)
+ public static Throwable deleteWithConfirm(File file, boolean expect, Throwable accumulate)
{
- assert file.exists() : "attempted to delete non-existing file " + file.getName();
- if (logger.isDebugEnabled())
- logger.debug("Deleting {}", file.getName());
+ boolean exists = file.exists();
+ assert exists || !expect : "attempted to delete non-existing file " + file.getName();
try
{
- Files.delete(file.toPath());
+ if (exists)
+ Files.delete(file.toPath());
}
- catch (IOException e)
+ catch (Throwable t)
{
- throw new FSWriteError(e, file);
+ try
+ {
+ throw new FSWriteError(t, file);
+ }
+ catch (Throwable t2)
+ {
+ accumulate = merge(accumulate, t2);
+ }
}
+ return accumulate;
+ }
+
+ public static void deleteWithConfirm(String file)
+ {
+ deleteWithConfirm(new File(file));
+ }
+
+ public static void deleteWithConfirm(File file)
+ {
+ maybeFail(deleteWithConfirm(file, true, null));
}
public static void renameWithOutConfirm(String from, String to)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java b/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
index 57295fe..91908c9 100644
--- a/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
@@ -183,9 +183,8 @@ public class MmappedSegmentedFile extends SegmentedFile
}
}
- public SegmentedFile complete(ChannelProxy channel, long overrideLength, boolean isFinal)
+ public SegmentedFile complete(ChannelProxy channel, long overrideLength)
{
- assert !isFinal || overrideLength <= 0;
long length = overrideLength > 0 ? overrideLength : channel.size();
// create the segments
return new MmappedSegmentedFile(channel, length, createSegments(channel, length));
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/io/util/SafeMemory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/SafeMemory.java b/src/java/org/apache/cassandra/io/util/SafeMemory.java
index f96afcc..ad11472 100644
--- a/src/java/org/apache/cassandra/io/util/SafeMemory.java
+++ b/src/java/org/apache/cassandra/io/util/SafeMemory.java
@@ -62,6 +62,11 @@ public class SafeMemory extends Memory implements SharedCloseable
peer = 0;
}
+ public Throwable close(Throwable accumulate)
+ {
+ return ref.ensureReleased(accumulate);
+ }
+
public SafeMemory copy(long newSize)
{
SafeMemory copy = new SafeMemory(newSize);
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/io/util/SafeMemoryWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/SafeMemoryWriter.java b/src/java/org/apache/cassandra/io/util/SafeMemoryWriter.java
index 1fc374f..1096b5f 100644
--- a/src/java/org/apache/cassandra/io/util/SafeMemoryWriter.java
+++ b/src/java/org/apache/cassandra/io/util/SafeMemoryWriter.java
@@ -71,6 +71,11 @@ public class SafeMemoryWriter extends DataOutputBuffer
memory.close();
}
+ public Throwable close(Throwable accumulate)
+ {
+ return memory.close(accumulate);
+ }
+
public long length()
{
return tailOffset(memory) + buffer.position();
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/io/util/SegmentedFile.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/SegmentedFile.java b/src/java/org/apache/cassandra/io/util/SegmentedFile.java
index cb4d132..edbd742 100644
--- a/src/java/org/apache/cassandra/io/util/SegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/SegmentedFile.java
@@ -37,6 +37,8 @@ import org.apache.cassandra.utils.Pair;
import org.apache.cassandra.utils.concurrent.RefCounted;
import org.apache.cassandra.utils.concurrent.SharedCloseableImpl;
+import static org.apache.cassandra.utils.Throwables.maybeFail;
+
/**
* Abstracts a read-only file that has been split into segments, each of which can be represented by an independent
* FileDataInput. Allows for iteration over the FileDataInputs, or random access to the FileDataInput for a given
@@ -169,21 +171,16 @@ public abstract class SegmentedFile extends SharedCloseableImpl
* Called after all potential boundaries have been added to apply this Builder to a concrete file on disk.
* @param channel The channel to the file on disk.
*/
- protected abstract SegmentedFile complete(ChannelProxy channel, long overrideLength, boolean isFinal);
+ protected abstract SegmentedFile complete(ChannelProxy channel, long overrideLength);
public SegmentedFile complete(String path)
{
- return complete(getChannel(path), -1, true);
- }
-
- public SegmentedFile complete(String path, boolean isFinal)
- {
- return complete(getChannel(path), -1, isFinal);
+ return complete(getChannel(path), -1);
}
public SegmentedFile complete(String path, long overrideLength)
{
- return complete(getChannel(path), overrideLength, false);
+ return complete(getChannel(path), overrideLength);
}
public void serializeBounds(DataOutput out) throws IOException
@@ -197,10 +194,16 @@ public abstract class SegmentedFile extends SharedCloseableImpl
throw new IOException("Cannot deserialize SSTable Summary component because the DiskAccessMode was changed!");
}
- public void close()
+ public Throwable close(Throwable accumulate)
{
if (channel != null)
- channel.close();
+ return channel.close(accumulate);
+ return accumulate;
+ }
+
+ public void close()
+ {
+ maybeFail(close(null));
}
private ChannelProxy getChannel(String path)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 c4fef07..d63be31 100644
--- a/src/java/org/apache/cassandra/io/util/SequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/util/SequentialWriter.java
@@ -24,8 +24,6 @@ import java.nio.channels.FileChannel;
import java.nio.channels.WritableByteChannel;
import java.nio.file.StandardOpenOption;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.io.FSReadError;
import org.apache.cassandra.io.FSWriteError;
@@ -34,15 +32,16 @@ import org.apache.cassandra.io.compress.CompressionParameters;
import org.apache.cassandra.io.sstable.Descriptor;
import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
import org.apache.cassandra.utils.CLibrary;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import static org.apache.cassandra.utils.Throwables.merge;
/**
* Adds buffering, mark, and fsyncing to OutputStream. We always fsync on close; we may also
* fsync incrementally if Config.trickle_fsync is enabled.
*/
-public class SequentialWriter extends OutputStream implements WritableByteChannel
+public class SequentialWriter extends OutputStream implements WritableByteChannel, Transactional
{
- private static final Logger logger = LoggerFactory.getLogger(SequentialWriter.class);
-
// isDirty - true if this.buffer contains any un-synced bytes
protected boolean isDirty = false, syncNeeded = false;
@@ -71,6 +70,55 @@ public class SequentialWriter extends OutputStream implements WritableByteChanne
protected Runnable runPostFlush;
+ private final TransactionalProxy txnProxy = txnProxy();
+ protected Descriptor descriptor;
+
+ // due to lack of multiple-inheritance, we proxy our transactional implementation
+ protected class TransactionalProxy extends AbstractTransactional
+ {
+ @Override
+ protected Throwable doCleanup(Throwable accumulate)
+ {
+ if (directoryFD >= 0)
+ {
+ try { CLibrary.tryCloseFD(directoryFD); }
+ catch (Throwable t) { accumulate = merge(accumulate, t); }
+ directoryFD = -1;
+ }
+
+ // close is idempotent
+ try { channel.close(); }
+ catch (Throwable t) { accumulate = merge(accumulate, t); }
+
+ if (buffer != null)
+ {
+ try { FileUtils.clean(buffer); }
+ catch (Throwable t) { accumulate = merge(accumulate, t); }
+ buffer = null;
+ }
+
+ return accumulate;
+ }
+
+ protected void doPrepare()
+ {
+ syncInternal();
+ // we must cleanup our file handles during prepareCommit for Windows compatibility as we cannot rename an open file;
+ // TODO: once we stop file renaming, remove this for clarity
+ releaseFileHandle();
+ }
+
+ protected Throwable doCommit(Throwable accumulate)
+ {
+ return accumulate;
+ }
+
+ protected Throwable doAbort(Throwable accumulate)
+ {
+ return FileUtils.deleteWithConfirm(filePath, false, accumulate);
+ }
+ }
+
public SequentialWriter(File file, int bufferSize, boolean offheap)
{
try
@@ -383,49 +431,53 @@ public class SequentialWriter extends OutputStream implements WritableByteChanne
return channel.isOpen();
}
- @Override
- public void close()
+ public SequentialWriter setDescriptor(Descriptor descriptor)
{
- if (buffer == null)
- return; // already closed
-
- syncInternal();
+ this.descriptor = descriptor;
+ return this;
+ }
- buffer = null;
+ public final void prepareToCommit()
+ {
+ txnProxy.prepareToCommit();
+ }
- cleanup(true);
+ public final Throwable commit(Throwable accumulate)
+ {
+ return txnProxy.commit(accumulate);
}
- public void abort()
+ public final Throwable abort(Throwable accumulate)
{
- cleanup(false);
+ return txnProxy.abort(accumulate);
}
- private void cleanup(boolean throwExceptions)
+ @Override
+ public final void close()
{
- if (directoryFD >= 0)
- {
- try { CLibrary.tryCloseFD(directoryFD); }
- catch (Throwable t) { handle(t, throwExceptions); }
- directoryFD = -1;
- }
+ txnProxy.close();
+ }
- // close is idempotent
- try { channel.close(); }
- catch (Throwable t) { handle(t, throwExceptions); }
+ public final void finish()
+ {
+ txnProxy.finish();
}
- private void handle(Throwable t, boolean throwExceptions)
+ protected TransactionalProxy txnProxy()
{
- if (!throwExceptions)
- logger.warn("Suppressing exception thrown while aborting writer", t);
- else
- throw new FSWriteError(t, getPath());
+ return new TransactionalProxy();
}
- // hack to make life easier for subclasses
- public void writeFullChecksum(Descriptor descriptor)
+ public void releaseFileHandle()
{
+ try
+ {
+ channel.close();
+ }
+ catch (IOException e)
+ {
+ throw new FSWriteError(e, filePath);
+ }
}
/**
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 1049d43..d4d49b3 100644
--- a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
@@ -122,7 +122,7 @@ public class StreamReceiveTask extends StreamTask
lockfile.create(task.sstables);
List<SSTableReader> readers = new ArrayList<>();
for (SSTableWriter writer : task.sstables)
- readers.add(writer.closeAndOpenReader());
+ readers.add(writer.finish(true));
lockfile.delete();
task.sstables.clear();
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/tools/SSTableImport.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/SSTableImport.java b/src/java/org/apache/cassandra/tools/SSTableImport.java
index d3a2683..7b187ac 100644
--- a/src/java/org/apache/cassandra/tools/SSTableImport.java
+++ b/src/java/org/apache/cassandra/tools/SSTableImport.java
@@ -71,7 +71,7 @@ public class SSTableImport
private final boolean isSorted;
private static final JsonFactory factory = new MappingJsonFactory().configure(
- JsonParser.Feature.INTERN_FIELD_NAMES, false);
+ JsonParser.Feature.INTERN_FIELD_NAMES, false);
static
{
@@ -143,10 +143,10 @@ public class SSTableImport
else
{
assert meta.isCQL3Table() || name.hasRemaining() : "Cell name should not be empty";
- value = stringAsType((String) fields.get(1),
- meta.getValueValidator(name.hasRemaining()
- ? comparator.cellFromByteBuffer(name)
- : meta.comparator.rowMarker(Composites.EMPTY)));
+ value = stringAsType((String) fields.get(1),
+ meta.getValueValidator(name.hasRemaining()
+ ? comparator.cellFromByteBuffer(name)
+ : meta.comparator.rowMarker(Composites.EMPTY)));
}
}
}
@@ -219,10 +219,10 @@ public class SSTableImport
cfamily.addAtom(new RangeTombstone(start, end, col.timestamp, col.localExpirationTime));
continue;
}
-
+
assert cfm.isCQL3Table() || col.getName().hasRemaining() : "Cell name should not be empty";
- CellName cname = col.getName().hasRemaining() ? cfm.comparator.cellFromByteBuffer(col.getName())
- : cfm.comparator.rowMarker(Composites.EMPTY);
+ CellName cname = col.getName().hasRemaining() ? cfm.comparator.cellFromByteBuffer(col.getName())
+ : cfm.comparator.rowMarker(Composites.EMPTY);
if (col.isExpiring())
{
@@ -345,13 +345,13 @@ public class SSTableImport
break;
}
- writer.closeAndOpenReader();
+ writer.finish(true);
return importedKeys;
}
private int importSorted(String jsonFile, ColumnFamily columnFamily, String ssTablePath,
- IPartitioner partitioner) throws IOException
+ IPartitioner partitioner) throws IOException
{
int importedKeys = 0; // already imported keys count
long start = System.nanoTime();
@@ -377,55 +377,56 @@ public class SSTableImport
System.out.printf("Importing %s keys...%n", keyCountToImport);
parser = getParser(jsonFile); // renewing parser
- SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(ssTablePath), keyCountToImport, ActiveRepairService.UNREPAIRED_SSTABLE);
+ try (SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(ssTablePath), keyCountToImport, ActiveRepairService.UNREPAIRED_SSTABLE);)
+ {
+ int lineNumber = 1;
+ DecoratedKey prevStoredKey = null;
- int lineNumber = 1;
- DecoratedKey prevStoredKey = null;
+ parser.nextToken(); // START_ARRAY
+ while (parser.nextToken() != null)
+ {
+ String key = parser.getCurrentName();
+ Map<?, ?> row = parser.readValueAs(new TypeReference<Map<?, ?>>(){});
+ DecoratedKey currentKey = partitioner.decorateKey(getKeyValidator(columnFamily).fromString((String) row.get("key")));
- parser.nextToken(); // START_ARRAY
- while (parser.nextToken() != null)
- {
- String key = parser.getCurrentName();
- Map<?, ?> row = parser.readValueAs(new TypeReference<Map<?, ?>>(){});
- DecoratedKey currentKey = partitioner.decorateKey(getKeyValidator(columnFamily).fromString((String) row.get("key")));
+ if (row.containsKey("metadata"))
+ parseMeta((Map<?, ?>) row.get("metadata"), columnFamily, null);
- if (row.containsKey("metadata"))
- parseMeta((Map<?, ?>) row.get("metadata"), columnFamily, null);
+ addColumnsToCF((List<?>) row.get("cells"), columnFamily);
- addColumnsToCF((List<?>) row.get("cells"), columnFamily);
+ if (prevStoredKey != null && prevStoredKey.compareTo(currentKey) != -1)
+ {
+ System.err
+ .printf("Line %d: Key %s is greater than previous, collection is not sorted properly. Aborting import. You might need to delete SSTables manually.%n",
+ lineNumber, key);
+ return -1;
+ }
- if (prevStoredKey != null && prevStoredKey.compareTo(currentKey) != -1)
- {
- System.err
- .printf("Line %d: Key %s is greater than previous, collection is not sorted properly. Aborting import. You might need to delete SSTables manually.%n",
- lineNumber, key);
- return -1;
- }
+ // saving decorated key
+ writer.append(currentKey, columnFamily);
+ columnFamily.clear();
- // saving decorated key
- writer.append(currentKey, columnFamily);
- columnFamily.clear();
+ prevStoredKey = currentKey;
+ importedKeys++;
+ lineNumber++;
- prevStoredKey = currentKey;
- importedKeys++;
- lineNumber++;
+ long current = System.nanoTime();
- long current = System.nanoTime();
+ if (TimeUnit.NANOSECONDS.toSeconds(current - start) >= 5) // 5 secs.
+ {
+ System.out.printf("Currently imported %d keys.%n", importedKeys);
+ start = current;
+ }
+
+ if (keyCountToImport == importedKeys)
+ break;
- if (TimeUnit.NANOSECONDS.toSeconds(current - start) >= 5) // 5 secs.
- {
- System.out.printf("Currently imported %d keys.%n", importedKeys);
- start = current;
}
- if (keyCountToImport == importedKeys)
- break;
+ writer.finish(true);
+ return importedKeys;
}
-
- writer.closeAndOpenReader();
-
- return importedKeys;
}
/**
@@ -511,7 +512,7 @@ public class SSTableImport
try
{
- new SSTableImport(keyCountToImport, isSorted).importJson(json, keyspace, cfamily, ssTable);
+ new SSTableImport(keyCountToImport, isSorted).importJson(json, keyspace, cfamily, ssTable);
}
catch (Exception e)
{
@@ -527,7 +528,7 @@ public class SSTableImport
private static void printProgramUsage()
{
System.out.printf("Usage: %s -s -K <keyspace> -c <column_family> -n <num_keys> <json> <sstable>%n%n",
- SSTableImport.class.getName());
+ SSTableImport.class.getName());
System.out.println("Options:");
for (Object o : options.getOptions())
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/utils/AlwaysPresentFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/AlwaysPresentFilter.java b/src/java/org/apache/cassandra/utils/AlwaysPresentFilter.java
index a7f6fce..44d8f24 100644
--- a/src/java/org/apache/cassandra/utils/AlwaysPresentFilter.java
+++ b/src/java/org/apache/cassandra/utils/AlwaysPresentFilter.java
@@ -35,6 +35,11 @@ public class AlwaysPresentFilter implements IFilter
return this;
}
+ public Throwable close(Throwable accumulate)
+ {
+ return accumulate;
+ }
+
public long serializedSize() { return 0; }
@Override
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/utils/Throwables.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/Throwables.java b/src/java/org/apache/cassandra/utils/Throwables.java
index 552ca87..0a2bd28 100644
--- a/src/java/org/apache/cassandra/utils/Throwables.java
+++ b/src/java/org/apache/cassandra/utils/Throwables.java
@@ -29,4 +29,9 @@ public class Throwables
return existingFail;
}
+ public static void maybeFail(Throwable fail)
+ {
+ if (fail != null)
+ com.google.common.base.Throwables.propagate(fail);
+ }
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/utils/concurrent/Ref.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/concurrent/Ref.java b/src/java/org/apache/cassandra/utils/concurrent/Ref.java
index 25f8510..ebabd79 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/Ref.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/Ref.java
@@ -11,12 +11,14 @@ import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
-import com.google.common.base.Throwables;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.cassandra.concurrent.NamedThreadFactory;
+import static org.apache.cassandra.utils.Throwables.maybeFail;
+import static org.apache.cassandra.utils.Throwables.merge;
+
/**
* An object that needs ref counting does the two following:
* - defines a Tidy object that will cleanup once it's gone,
@@ -77,14 +79,19 @@ public final class Ref<T> implements RefCounted<T>, AutoCloseable
state.release(false);
}
+ public Throwable ensureReleased(Throwable accumulate)
+ {
+ return state.ensureReleased(accumulate);
+ }
+
public void ensureReleased()
{
- state.ensureReleased();
+ maybeFail(state.ensureReleased(null));
}
public void close()
{
- state.ensureReleased();
+ ensureReleased();
}
public T get()
@@ -150,14 +157,15 @@ public final class Ref<T> implements RefCounted<T>, AutoCloseable
assert released == 0;
}
- void ensureReleased()
+ Throwable ensureReleased(Throwable accumulate)
{
if (releasedUpdater.getAndSet(this, 1) == 0)
{
- globalState.release(this);
+ accumulate = globalState.release(this, accumulate);
if (DEBUG_ENABLED)
debug.deallocate();
}
+ return accumulate;
}
void release(boolean leak)
@@ -174,7 +182,7 @@ public final class Ref<T> implements RefCounted<T>, AutoCloseable
}
return;
}
- globalState.release(this);
+ Throwable fail = globalState.release(this, null);
if (leak)
{
String id = this.toString();
@@ -186,6 +194,8 @@ public final class Ref<T> implements RefCounted<T>, AutoCloseable
{
debug.deallocate();
}
+ if (fail != null)
+ logger.error("Error when closing {}", globalState, fail);
}
}
@@ -264,7 +274,7 @@ public final class Ref<T> implements RefCounted<T>, AutoCloseable
}
// release a single reference, and cleanup if no more are extant
- void release(Ref.State ref)
+ Throwable release(Ref.State ref, Throwable accumulate)
{
locallyExtant.remove(ref);
if (-1 == counts.decrementAndGet())
@@ -276,10 +286,10 @@ public final class Ref<T> implements RefCounted<T>, AutoCloseable
}
catch (Throwable t)
{
- logger.error("Error when closing {}", this, t);
- Throwables.propagate(t);
+ accumulate = merge(accumulate, t);
}
}
+ return accumulate;
}
int count()
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/utils/concurrent/Refs.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/concurrent/Refs.java b/src/java/org/apache/cassandra/utils/concurrent/Refs.java
index 1c6486e..dd65971 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/Refs.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/Refs.java
@@ -9,6 +9,7 @@ import com.google.common.base.Throwables;
import com.google.common.collect.Iterables;
import com.google.common.collect.Iterators;
+import static org.apache.cassandra.utils.Throwables.maybeFail;
import static org.apache.cassandra.utils.Throwables.merge;
/**
@@ -204,7 +205,10 @@ public final class Refs<T extends RefCounted<T>> extends AbstractCollection<T> i
public static void release(Iterable<? extends Ref<?>> refs)
{
- Throwable fail = null;
+ maybeFail(release(refs, null));
+ }
+ public static Throwable release(Iterable<? extends Ref<?>> refs, Throwable accumulate)
+ {
for (Ref ref : refs)
{
try
@@ -213,11 +217,10 @@ public final class Refs<T extends RefCounted<T>> extends AbstractCollection<T> i
}
catch (Throwable t)
{
- fail = merge(fail, t);
+ accumulate = merge(accumulate, t);
}
}
- if (fail != null)
- throw Throwables.propagate(fail);
+ return accumulate;
}
public static <T extends SelfRefCounted<T>> Iterable<Ref<T>> selfRefs(Iterable<T> refs)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/utils/concurrent/SharedCloseable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/concurrent/SharedCloseable.java b/src/java/org/apache/cassandra/utils/concurrent/SharedCloseable.java
index 1e5a026..a3a1863 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/SharedCloseable.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/SharedCloseable.java
@@ -31,5 +31,6 @@ public interface SharedCloseable extends AutoCloseable
* Throws an exception if the shared resource has already been closed.
*/
public SharedCloseable sharedCopy();
+ public Throwable close(Throwable accumulate);
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/utils/concurrent/SharedCloseableImpl.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/concurrent/SharedCloseableImpl.java b/src/java/org/apache/cassandra/utils/concurrent/SharedCloseableImpl.java
index 0d3a843..d85fd54 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/SharedCloseableImpl.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/SharedCloseableImpl.java
@@ -44,4 +44,9 @@ public abstract class SharedCloseableImpl implements SharedCloseable
{
ref.ensureReleased();
}
+
+ public Throwable close(Throwable accumulate)
+ {
+ return ref.ensureReleased(accumulate);
+ }
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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
new file mode 100644
index 0000000..bcf5095
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/concurrent/Transactional.java
@@ -0,0 +1,198 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied. See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+package org.apache.cassandra.utils.concurrent;
+
+import java.util.Set;
+
+import com.google.common.collect.ImmutableSet;
+
+import static org.apache.cassandra.utils.Throwables.maybeFail;
+import static org.apache.cassandra.utils.Throwables.merge;
+
+/**
+ * An abstraction for Transactional behaviour. An object implementing this interface has a lifetime
+ * of the following pattern:
+ *
+ * Throwable failure = null;
+ * try (Transactional t1, t2 = ...)
+ * {
+ * // do work with t1 and t2
+ * t1.prepareToCommit();
+ * t2.prepareToCommit();
+ * failure = t1.commit(failure);
+ * failure = t2.commit(failure);
+ * }
+ * logger.error(failure);
+ *
+ * If something goes wrong before commit() is called on any transaction, then on exiting the try block
+ * the auto close method should invoke cleanup() and then abort() to reset any state.
+ * 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
+ * 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.
+ */
+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.
+ *
+ * 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
+ {
+ public static enum State
+ {
+ IN_PROGRESS,
+ READY_TO_COMMIT,
+ COMMITTED,
+ ABORTED;
+ }
+
+ private State state = State.IN_PROGRESS;
+
+ // the methods for actually performing the necessary behaviours, that are themselves protected against
+ // improper use by the external implementations provided by this class. empty default implementations
+ // could be provided, but we consider it safer to force implementers to consider explicitly their presence
+
+ protected abstract Throwable doCommit(Throwable accumulate);
+ protected abstract Throwable doAbort(Throwable accumulate);
+
+ // this only needs to perform cleanup of state unique to this instance; any internal
+ // Transactional objects will perform cleanup in the commit() or abort() calls
+ protected abstract Throwable doCleanup(Throwable accumulate);
+
+ /**
+ * Do any preparatory work prior to commit. This method should throw any exceptions that can be encountered
+ * during the finalization of the behaviour.
+ */
+ protected abstract void doPrepare();
+
+ /**
+ * commit any effects of this transaction object graph, then cleanup; delegates first to doCommit, then to doCleanup
+ */
+ public final Throwable commit(Throwable accumulate)
+ {
+ if (state != State.READY_TO_COMMIT)
+ throw new IllegalStateException("Commit attempted before prepared to commit");
+ accumulate = doCommit(accumulate);
+ accumulate = doCleanup(accumulate);
+ state = State.COMMITTED;
+ return accumulate;
+ }
+
+ /**
+ * rollback any effects of this transaction object graph; delegates first to doCleanup, then to doAbort
+ */
+ public final Throwable abort(Throwable accumulate)
+ {
+ if (state == State.ABORTED)
+ return accumulate;
+ if (state == State.COMMITTED)
+ {
+ try
+ {
+ throw new IllegalStateException("Attempted to abort a committed operation");
+ }
+ catch (Throwable t)
+ {
+ accumulate = merge(accumulate, t);
+ }
+ return accumulate;
+ }
+ state = State.ABORTED;
+ // we cleanup first so that, e.g., file handles can be released prior to deletion
+ accumulate = doCleanup(accumulate);
+ accumulate = doAbort(accumulate);
+ return accumulate;
+ }
+
+ // if we are committed or aborted, then we are done; otherwise abort
+ public final void close()
+ {
+ switch (state)
+ {
+ case COMMITTED:
+ case ABORTED:
+ break;
+ default:
+ abort();
+ }
+ }
+
+ /**
+ * The first phase of commit: delegates to doPrepare(), with valid state transition enforcement.
+ * This call should be propagated onto any child objects participating in the transaction
+ */
+ public final void prepareToCommit()
+ {
+ if (state != State.IN_PROGRESS)
+ throw new IllegalStateException("Cannot prepare to commit unless IN_PROGRESS; state is " + state);
+
+ doPrepare();
+ state = State.READY_TO_COMMIT;
+ }
+
+ /**
+ * convenience method to both prepareToCommit() and commit() in one operation;
+ * only of use to outer-most transactional object of an object graph
+ */
+ public Object finish()
+ {
+ prepareToCommit();
+ commit();
+ return this;
+ }
+
+ // convenience method wrapping abort, and throwing any exception encountered
+ // only of use to (and to be used by) outer-most object in a transactional graph
+ public final void abort()
+ {
+ maybeFail(abort(null));
+ }
+
+ // convenience method wrapping commit, and throwing any exception encountered
+ // only of use to (and to be used by) outer-most object in a transactional graph
+ public final void commit()
+ {
+ maybeFail(commit(null));
+ }
+
+ public final State state()
+ {
+ return state;
+ }
+ }
+
+ // commit should generally never throw an exception, and preferably never generate one,
+ // 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);
+
+ // release any resources, then rollback all state changes (unless commit() has already been invoked)
+ public Throwable abort(Throwable accumulate);
+
+ public void prepareToCommit();
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 128d1b0..09121f4 100644
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@ -336,7 +336,7 @@ public class ScrubTest
writer.append(Util.dk("c"), cf);
writer.append(Util.dk("y"), cf);
writer.append(Util.dk("d"), cf);
- writer.closeAndOpenReader();
+ writer.finish();
*/
String root = System.getProperty("corrupt-sstable-root");
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 231b3f3..1dc72ae 100644
--- a/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
@@ -156,11 +156,12 @@ public class AntiCompactionTest
File dir = cfs.directories.getDirectoryForNewSSTables();
String filename = cfs.getTempSSTablePath(dir);
- SSTableWriter writer = SSTableWriter.create(filename, 0, 0);
-
- for (int i = 0; i < count * 5; i++)
- writer.append(StorageService.getPartitioner().decorateKey(ByteBufferUtil.bytes(i)), cf);
- return writer.closeAndOpenReader();
+ try (SSTableWriter writer = SSTableWriter.create(filename, 0, 0);)
+ {
+ for (int i = 0; i < count * 5; i++)
+ writer.append(StorageService.getPartitioner().decorateKey(ByteBufferUtil.bytes(i)), cf);
+ return writer.finish(true);
+ }
}
public void generateSStable(ColumnFamilyStore store, String Suffix)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 42ea0c7..18418e8 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
@@ -240,9 +240,9 @@ public class CompactionsTest
long newSize1 = it.next().uncompressedLength();
long newSize2 = it.next().uncompressedLength();
assertEquals("candidate sstable should not be tombstone-compacted because its key range overlap with other sstable",
- originalSize1, newSize1);
+ originalSize1, newSize1);
assertEquals("candidate sstable should not be tombstone-compacted because its key range overlap with other sstable",
- originalSize2, newSize2);
+ originalSize2, newSize2);
// now let's enable the magic property
store.metadata.compactionStrategyOptions.put("unchecked_tombstone_compaction", "true");
@@ -401,21 +401,24 @@ 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);
-
+ try (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);
- writer.append(Util.dk("0"), cf);
- writer.append(Util.dk("1"), cf);
- writer.append(Util.dk("3"), cf);
+ cfs.addSSTable(writer.finish(true));
+ }
- cfs.addSSTable(writer.closeAndOpenReader());
- writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(dir.getDirectoryForNewSSTables())), 0, 0, 0);
+ try (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("2"), cf);
+ writer.append(Util.dk("3"), cf);
+ cfs.addSSTable(writer.finish(true));
+ }
- 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;
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/test/unit/org/apache/cassandra/io/RandomAccessReaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/RandomAccessReaderTest.java b/test/unit/org/apache/cassandra/io/RandomAccessReaderTest.java
index 678b926..fe04096 100644
--- a/test/unit/org/apache/cassandra/io/RandomAccessReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/RandomAccessReaderTest.java
@@ -30,7 +30,7 @@ public class RandomAccessReaderTest
SequentialWriter writer = new SequentialWriter(f, CompressionParameters.DEFAULT_CHUNK_LENGTH, false);
writer.write(expected.getBytes());
- writer.close();
+ writer.finish();
assert f.exists();
@@ -58,7 +58,7 @@ public class RandomAccessReaderTest
SequentialWriter writer = new SequentialWriter(f, CompressionParameters.DEFAULT_CHUNK_LENGTH, false);
writer.write(expected.getBytes());
- writer.close();
+ writer.finish();
assert f.exists();
@@ -87,7 +87,7 @@ public class RandomAccessReaderTest
SequentialWriter writer = new SequentialWriter(f, CompressionParameters.DEFAULT_CHUNK_LENGTH, false);
for (int i = 0; i < numIterations; i++)
writer.write(expected.getBytes());
- writer.close();
+ writer.finish();
assert f.exists();
@@ -166,7 +166,7 @@ public class RandomAccessReaderTest
SequentialWriter writer = new SequentialWriter(f, CompressionParameters.DEFAULT_CHUNK_LENGTH, false);
for (int i = 0; i < expected.length; i++)
writer.write(expected[i].getBytes());
- writer.close();
+ writer.finish();
assert f.exists();
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
index f4d3e87..cfc4bb8 100644
--- a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
@@ -81,7 +81,7 @@ public class CompressedRandomAccessReaderTest
for (int i = 0; i < 20; i++)
writer.write("x".getBytes());
- writer.close();
+ writer.finish();
CompressedRandomAccessReader reader = CompressedRandomAccessReader.open(channel, new CompressionMetadata(filename + ".metadata", f.length()));
String res = reader.readLine();
@@ -124,7 +124,7 @@ public class CompressedRandomAccessReaderTest
writer.resetAndTruncate(mark);
writer.write("brown fox jumps over the lazy dog".getBytes());
- writer.close();
+ writer.finish();
assert f.exists();
RandomAccessReader reader = compressed
@@ -161,10 +161,11 @@ public class CompressedRandomAccessReaderTest
metadata.deleteOnExit();
MetadataCollector sstableMetadataCollector = new MetadataCollector(new SimpleDenseCellNameType(BytesType.instance)).replayPosition(null);
- SequentialWriter writer = new CompressedSequentialWriter(file, metadata.getPath(), new CompressionParameters(SnappyCompressor.instance), sstableMetadataCollector);
-
- writer.write(CONTENT.getBytes());
- writer.close();
+ try (SequentialWriter writer = new CompressedSequentialWriter(file, metadata.getPath(), new CompressionParameters(SnappyCompressor.instance), sstableMetadataCollector))
+ {
+ writer.write(CONTENT.getBytes());
+ writer.finish();
+ }
ChannelProxy channel = new ChannelProxy(file);
@@ -175,8 +176,6 @@ public class CompressedRandomAccessReaderTest
RandomAccessReader reader = CompressedRandomAccessReader.open(channel, meta);
// read and verify compressed data
assertEquals(CONTENT, reader.readLine());
- // close reader
- reader.close();
Random random = new Random();
RandomAccessFile checksumModifier = null;
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java b/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
index 46da343..184319f 100644
--- a/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
@@ -17,23 +17,31 @@
*/
package org.apache.cassandra.io.compress;
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
import java.io.File;
import java.io.IOException;
import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.Random;
+import java.util.*;
+import static org.apache.commons.io.FileUtils.readFileToByteArray;
import static org.junit.Assert.assertEquals;
+
+import org.junit.After;
import org.junit.Test;
+import junit.framework.Assert;
+import org.apache.cassandra.db.composites.CellNames;
import org.apache.cassandra.db.composites.SimpleDenseCellNameType;
import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.UTF8Type;
import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
import org.apache.cassandra.io.util.ChannelProxy;
import org.apache.cassandra.io.util.FileMark;
import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.io.util.SequentialWriterTest;
-public class CompressedSequentialWriterTest
+public class CompressedSequentialWriterTest extends SequentialWriterTest
{
private ICompressor compressor;
@@ -78,30 +86,31 @@ public class CompressedSequentialWriterTest
try
{
MetadataCollector sstableMetadataCollector = new MetadataCollector(new SimpleDenseCellNameType(BytesType.instance)).replayPosition(null);
- CompressedSequentialWriter writer = new CompressedSequentialWriter(f, filename + ".metadata", new CompressionParameters(compressor), sstableMetadataCollector);
-
byte[] dataPre = new byte[bytesToTest];
byte[] rawPost = new byte[bytesToTest];
- Random r = new Random();
-
- // Test both write with byte[] and ByteBuffer
- r.nextBytes(dataPre);
- r.nextBytes(rawPost);
- ByteBuffer dataPost = makeBB(bytesToTest);
- dataPost.put(rawPost);
- dataPost.flip();
-
- writer.write(dataPre);
- FileMark mark = writer.mark();
-
- // Write enough garbage to transition chunk
- for (int i = 0; i < CompressionParameters.DEFAULT_CHUNK_LENGTH; i++)
+ try (CompressedSequentialWriter writer = new CompressedSequentialWriter(f, filename + ".metadata", new CompressionParameters(compressor), sstableMetadataCollector);)
{
- writer.write((byte)i);
+ Random r = new Random();
+
+ // Test both write with byte[] and ByteBuffer
+ r.nextBytes(dataPre);
+ r.nextBytes(rawPost);
+ ByteBuffer dataPost = makeBB(bytesToTest);
+ dataPost.put(rawPost);
+ dataPost.flip();
+
+ writer.write(dataPre);
+ FileMark mark = writer.mark();
+
+ // Write enough garbage to transition chunk
+ for (int i = 0; i < CompressionParameters.DEFAULT_CHUNK_LENGTH; i++)
+ {
+ writer.write((byte)i);
+ }
+ writer.resetAndTruncate(mark);
+ writer.write(dataPost);
+ writer.finish();
}
- writer.resetAndTruncate(mark);
- writer.write(dataPost);
- writer.close();
assert f.exists();
RandomAccessReader reader = CompressedRandomAccessReader.open(channel, new CompressionMetadata(filename + ".metadata", f.length()));
@@ -137,4 +146,85 @@ public class CompressedSequentialWriterTest
? ByteBuffer.allocateDirect(size)
: ByteBuffer.allocate(size);
}
+
+ private final List<TestableCSW> writers = new ArrayList<>();
+
+ @After
+ public void cleanup()
+ {
+ for (TestableCSW sw : writers)
+ sw.cleanup();
+ writers.clear();
+ }
+
+ protected TestableTransaction newTest() throws IOException
+ {
+ TestableCSW sw = new TestableCSW();
+ writers.add(sw);
+ return sw;
+ }
+
+ private static class TestableCSW extends TestableSW
+ {
+ final File offsetsFile;
+
+ private TestableCSW() throws IOException
+ {
+ this(tempFile("compressedsequentialwriter"),
+ tempFile("compressedsequentialwriter.offsets"));
+ }
+
+ private TestableCSW(File file, File offsetsFile) throws IOException
+ {
+ this(file, offsetsFile, new CompressedSequentialWriter(file, offsetsFile.getPath(), new CompressionParameters(LZ4Compressor.instance, BUFFER_SIZE, new HashMap<String, String>()), new MetadataCollector(CellNames.fromAbstractType(UTF8Type.instance, false))));
+ }
+
+ private TestableCSW(File file, File offsetsFile, CompressedSequentialWriter sw) throws IOException
+ {
+ super(file, sw);
+ this.offsetsFile = offsetsFile;
+ }
+
+ protected void assertInProgress() throws Exception
+ {
+ Assert.assertTrue(file.exists());
+ Assert.assertFalse(offsetsFile.exists());
+ byte[] compressed = readFileToByteArray(file);
+ byte[] uncompressed = new byte[partialContents.length];
+ LZ4Compressor.instance.uncompress(compressed, 0, compressed.length - 4, uncompressed, 0);
+ Assert.assertTrue(Arrays.equals(partialContents, uncompressed));
+ }
+
+ protected void assertPrepared() throws Exception
+ {
+ Assert.assertTrue(file.exists());
+ Assert.assertTrue(offsetsFile.exists());
+ DataInputStream offsets = new DataInputStream(new ByteArrayInputStream(readFileToByteArray(offsetsFile)));
+ Assert.assertTrue(offsets.readUTF().endsWith("LZ4Compressor"));
+ Assert.assertEquals(0, offsets.readInt());
+ Assert.assertEquals(BUFFER_SIZE, offsets.readInt());
+ Assert.assertEquals(fullContents.length, offsets.readLong());
+ Assert.assertEquals(2, offsets.readInt());
+ Assert.assertEquals(0, offsets.readLong());
+ int offset = (int) offsets.readLong();
+ byte[] compressed = readFileToByteArray(file);
+ byte[] uncompressed = new byte[fullContents.length];
+ LZ4Compressor.instance.uncompress(compressed, 0, offset - 4, uncompressed, 0);
+ LZ4Compressor.instance.uncompress(compressed, offset, compressed.length - (4 + offset), uncompressed, partialContents.length);
+ Assert.assertTrue(Arrays.equals(fullContents, uncompressed));
+ }
+
+ protected void assertAborted() throws Exception
+ {
+ super.assertAborted();
+ Assert.assertFalse(offsetsFile.exists());
+ }
+
+ void cleanup()
+ {
+ file.delete();
+ offsetsFile.delete();
+ }
+ }
+
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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
new file mode 100644
index 0000000..dfb55a1
--- /dev/null
+++ b/test/unit/org/apache/cassandra/io/sstable/BigTableWriterTest.java
@@ -0,0 +1,130 @@
+/*
+* 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.io.IOException;
+
+import org.junit.After;
+import org.junit.BeforeClass;
+
+import junit.framework.Assert;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.db.ArrayBackedSortedColumns;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.concurrent.AbstractTransactionalTest;
+
+public class BigTableWriterTest extends AbstractTransactionalTest
+{
+ public static final String KEYSPACE1 = "BigTableWriterTest";
+ public static final String CF_STANDARD = "Standard1";
+
+ private static ColumnFamilyStore cfs;
+
+ @BeforeClass
+ public static void defineSchema() throws Exception
+ {
+ SchemaLoader.prepareServer();
+ SchemaLoader.createKeyspace(KEYSPACE1,
+ SimpleStrategy.class,
+ KSMetaData.optsWithRF(1),
+ SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD));
+ cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD);
+ }
+
+ protected TestableTransaction newTest() throws IOException
+ {
+ return new TestableBTW();
+ }
+
+ private static class TestableBTW extends TestableTransaction
+ {
+ final File file;
+ final Descriptor descriptor;
+ final SSTableWriter writer;
+
+ private TestableBTW() throws IOException
+ {
+ this(cfs.getTempSSTablePath(cfs.directories.getDirectoryForNewSSTables()));
+ }
+
+ private TestableBTW(String file) throws IOException
+ {
+ this(file, SSTableWriter.create(file, 0, 0));
+ }
+
+ private TestableBTW(String file, SSTableWriter sw) throws IOException
+ {
+ super(sw);
+ this.file = new File(file);
+ this.descriptor = Descriptor.fromFilename(file);
+ this.writer = sw;
+ ArrayBackedSortedColumns cf = ArrayBackedSortedColumns.factory.create(cfs.metadata);
+ for (int i = 0; i < 10; i++)
+ cf.addColumn(Util.cellname(i), SSTableRewriterTest.random(0, 1000), 1);
+ for (int i = 0; i < 100; i++)
+ writer.append(StorageService.getPartitioner().decorateKey(ByteBufferUtil.bytes(i)), cf);
+ }
+
+ 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);
+ 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);
+ }
+
+ 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);
+ Assert.assertFalse(file.exists());
+ }
+
+ protected void assertCommitted() throws Exception
+ {
+ assertPrepared();
+ }
+
+ private void assertExists(Descriptor.Type type, Component ... components)
+ {
+ for (Component component : components)
+ Assert.assertTrue(new File(descriptor.asType(type).filenameFor(component)).exists());
+ }
+ private void assertNotExists(Descriptor.Type type, Component ... components)
+ {
+ for (Component component : components)
+ Assert.assertFalse(type.toString() + " " + component.toString(), new File(descriptor.asType(type).filenameFor(component)).exists());
+ }
+ }
+
+}
[6/7] cassandra git commit: Introduce Transactional API for internal
state changes
Posted by be...@apache.org.
Introduce Transactional API for internal state changes
patch by benedict; reviewed by josh for CASSANDRA-8984
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/8704006b
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/8704006b
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/8704006b
Branch: refs/heads/trunk
Commit: 8704006bfa75a78cb904e35662e4c8bafc1f2330
Parents: ff10d63
Author: Benedict Elliott Smith <be...@apache.org>
Authored: Sun May 17 13:50:03 2015 +0100
Committer: Benedict Elliott Smith <be...@apache.org>
Committed: Sun May 17 13:50:03 2015 +0100
----------------------------------------------------------------------
CHANGES.txt | 1 +
src/java/org/apache/cassandra/db/Memtable.java | 14 +-
.../db/compaction/CompactionManager.java | 24 +-
.../cassandra/db/compaction/CompactionTask.java | 31 +--
.../cassandra/db/compaction/Scrubber.java | 21 +-
.../cassandra/db/compaction/Upgrader.java | 11 +-
.../writers/CompactionAwareWriter.java | 42 +++-
.../writers/DefaultCompactionWriter.java | 18 +-
.../writers/MajorLeveledCompactionWriter.java | 28 +--
.../writers/MaxSSTableSizeWriter.java | 16 +-
.../SplittingSizeTieredCompactionWriter.java | 17 +-
.../io/compress/CompressedSequentialWriter.java | 59 ++---
.../io/compress/CompressionMetadata.java | 131 +++++-----
.../cassandra/io/sstable/IndexSummary.java | 11 +-
.../io/sstable/IndexSummaryBuilder.java | 13 +-
.../apache/cassandra/io/sstable/SSTable.java | 2 +-
.../cassandra/io/sstable/SSTableRewriter.java | 250 +++++++++----------
.../io/sstable/SSTableSimpleUnsortedWriter.java | 56 +++--
.../io/sstable/SSTableSimpleWriter.java | 9 +-
.../io/sstable/format/SSTableReader.java | 13 +-
.../io/sstable/format/SSTableWriter.java | 137 +++++++---
.../io/sstable/format/big/BigTableWriter.java | 248 +++++++++---------
.../io/util/BufferedPoolingSegmentedFile.java | 3 +-
.../io/util/BufferedSegmentedFile.java | 3 +-
.../io/util/ChecksummedSequentialWriter.java | 38 ++-
.../io/util/CompressedPoolingSegmentedFile.java | 4 +-
.../io/util/CompressedSegmentedFile.java | 9 +-
.../org/apache/cassandra/io/util/FileUtils.java | 39 ++-
.../cassandra/io/util/MmappedSegmentedFile.java | 3 +-
.../apache/cassandra/io/util/SafeMemory.java | 5 +
.../cassandra/io/util/SafeMemoryWriter.java | 5 +
.../apache/cassandra/io/util/SegmentedFile.java | 23 +-
.../cassandra/io/util/SequentialWriter.java | 116 ++++++---
.../cassandra/streaming/StreamReceiveTask.java | 2 +-
.../apache/cassandra/tools/SSTableImport.java | 97 +++----
.../cassandra/utils/AlwaysPresentFilter.java | 5 +
.../org/apache/cassandra/utils/Throwables.java | 5 +
.../apache/cassandra/utils/concurrent/Ref.java | 28 ++-
.../apache/cassandra/utils/concurrent/Refs.java | 11 +-
.../utils/concurrent/SharedCloseable.java | 1 +
.../utils/concurrent/SharedCloseableImpl.java | 5 +
.../utils/concurrent/Transactional.java | 198 +++++++++++++++
.../unit/org/apache/cassandra/db/ScrubTest.java | 2 +-
.../db/compaction/AntiCompactionTest.java | 11 +-
.../db/compaction/CompactionsTest.java | 31 +--
.../cassandra/io/RandomAccessReaderTest.java | 8 +-
.../CompressedRandomAccessReaderTest.java | 15 +-
.../CompressedSequentialWriterTest.java | 136 ++++++++--
.../io/sstable/BigTableWriterTest.java | 130 ++++++++++
.../io/sstable/SSTableRewriterTest.java | 154 +++++-------
.../cassandra/io/sstable/SSTableUtils.java | 2 +-
.../io/util/BufferedRandomAccessFileTest.java | 28 +--
.../util/ChecksummedSequentialWriterTest.java | 92 +++++++
.../cassandra/io/util/DataOutputTest.java | 1 +
.../cassandra/io/util/SequentialWriterTest.java | 117 +++++++++
.../compress/CompressedInputStreamTest.java | 2 +-
.../cassandra/tools/SSTableExportTest.java | 16 +-
.../concurrent/AbstractTransactionalTest.java | 136 ++++++++++
58 files changed, 1731 insertions(+), 902 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index e3b25b5..9f14fba 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
2.2.0-beta1
+ * Introduce Transactional API for internal state changes (CASSANDRA-8984)
* Add a flag in cassandra.yaml to enable UDFs (CASSANDRA-9404)
* Better support of null for UDF (CASSANDRA-8374)
* Use ecj instead of javassist for UDFs (CASSANDRA-8241)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 ef47aba..3509b27 100644
--- a/src/java/org/apache/cassandra/db/Memtable.java
+++ b/src/java/org/apache/cassandra/db/Memtable.java
@@ -27,7 +27,6 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
-import com.google.common.base.Throwables;
import org.apache.cassandra.io.sstable.Descriptor;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.sstable.format.SSTableWriter;
@@ -341,8 +340,7 @@ public class Memtable
SSTableReader ssTable;
// errors when creating the writer that may leave empty temp files.
- SSTableWriter writer = createFlushWriter(cfs.getTempSSTablePath(sstableDirectory));
- try
+ try (SSTableWriter writer = createFlushWriter(cfs.getTempSSTablePath(sstableDirectory)))
{
boolean trackContention = logger.isDebugEnabled();
int heavilyContendedRowCount = 0;
@@ -372,16 +370,13 @@ public class Memtable
if (writer.getFilePointer() > 0)
{
- writer.isolateReferences();
-
// temp sstables should contain non-repaired data.
- ssTable = writer.closeAndOpenReader();
+ ssTable = writer.finish(true);
logger.info(String.format("Completed flushing %s (%d bytes) for commitlog position %s",
ssTable.getFilename(), new File(ssTable.getFilename()).length(), context));
}
else
{
- writer.abort();
ssTable = null;
logger.info("Completed flushing; nothing needed to be retained. Commitlog position was {}",
context);
@@ -392,11 +387,6 @@ public class Memtable
return ssTable;
}
- catch (Throwable e)
- {
- writer.abort();
- throw Throwables.propagate(e);
- }
}
public SSTableWriter createFlushWriter(String filename)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 35e288d..fc83cc5 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -21,7 +21,6 @@ import java.io.File;
import java.io.IOException;
import java.lang.management.ManagementFactory;
import java.util.ArrayList;
-import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
@@ -42,7 +41,6 @@ import javax.management.openmbean.OpenDataException;
import javax.management.openmbean.TabularData;
import com.google.common.base.Predicate;
-import com.google.common.base.Throwables;
import com.google.common.collect.*;
import com.google.common.util.concurrent.*;
import org.slf4j.Logger;
@@ -787,9 +785,9 @@ public class CompactionManager implements CompactionManagerMBean
metrics.beginCompaction(ci);
Set<SSTableReader> oldSSTable = Sets.newHashSet(sstable);
- SSTableRewriter writer = new SSTableRewriter(cfs, oldSSTable, sstable.maxDataAge, false);
List<SSTableReader> finished;
- try (CompactionController controller = new CompactionController(cfs, sstableSet, getDefaultGcBefore(cfs)))
+ try (SSTableRewriter writer = new SSTableRewriter(cfs, oldSSTable, sstable.maxDataAge, false);
+ CompactionController controller = new CompactionController(cfs, sstableSet, getDefaultGcBefore(cfs)))
{
writer.switchWriter(createWriter(cfs, compactionFileLocation, expectedBloomFilterSize, sstable.getSSTableMetadata().repairedAt, sstable));
@@ -813,11 +811,6 @@ public class CompactionManager implements CompactionManagerMBean
finished = writer.finish();
cfs.getDataTracker().markCompactedSSTablesReplaced(oldSSTable, finished, OperationType.CLEANUP);
}
- catch (Throwable e)
- {
- writer.abort();
- throw Throwables.propagate(e);
- }
finally
{
scanner.close();
@@ -1178,13 +1171,12 @@ public class CompactionManager implements CompactionManagerMBean
Set<SSTableReader> sstableAsSet = new HashSet<>(anticompactionGroup);
File destination = cfs.directories.getWriteableLocationAsFile(cfs.getExpectedCompactedFileSize(sstableAsSet, OperationType.ANTICOMPACTION));
- SSTableRewriter repairedSSTableWriter = new SSTableRewriter(cfs, sstableAsSet, groupMaxDataAge, false);
- SSTableRewriter unRepairedSSTableWriter = new SSTableRewriter(cfs, sstableAsSet, groupMaxDataAge, false);
-
long repairedKeyCount = 0;
long unrepairedKeyCount = 0;
AbstractCompactionStrategy strategy = cfs.getCompactionStrategy();
- try (AbstractCompactionStrategy.ScannerList scanners = strategy.getScanners(anticompactionGroup);
+ try (SSTableRewriter repairedSSTableWriter = new SSTableRewriter(cfs, sstableAsSet, groupMaxDataAge, false);
+ SSTableRewriter unRepairedSSTableWriter = new SSTableRewriter(cfs, sstableAsSet, groupMaxDataAge, false);
+ AbstractCompactionStrategy.ScannerList scanners = strategy.getScanners(anticompactionGroup);
CompactionController controller = new CompactionController(cfs, sstableAsSet, CFMetaData.DEFAULT_GC_GRACE_SECONDS))
{
int expectedBloomFilterSize = Math.max(cfs.metadata.getMinIndexInterval(), (int)(SSTableReader.getApproximateKeyCount(anticompactionGroup)));
@@ -1221,8 +1213,8 @@ public class CompactionManager implements CompactionManagerMBean
// we have the same readers being rewritten by both writers, so we ask the first one NOT to close them
// so that the second one can do so safely, without leaving us with references < 0 or any other ugliness
List<SSTableReader> anticompactedSSTables = new ArrayList<>();
- anticompactedSSTables.addAll(repairedSSTableWriter.finish(repairedAt));
- anticompactedSSTables.addAll(unRepairedSSTableWriter.finish(ActiveRepairService.UNREPAIRED_SSTABLE));
+ anticompactedSSTables.addAll(repairedSSTableWriter.setRepairedAt(repairedAt).finish());
+ anticompactedSSTables.addAll(unRepairedSSTableWriter.setRepairedAt(ActiveRepairService.UNREPAIRED_SSTABLE).finish());
cfs.getDataTracker().markCompactedSSTablesReplaced(sstableAsSet, anticompactedSSTables, OperationType.ANTICOMPACTION);
logger.debug("Repaired {} keys out of {} for {}/{} in {}", repairedKeyCount,
@@ -1236,8 +1228,6 @@ public class CompactionManager implements CompactionManagerMBean
{
JVMStabilityInspector.inspectThrowable(e);
logger.error("Error anticompacting " + anticompactionGroup, e);
- repairedSSTableWriter.abort();
- unRepairedSSTableWriter.abort();
}
return 0;
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 f472711..c397d9a 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@ -167,20 +167,12 @@ public class CompactionTask extends AbstractCompactionTask
if (collector != null)
collector.beginCompaction(ci);
long lastCheckObsoletion = start;
- CompactionAwareWriter writer = null;
- try
+
+ if (!controller.cfs.getCompactionStrategy().isActive)
+ throw new CompactionInterruptedException(ci.getCompactionInfo());
+
+ try (CompactionAwareWriter writer = getCompactionAwareWriter(cfs, sstables, actuallyCompact))
{
- if (!controller.cfs.getCompactionStrategy().isActive)
- throw new CompactionInterruptedException(ci.getCompactionInfo());
- if (!iter.hasNext())
- {
- // don't mark compacted in the finally block, since if there _is_ nondeleted data,
- // we need to sync it (via closeAndOpen) first, so there is no period during which
- // a crash could cause data loss.
- cfs.markObsolete(sstables, compactionType);
- return;
- }
- writer = getCompactionAwareWriter(cfs, sstables, actuallyCompact);
estimatedKeys = writer.estimatedKeys();
while (iter.hasNext())
{
@@ -201,19 +193,6 @@ public class CompactionTask extends AbstractCompactionTask
// don't replace old sstables yet, as we need to mark the compaction finished in the system table
newSStables = writer.finish();
}
- catch (Throwable t)
- {
- try
- {
- if (writer != null)
- writer.abort();
- }
- catch (Throwable t2)
- {
- t.addSuppressed(t2);
- }
- throw t;
- }
finally
{
// point of no return -- the new sstables are live on disk; next we'll start deleting the old ones
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 691566e..29472b3 100644
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@ -127,8 +127,8 @@ public class Scrubber implements Closeable
{
outputHandler.output(String.format("Scrubbing %s (%s bytes)", sstable, dataFile.length()));
Set<SSTableReader> oldSSTable = Sets.newHashSet(sstable);
- SSTableRewriter writer = new SSTableRewriter(cfs, oldSSTable, sstable.maxDataAge, isOffline);
- try
+
+ try (SSTableRewriter writer = new SSTableRewriter(cfs, oldSSTable, sstable.maxDataAge, isOffline);)
{
nextIndexKey = ByteBufferUtil.readWithShortLength(indexFile);
{
@@ -271,26 +271,27 @@ 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;
- SSTableWriter inOrderWriter = CompactionManager.createWriter(cfs, destination, expectedBloomFilterSize, repairedAt, sstable);
- for (Row row : outOfOrderRows)
- inOrderWriter.append(row.key, row.cf);
- newInOrderSstable = inOrderWriter.closeAndOpenReader(sstable.maxDataAge);
+ try (SSTableWriter inOrderWriter = CompactionManager.createWriter(cfs, destination, expectedBloomFilterSize, repairedAt, sstable);)
+ {
+ for (Row row : outOfOrderRows)
+ inOrderWriter.append(row.key, row.cf);
+ newInOrderSstable = inOrderWriter.finish(-1, sstable.maxDataAge, true);
+ }
if (!isOffline)
cfs.getDataTracker().addSSTables(Collections.singleton(newInOrderSstable));
outputHandler.warn(String.format("%d out of order rows found while scrubbing %s; Those have been written (in order) to a new sstable (%s)", outOfOrderRows.size(), sstable, newInOrderSstable));
}
// finish obsoletes the old sstable
- List<SSTableReader> finished = writer.finish(badRows > 0 ? ActiveRepairService.UNREPAIRED_SSTABLE : sstable.getSSTableMetadata().repairedAt);
+ List<SSTableReader> finished = writer.setRepairedAt(badRows > 0 ? ActiveRepairService.UNREPAIRED_SSTABLE : sstable.getSSTableMetadata().repairedAt).finish();
if (!finished.isEmpty())
newSstable = finished.get(0);
if (!isOffline)
cfs.getDataTracker().markCompactedSSTablesReplaced(oldSSTable, finished, OperationType.SCRUB);
}
- catch (Throwable t)
+ catch (IOException e)
{
- writer.abort();
- throw Throwables.propagate(t);
+ throw Throwables.propagate(e);
}
finally
{
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 75964e1..30584fd 100644
--- a/src/java/org/apache/cassandra/db/compaction/Upgrader.java
+++ b/src/java/org/apache/cassandra/db/compaction/Upgrader.java
@@ -81,8 +81,9 @@ public class Upgrader
{
outputHandler.output("Upgrading " + sstable);
Set<SSTableReader> toUpgrade = Sets.newHashSet(sstable);
- SSTableRewriter writer = new SSTableRewriter(cfs, toUpgrade, CompactionTask.getMaxDataAge(toUpgrade), true);
- try (AbstractCompactionStrategy.ScannerList scanners = strategy.getScanners(toUpgrade))
+
+ try (SSTableRewriter writer = new SSTableRewriter(cfs, toUpgrade, CompactionTask.getMaxDataAge(toUpgrade), true);
+ AbstractCompactionStrategy.ScannerList scanners = strategy.getScanners(toUpgrade))
{
Iterator<AbstractCompactedRow> iter = new CompactionIterable(compactionType, scanners.scanners, controller, DatabaseDescriptor.getSSTableFormat()).iterator();
writer.switchWriter(createCompactionWriter(sstable.getSSTableMetadata().repairedAt));
@@ -94,12 +95,6 @@ public class Upgrader
writer.finish();
outputHandler.output("Upgrade of " + sstable + " complete.");
-
- }
- catch (Throwable t)
- {
- writer.abort();
- throw Throwables.propagate(t);
}
finally
{
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 2903ced..fe43186 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java
@@ -25,28 +25,32 @@ import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.Directories;
import org.apache.cassandra.db.compaction.AbstractCompactedRow;
import org.apache.cassandra.db.compaction.CompactionTask;
+import org.apache.cassandra.io.sstable.SSTableRewriter;
import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.utils.concurrent.Transactional;
/**
* Class that abstracts away the actual writing of files to make it possible to use CompactionTask for more
* use cases.
*/
-public abstract class CompactionAwareWriter
+public abstract class CompactionAwareWriter extends Transactional.AbstractTransactional implements Transactional
{
protected final ColumnFamilyStore cfs;
protected final Set<SSTableReader> nonExpiredSSTables;
protected final long estimatedTotalKeys;
protected final long maxAge;
protected final long minRepairedAt;
+ protected final SSTableRewriter sstableWriter;
- public CompactionAwareWriter(ColumnFamilyStore cfs, Set<SSTableReader> nonExpiredSSTables)
+ public CompactionAwareWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables, boolean offline)
{
this.cfs = cfs;
this.nonExpiredSSTables = nonExpiredSSTables;
this.estimatedTotalKeys = SSTableReader.getApproximateKeyCount(nonExpiredSSTables);
this.maxAge = CompactionTask.getMaxDataAge(nonExpiredSSTables);
this.minRepairedAt = CompactionTask.getMinRepairedAt(nonExpiredSSTables);
+ this.sstableWriter = new SSTableRewriter(cfs, allSSTables, maxAge, offline);
}
/**
@@ -56,16 +60,40 @@ public abstract class CompactionAwareWriter
*/
public abstract boolean append(AbstractCompactedRow row);
- /**
- * abort the compaction writer - make sure that all files created are removed etc
- */
- public abstract void abort();
+ @Override
+ protected Throwable doAbort(Throwable accumulate)
+ {
+ return sstableWriter.abort(accumulate);
+ }
+
+ @Override
+ protected Throwable doCleanup(Throwable accumulate)
+ {
+ return accumulate;
+ }
+
+ @Override
+ protected Throwable doCommit(Throwable accumulate)
+ {
+ return sstableWriter.commit(accumulate);
+ }
+
+ @Override
+ protected void doPrepare()
+ {
+ sstableWriter.prepareToCommit();
+ }
/**
* we are done, return the finished sstables so that the caller can mark the old ones as compacted
* @return all the written sstables sstables
*/
- public abstract List<SSTableReader> finish();
+ @Override
+ public List<SSTableReader> finish()
+ {
+ super.finish();
+ return sstableWriter.finished();
+ }
/**
* estimated number of keys we should write
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 d51c82d..3589b54 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java
@@ -34,6 +34,8 @@ import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.sstable.format.SSTableWriter;
import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+import static org.apache.cassandra.utils.Throwables.maybeFail;
+
/**
* The default compaction writer - creates one output file in L0
@@ -41,13 +43,11 @@ import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
public class DefaultCompactionWriter extends CompactionAwareWriter
{
protected static final Logger logger = LoggerFactory.getLogger(DefaultCompactionWriter.class);
- private final SSTableRewriter sstableWriter;
public DefaultCompactionWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables, boolean offline, OperationType compactionType)
{
- super(cfs, nonExpiredSSTables);
+ super(cfs, allSSTables, nonExpiredSSTables, offline);
logger.debug("Expected bloom filter size : {}", estimatedTotalKeys);
- sstableWriter = new SSTableRewriter(cfs, allSSTables, maxAge, offline);
long expectedWriteSize = cfs.getExpectedCompactedFileSize(nonExpiredSSTables, compactionType);
File sstableDirectory = cfs.directories.getLocationForDisk(getWriteDirectory(expectedWriteSize));
SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(sstableDirectory)),
@@ -66,18 +66,6 @@ public class DefaultCompactionWriter extends CompactionAwareWriter
}
@Override
- public void abort()
- {
- sstableWriter.abort();
- }
-
- @Override
- public List<SSTableReader> finish()
- {
- return sstableWriter.finish();
- }
-
- @Override
public long estimatedKeys()
{
return estimatedTotalKeys;
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 b2f8fe1..d48140e 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java
@@ -40,7 +40,6 @@ import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
public class MajorLeveledCompactionWriter extends CompactionAwareWriter
{
private static final Logger logger = LoggerFactory.getLogger(MajorLeveledCompactionWriter.class);
- private final SSTableRewriter rewriter;
private final long maxSSTableSize;
private final long expectedWriteSize;
private final Set<SSTableReader> allSSTables;
@@ -53,10 +52,9 @@ public class MajorLeveledCompactionWriter extends CompactionAwareWriter
public MajorLeveledCompactionWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables, long maxSSTableSize, boolean offline, OperationType compactionType)
{
- super(cfs, nonExpiredSSTables);
+ super(cfs, allSSTables, nonExpiredSSTables, offline);
this.maxSSTableSize = maxSSTableSize;
this.allSSTables = allSSTables;
- rewriter = new SSTableRewriter(cfs, allSSTables, CompactionTask.getMaxDataAge(nonExpiredSSTables), offline);
expectedWriteSize = Math.min(maxSSTableSize, cfs.getExpectedCompactedFileSize(nonExpiredSSTables, compactionType));
long estimatedSSTables = Math.max(1, SSTableReader.getTotalBytes(nonExpiredSSTables) / maxSSTableSize);
long keysPerSSTable = estimatedTotalKeys / estimatedSSTables;
@@ -72,17 +70,17 @@ public class MajorLeveledCompactionWriter extends CompactionAwareWriter
cfs.metadata,
cfs.partitioner,
new MetadataCollector(allSSTables, cfs.metadata.comparator, currentLevel, skipAncestors));
- rewriter.switchWriter(writer);
+ sstableWriter.switchWriter(writer);
}
@Override
public boolean append(AbstractCompactedRow row)
{
- long posBefore = rewriter.currentWriter().getOnDiskFilePointer();
- RowIndexEntry rie = rewriter.append(row);
- totalWrittenInLevel += rewriter.currentWriter().getOnDiskFilePointer() - posBefore;
+ long posBefore = sstableWriter.currentWriter().getOnDiskFilePointer();
+ RowIndexEntry rie = sstableWriter.append(row);
+ totalWrittenInLevel += sstableWriter.currentWriter().getOnDiskFilePointer() - posBefore;
partitionsWritten++;
- if (rewriter.currentWriter().getOnDiskFilePointer() > maxSSTableSize)
+ if (sstableWriter.currentWriter().getOnDiskFilePointer() > maxSSTableSize)
{
if (totalWrittenInLevel > LeveledManifest.maxBytesForLevel(currentLevel, maxSSTableSize))
{
@@ -98,23 +96,11 @@ public class MajorLeveledCompactionWriter extends CompactionAwareWriter
cfs.metadata,
cfs.partitioner,
new MetadataCollector(allSSTables, cfs.metadata.comparator, currentLevel, skipAncestors));
- rewriter.switchWriter(writer);
+ sstableWriter.switchWriter(writer);
partitionsWritten = 0;
sstablesWritten++;
}
return rie != null;
}
-
- @Override
- public void abort()
- {
- rewriter.abort();
- }
-
- @Override
- public List<SSTableReader> finish()
- {
- return rewriter.finish();
- }
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 1a99059..ab24bf8 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java
@@ -36,7 +36,6 @@ public class MaxSSTableSizeWriter extends CompactionAwareWriter
{
private final long estimatedTotalKeys;
private final long expectedWriteSize;
- private final SSTableRewriter sstableWriter;
private final long maxSSTableSize;
private final int level;
private final long estimatedSSTables;
@@ -44,7 +43,7 @@ public class MaxSSTableSizeWriter extends CompactionAwareWriter
public MaxSSTableSizeWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables, long maxSSTableSize, int level, boolean offline, OperationType compactionType)
{
- super(cfs, nonExpiredSSTables);
+ super(cfs, allSSTables, nonExpiredSSTables, offline);
this.allSSTables = allSSTables;
this.level = level;
this.maxSSTableSize = maxSSTableSize;
@@ -52,7 +51,6 @@ public class MaxSSTableSizeWriter extends CompactionAwareWriter
expectedWriteSize = Math.min(maxSSTableSize, totalSize);
estimatedTotalKeys = SSTableReader.getApproximateKeyCount(nonExpiredSSTables);
estimatedSSTables = Math.max(1, estimatedTotalKeys / maxSSTableSize);
- sstableWriter = new SSTableRewriter(cfs, allSSTables, CompactionTask.getMaxDataAge(nonExpiredSSTables), offline);
File sstableDirectory = cfs.directories.getLocationForDisk(getWriteDirectory(expectedWriteSize));
SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(sstableDirectory)),
estimatedTotalKeys / estimatedSSTables,
@@ -83,18 +81,6 @@ public class MaxSSTableSizeWriter extends CompactionAwareWriter
}
@Override
- public void abort()
- {
- sstableWriter.abort();
- }
-
- @Override
- public List<SSTableReader> finish()
- {
- return sstableWriter.finish();
- }
-
- @Override
public long estimatedKeys()
{
return estimatedTotalKeys;
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 c97270c..2a452c7 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
@@ -48,7 +48,6 @@ public class SplittingSizeTieredCompactionWriter extends CompactionAwareWriter
public static final long DEFAULT_SMALLEST_SSTABLE_BYTES = 50_000_000;
private final double[] ratios;
- private final SSTableRewriter sstableWriter;
private final long totalSize;
private final Set<SSTableReader> allSSTables;
private long currentBytesToWrite;
@@ -61,7 +60,7 @@ public class SplittingSizeTieredCompactionWriter extends CompactionAwareWriter
public SplittingSizeTieredCompactionWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables, OperationType compactionType, long smallestSSTable)
{
- super(cfs, nonExpiredSSTables);
+ super(cfs, allSSTables, nonExpiredSSTables, false);
this.allSSTables = allSSTables;
totalSize = cfs.getExpectedCompactedFileSize(nonExpiredSSTables, compactionType);
double[] potentialRatios = new double[20];
@@ -83,7 +82,6 @@ public class SplittingSizeTieredCompactionWriter extends CompactionAwareWriter
}
}
ratios = Arrays.copyOfRange(potentialRatios, 0, noPointIndex);
- sstableWriter = new SSTableRewriter(cfs, allSSTables, CompactionTask.getMaxDataAge(nonExpiredSSTables), false);
File sstableDirectory = cfs.directories.getLocationForDisk(getWriteDirectory(Math.round(totalSize * ratios[currentRatioIndex])));
long currentPartitionsToWrite = Math.round(estimatedTotalKeys * ratios[currentRatioIndex]);
currentBytesToWrite = Math.round(totalSize * ratios[currentRatioIndex]);
@@ -119,17 +117,4 @@ public class SplittingSizeTieredCompactionWriter extends CompactionAwareWriter
}
return rie != null;
}
-
-
- @Override
- public void abort()
- {
- sstableWriter.abort();
- }
-
- @Override
- public List<SSTableReader> finish()
- {
- return sstableWriter.finish();
- }
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
index eb9dcf8..6218526 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
@@ -28,17 +28,12 @@ import java.util.zip.Adler32;
import org.apache.cassandra.io.FSReadError;
import org.apache.cassandra.io.FSWriteError;
import org.apache.cassandra.io.sstable.CorruptSSTableException;
-import org.apache.cassandra.io.sstable.Descriptor;
import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
import org.apache.cassandra.io.util.DataIntegrityMetadata;
import org.apache.cassandra.io.util.FileMark;
import org.apache.cassandra.io.util.SequentialWriter;
import org.apache.cassandra.utils.FBUtilities;
-import static org.apache.cassandra.io.compress.CompressionMetadata.Writer.OpenType.FINAL;
-import static org.apache.cassandra.io.compress.CompressionMetadata.Writer.OpenType.SHARED;
-import static org.apache.cassandra.io.compress.CompressionMetadata.Writer.OpenType.SHARED_FINAL;
-
public class CompressedSequentialWriter extends SequentialWriter
{
private final DataIntegrityMetadata.ChecksumWriter crcMetadata;
@@ -97,12 +92,6 @@ public class CompressedSequentialWriter extends SequentialWriter
}
@Override
- public void sync()
- {
- throw new UnsupportedOperationException();
- }
-
- @Override
public void flush()
{
throw new UnsupportedOperationException();
@@ -163,13 +152,11 @@ public class CompressedSequentialWriter extends SequentialWriter
runPostFlush.run();
}
- public CompressionMetadata open(long overrideLength, boolean isFinal)
+ public CompressionMetadata open(long overrideLength)
{
if (overrideLength <= 0)
- return metadataWriter.open(uncompressedSize, chunkOffset, isFinal ? FINAL : SHARED_FINAL);
- // we are early opening the file, make sure we open metadata with the correct size
- assert !isFinal;
- return metadataWriter.open(overrideLength, chunkOffset, SHARED);
+ overrideLength = uncompressedSize;
+ return metadataWriter.open(overrideLength, chunkOffset);
}
@Override
@@ -279,36 +266,36 @@ public class CompressedSequentialWriter extends SequentialWriter
}
}
- @Override
- public void close()
+ protected class TransactionalProxy extends SequentialWriter.TransactionalProxy
{
- if (buffer == null)
- return;
-
- long finalPosition = current();
-
- super.close();
- sstableMetadataCollector.addCompressionRatio(compressedSize, uncompressedSize);
- try
+ @Override
+ protected Throwable doCommit(Throwable accumulate)
{
- metadataWriter.close(finalPosition, chunkCount);
+ return metadataWriter.commit(accumulate);
}
- catch (IOException e)
+
+ @Override
+ protected Throwable doAbort(Throwable accumulate)
{
- throw new FSWriteError(e, getPath());
+ return super.doAbort(metadataWriter.abort(accumulate));
}
- }
- public void abort()
- {
- super.abort();
- metadataWriter.abort();
+ @Override
+ protected void doPrepare()
+ {
+ syncInternal();
+ if (descriptor != null)
+ crcMetadata.writeFullChecksum(descriptor);
+ releaseFileHandle();
+ sstableMetadataCollector.addCompressionRatio(compressedSize, uncompressedSize);
+ metadataWriter.finalizeLength(current(), chunkCount).prepareToCommit();
+ }
}
@Override
- public void writeFullChecksum(Descriptor descriptor)
+ protected SequentialWriter.TransactionalProxy txnProxy()
{
- crcMetadata.writeFullChecksum(descriptor);
+ return new TransactionalProxy();
}
/**
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java b/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
index 928541a..a6c7a8b 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
@@ -36,9 +36,9 @@ import java.util.SortedSet;
import java.util.TreeSet;
import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Throwables;
import com.google.common.primitives.Longs;
-import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.TypeSizes;
import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.io.FSReadError;
@@ -47,12 +47,12 @@ import org.apache.cassandra.io.IVersionedSerializer;
import org.apache.cassandra.io.sstable.Component;
import org.apache.cassandra.io.sstable.CorruptSSTableException;
import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.format.Version;
import org.apache.cassandra.io.util.DataOutputPlus;
import org.apache.cassandra.io.util.FileUtils;
import org.apache.cassandra.io.util.Memory;
import org.apache.cassandra.io.util.SafeMemory;
import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.concurrent.Transactional;
/**
* Holds metadata about compressed file
@@ -265,7 +265,7 @@ public class CompressionMetadata
chunkOffsets.close();
}
- public static class Writer
+ public static class Writer extends Transactional.AbstractTransactional implements Transactional
{
// path to the file
private final CompressionParameters parameters;
@@ -274,6 +274,8 @@ public class CompressionMetadata
private SafeMemory offsets = new SafeMemory(maxCount * 8L);
private int count = 0;
+ // provided by user when setDescriptor
+ private long dataLength, chunkCount;
private Writer(CompressionParameters parameters, String path)
{
@@ -321,61 +323,60 @@ public class CompressionMetadata
}
}
- static enum OpenType
+ // we've written everything; wire up some final metadata state
+ public Writer finalizeLength(long dataLength, int chunkCount)
{
- // i.e. FinishType == EARLY; we will use the RefCountedMemory in possibly multiple instances
- SHARED,
- // i.e. FinishType == EARLY, but the sstable has been completely written, so we can
- // finalise the contents and size of the memory, but must retain a reference to it
- SHARED_FINAL,
- // i.e. FinishType == NORMAL or FINISH_EARLY, i.e. we have actually finished writing the table
- // and will never need to open the metadata again, so we can release any references to it here
- FINAL
+ this.dataLength = dataLength;
+ this.chunkCount = chunkCount;
+ return this;
}
- public CompressionMetadata open(long dataLength, long compressedLength, OpenType type)
+ public void doPrepare()
{
- SafeMemory offsets;
- int count = this.count;
- switch (type)
+ assert chunkCount == count;
+
+ // finalize the size of memory used if it won't now change;
+ // unnecessary if already correct size
+ if (offsets.size() != count * 8L)
+ {
+ SafeMemory tmp = offsets;
+ offsets = offsets.copy(count * 8L);
+ tmp.free();
+ }
+
+ // flush the data to disk
+ DataOutputStream out = null;
+ try
+ {
+ out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(filePath)));
+ writeHeader(out, dataLength, count);
+ for (int i = 0 ; i < count ; i++)
+ out.writeLong(offsets.getLong(i * 8L));
+ }
+ catch (IOException e)
{
- case FINAL: case SHARED_FINAL:
- if (this.offsets.size() != count * 8L)
- {
- // finalize the size of memory used if it won't now change;
- // unnecessary if already correct size
- SafeMemory tmp = this.offsets.copy(count * 8L);
- this.offsets.free();
- this.offsets = tmp;
- }
-
- if (type == OpenType.SHARED_FINAL)
- {
- offsets = this.offsets.sharedCopy();
- }
- else
- {
- offsets = this.offsets;
- // null out our reference to the original shared data to catch accidental reuse
- // note that since noone is writing to this Writer while we open it, null:ing out this.offsets is safe
- this.offsets = null;
- }
- break;
-
- case SHARED:
- offsets = this.offsets.sharedCopy();
- // we should only be opened on a compression data boundary; truncate our size to this boundary
- count = (int) (dataLength / parameters.chunkLength());
- if (dataLength % parameters.chunkLength() != 0)
- count++;
- // grab our actual compressed length from the next offset from our the position we're opened to
- if (count < this.count)
- compressedLength = offsets.getLong(count * 8L);
- break;
-
- default:
- throw new AssertionError();
+ throw Throwables.propagate(e);
}
+ finally
+ {
+ FileUtils.closeQuietly(out);
+ }
+ }
+
+ public CompressionMetadata open(long dataLength, long compressedLength)
+ {
+ SafeMemory offsets = this.offsets.sharedCopy();
+
+ // calculate how many entries we need, if our dataLength is truncated
+ int count = (int) (dataLength / parameters.chunkLength());
+ if (dataLength % parameters.chunkLength() != 0)
+ count++;
+
+ assert count > 0;
+ // grab our actual compressed length from the next offset from our the position we're opened to
+ if (count < this.count)
+ compressedLength = offsets.getLong(count * 8L);
+
return new CompressionMetadata(filePath, parameters, offsets, count * 8L, dataLength, compressedLength);
}
@@ -402,27 +403,19 @@ public class CompressionMetadata
count = chunkIndex;
}
- public void close(long dataLength, int chunks) throws IOException
+ protected Throwable doCleanup(Throwable failed)
{
- DataOutputStream out = null;
- try
- {
- out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(filePath)));
- assert chunks == count;
- writeHeader(out, dataLength, chunks);
- for (int i = 0 ; i < count ; i++)
- out.writeLong(offsets.getLong(i * 8L));
- }
- finally
- {
- FileUtils.closeQuietly(out);
- }
+ return offsets.close(failed);
}
- public void abort()
+ protected Throwable doCommit(Throwable accumulate)
{
- if (offsets != null)
- offsets.close();
+ return accumulate;
+ }
+
+ protected Throwable doAbort(Throwable accumulate)
+ {
+ return FileUtils.deleteWithConfirm(filePath, false, accumulate);
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/io/sstable/IndexSummary.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummary.java b/src/java/org/apache/cassandra/io/sstable/IndexSummary.java
index fbefe13..59c5eef 100644
--- a/src/java/org/apache/cassandra/io/sstable/IndexSummary.java
+++ b/src/java/org/apache/cassandra/io/sstable/IndexSummary.java
@@ -18,16 +18,20 @@
package org.apache.cassandra.io.sstable;
import java.io.DataInputStream;
+import java.io.File;
+import java.io.FileOutputStream;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
import org.apache.cassandra.db.DecoratedKey;
import org.apache.cassandra.db.RowPosition;
import org.apache.cassandra.dht.IPartitioner;
-import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.io.util.Memory;
-import org.apache.cassandra.io.util.MemoryOutputStream;
+import org.apache.cassandra.io.util.*;
+import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.concurrent.WrappedSharedCloseable;
import org.apache.cassandra.utils.memory.MemoryUtil;
@@ -46,6 +50,7 @@ import static org.apache.cassandra.io.sstable.Downsampling.BASE_SAMPLING_LEVEL;
*/
public class IndexSummary extends WrappedSharedCloseable
{
+ private static final Logger logger = LoggerFactory.getLogger(IndexSummary.class);
public static final IndexSummarySerializer serializer = new IndexSummarySerializer();
/**
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java b/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java
index c7c51e5..12e41c8 100644
--- a/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java
+++ b/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java
@@ -203,12 +203,16 @@ public class IndexSummaryBuilder implements AutoCloseable
}
}
- public IndexSummary build(IPartitioner partitioner)
+ public void prepareToCommit()
{
// this method should only be called when we've finished appending records, so we truncate the
// memory we're using to the exact amount required to represent it before building our summary
entries.setCapacity(entries.length());
offsets.setCapacity(offsets.length());
+ }
+
+ public IndexSummary build(IPartitioner partitioner)
+ {
return build(partitioner, null);
}
@@ -240,6 +244,13 @@ public class IndexSummaryBuilder implements AutoCloseable
offsets.close();
}
+ public Throwable close(Throwable accumulate)
+ {
+ accumulate = entries.close(accumulate);
+ accumulate = offsets.close(accumulate);
+ return accumulate;
+ }
+
public static int entriesAtSamplingLevel(int samplingLevel, int maxSummarySize)
{
return (int) Math.ceil((samplingLevel * maxSummarySize) / (double) BASE_SAMPLING_LEVEL);
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 f486b78..bc3486a 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTable.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTable.java
@@ -195,7 +195,7 @@ public abstract class SSTable
}
}
- private static Set<Component> discoverComponentsFor(Descriptor desc)
+ public static Set<Component> discoverComponentsFor(Descriptor desc)
{
Set<Component.Type> knownTypes = Sets.difference(Component.TYPES, Collections.singleton(Component.Type.CUSTOM));
Set<Component> components = Sets.newHashSetWithExpectedSize(knownTypes.size());
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 8890659..a526ec9 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.io.sstable;
import java.util.*;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import org.apache.cassandra.config.DatabaseDescriptor;
@@ -33,6 +32,8 @@ import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.sstable.format.SSTableWriter;
import org.apache.cassandra.utils.CLibrary;
import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.concurrent.Refs;
+import org.apache.cassandra.utils.concurrent.Transactional;
import static org.apache.cassandra.utils.Throwables.merge;
@@ -51,7 +52,7 @@ import static org.apache.cassandra.utils.Throwables.merge;
* but leave any hard-links in place for the readers we opened to cleanup when they're finished as we would had we finished
* successfully.
*/
-public class SSTableRewriter
+public class SSTableRewriter extends Transactional.AbstractTransactional implements Transactional
{
private static long preemptiveOpenInterval;
static
@@ -77,7 +78,9 @@ public class SSTableRewriter
private final ColumnFamilyStore cfs;
private final long maxAge;
- private final List<SSTableReader> finished = new ArrayList<>();
+ private long repairedAt = -1;
+ // the set of final readers we will expose on commit
+ private final List<SSTableReader> preparedForCommit = new ArrayList<>();
private final Set<SSTableReader> rewriting; // the readers we are rewriting (updated as they are replaced)
private final Map<Descriptor, DecoratedKey> originalStarts = new HashMap<>(); // the start key for each reader we are rewriting
private final Map<Descriptor, Integer> fileDescriptors = new HashMap<>(); // the file descriptors for each reader descriptor we are rewriting
@@ -85,21 +88,18 @@ public class SSTableRewriter
private SSTableReader currentlyOpenedEarly; // the reader for the most recent (re)opening of the target file
private long currentlyOpenedEarlyAt; // the position (in MB) in the target file we last (re)opened at
- private final List<SSTableReader> finishedReaders = new ArrayList<>();
- private final Queue<Finished> finishedEarly = new ArrayDeque<>();
- // as writers are closed from finishedEarly, their last readers are moved
- // into discard, so that abort can cleanup after us safely
- private final List<SSTableReader> discard = new ArrayList<>();
- private final boolean isOffline; // true for operations that are performed without Cassandra running (prevents updates of DataTracker)
+ private final List<Finished> finishedWriters = new ArrayList<>();
+ // as writers are closed from finishedWriters, their last readers are moved into discard, so that abort can cleanup
+ // after us safely; we use a set so we can add in both prepareToCommit and abort
+ private final Set<SSTableReader> discard = new HashSet<>();
+ // true for operations that are performed without Cassandra running (prevents updates of DataTracker)
+ private final boolean isOffline;
private SSTableWriter writer;
private Map<DecoratedKey, RowIndexEntry> cachedKeys = new HashMap<>();
- private State state = State.WORKING;
- private static enum State
- {
- WORKING, FINISHED, ABORTED
- }
+ // for testing (TODO: remove when have byteman setup)
+ private boolean throwEarly, throwLate;
public SSTableRewriter(ColumnFamilyStore cfs, Set<SSTableReader> rewriting, long maxAge, boolean isOffline)
{
@@ -178,7 +178,7 @@ public class SSTableRewriter
}
else
{
- SSTableReader reader = writer.openEarly(maxAge);
+ SSTableReader reader = writer.setMaxDataAge(maxAge).openEarly();
if (reader != null)
{
replaceEarlyOpenedFile(currentlyOpenedEarly, reader);
@@ -190,29 +190,19 @@ public class SSTableRewriter
}
}
- public void abort()
+ protected Throwable doAbort(Throwable accumulate)
{
- switch (state)
- {
- case ABORTED:
- return;
- case FINISHED:
- throw new IllegalStateException("Cannot abort - changes have already been committed");
- }
- state = State.ABORTED;
-
- Throwable fail = null;
try
{
moveStarts(null, null, true);
}
catch (Throwable t)
{
- fail = merge(fail, t);
+ accumulate = merge(accumulate, t);
}
- // remove already completed SSTables
- for (SSTableReader sstable : finished)
+ // cleanup any sstables we prepared for commit
+ for (SSTableReader sstable : preparedForCommit)
{
try
{
@@ -221,50 +211,41 @@ public class SSTableRewriter
}
catch (Throwable t)
{
- fail = merge(fail, t);
+ accumulate = merge(accumulate , t);
}
}
+ // abort the writers, and add the early opened readers to our discard pile
+
if (writer != null)
- finishedEarly.add(new Finished(writer, currentlyOpenedEarly));
+ finishedWriters.add(new Finished(writer, currentlyOpenedEarly));
- // abort the writers
- for (Finished finished : finishedEarly)
+ for (Finished finished : finishedWriters)
{
- try
- {
- finished.writer.abort();
- }
- catch (Throwable t)
- {
- fail = merge(fail, t);
- }
- try
- {
- if (finished.reader != null)
- {
- // if we've already been opened, add ourselves to the discard pile
- discard.add(finished.reader);
- finished.reader.markObsolete();
- }
- }
- catch (Throwable t)
- {
- fail = merge(fail, t);
- }
- }
+ accumulate = finished.writer.abort(accumulate);
- try
- {
- replaceWithFinishedReaders(Collections.<SSTableReader>emptyList());
- }
- catch (Throwable t)
- {
- fail = merge(fail, t);
+ // if we've already been opened, add ourselves to the discard pile
+ if (finished.reader != null)
+ discard.add(finished.reader);
}
- if (fail != null)
- throw Throwables.propagate(fail);
+ accumulate = replaceWithFinishedReaders(Collections.<SSTableReader>emptyList(), accumulate);
+ return accumulate;
+ }
+
+ protected Throwable doCommit(Throwable accumulate)
+ {
+ for (Finished f : finishedWriters)
+ accumulate = f.writer.commit(accumulate);
+ accumulate = replaceWithFinishedReaders(preparedForCommit, accumulate);
+
+ return accumulate;
+ }
+
+ protected Throwable doCleanup(Throwable accumulate)
+ {
+ // we have no state of our own to cleanup; Transactional objects cleanup their own state in abort or commit
+ return accumulate;
}
/**
@@ -369,41 +350,38 @@ public class SSTableRewriter
public void switchWriter(SSTableWriter newWriter)
{
- if (writer == null)
+ if (writer == null || writer.getFilePointer() == 0)
{
+ if (writer != null)
+ writer.abort();
writer = newWriter;
return;
}
- if (writer.getFilePointer() != 0)
- {
- // If early re-open is disabled, simply finalize the writer and store it
- if (preemptiveOpenInterval == Long.MAX_VALUE)
- {
- SSTableReader reader = writer.finish(SSTableWriter.FinishType.NORMAL, maxAge, -1);
- finishedReaders.add(reader);
- }
- else
- {
- // we leave it as a tmp file, but we open it and add it to the dataTracker
- SSTableReader reader = writer.finish(SSTableWriter.FinishType.EARLY, maxAge, -1);
- replaceEarlyOpenedFile(currentlyOpenedEarly, reader);
- moveStarts(reader, reader.last, false);
- finishedEarly.add(new Finished(writer, reader));
- }
- }
- else
+ SSTableReader reader = null;
+ if (preemptiveOpenInterval != Long.MAX_VALUE)
{
- writer.abort();
+ // we leave it as a tmp file, but we open it and add it to the dataTracker
+ reader = writer.setMaxDataAge(maxAge).openFinalEarly();
+ replaceEarlyOpenedFile(currentlyOpenedEarly, reader);
+ moveStarts(reader, reader.last, false);
}
+ finishedWriters.add(new Finished(writer, reader));
+
currentlyOpenedEarly = null;
currentlyOpenedEarlyAt = 0;
writer = newWriter;
}
- public List<SSTableReader> finish()
+ /**
+ * @param repairedAt the repair time, -1 if we should use the time we supplied when we created
+ * the SSTableWriter (and called rewriter.switchWriter(..)), actual time if we want to override the
+ * repair time.
+ */
+ public SSTableRewriter setRepairedAt(long repairedAt)
{
- return finish(-1);
+ this.repairedAt = repairedAt;
+ return this;
}
/**
@@ -417,94 +395,92 @@ public class SSTableRewriter
* gymnastics (ie, call DataTracker#markCompactedSSTablesReplaced(..))
*
*
- * @param repairedAt the repair time, -1 if we should use the time we supplied when we created
- * the SSTableWriter (and called rewriter.switchWriter(..)), actual time if we want to override the
- * repair time.
*/
- public List<SSTableReader> finish(long repairedAt)
+ public List<SSTableReader> finish()
{
- return finishAndMaybeThrow(repairedAt, false, false);
+ super.finish();
+ return finished();
}
- @VisibleForTesting
- void finishAndThrow(boolean throwEarly)
+ public List<SSTableReader> finished()
{
- finishAndMaybeThrow(-1, throwEarly, !throwEarly);
+ assert state() == State.COMMITTED || state() == State.READY_TO_COMMIT;
+ return preparedForCommit;
}
- private List<SSTableReader> finishAndMaybeThrow(long repairedAt, boolean throwEarly, boolean throwLate)
+ protected void doPrepare()
{
- switch (state)
- {
- case FINISHED: case ABORTED:
- throw new IllegalStateException("Cannot finish - changes have already been " + state.toString().toLowerCase());
- }
-
- List<SSTableReader> newReaders = new ArrayList<>();
switchWriter(null);
if (throwEarly)
throw new RuntimeException("exception thrown early in finish, for testing");
// No early open to finalize and replace
- if (preemptiveOpenInterval == Long.MAX_VALUE)
- {
- replaceWithFinishedReaders(finishedReaders);
- if (throwLate)
- throw new RuntimeException("exception thrown after all sstables finished, for testing");
- return finishedReaders;
- }
-
- while (!finishedEarly.isEmpty())
+ for (Finished f : finishedWriters)
{
- Finished f = finishedEarly.peek();
- if (f.writer.getFilePointer() > 0)
- {
- if (f.reader != null)
- discard.add(f.reader);
+ if (f.reader != null)
+ discard.add(f.reader);
- SSTableReader newReader = f.writer.finish(SSTableWriter.FinishType.FINISH_EARLY, maxAge, repairedAt);
+ f.writer.setRepairedAt(repairedAt).setMaxDataAge(maxAge).setOpenResult(true).prepareToCommit();
+ SSTableReader newReader = f.writer.finished();
- if (f.reader != null)
- f.reader.setReplacedBy(newReader);
+ if (f.reader != null)
+ f.reader.setReplacedBy(newReader);
- finished.add(newReader);
- newReaders.add(newReader);
- }
- else
- {
- f.writer.abort();
- assert f.reader == null;
- }
- finishedEarly.poll();
+ preparedForCommit.add(newReader);
}
if (throwLate)
throw new RuntimeException("exception thrown after all sstables finished, for testing");
+ }
- replaceWithFinishedReaders(newReaders);
- state = State.FINISHED;
- return finished;
+ @VisibleForTesting
+ void throwDuringPrepare(boolean throwEarly)
+ {
+ this.throwEarly = throwEarly;
+ this.throwLate = !throwEarly;
}
// cleanup all our temporary readers and swap in our new ones
- private void replaceWithFinishedReaders(List<SSTableReader> finished)
+ private Throwable replaceWithFinishedReaders(List<SSTableReader> finished, Throwable accumulate)
{
if (isOffline)
{
for (SSTableReader reader : discard)
{
- if (reader.getCurrentReplacement() == reader)
- reader.markObsolete();
- reader.selfRef().release();
+ try
+ {
+ if (reader.getCurrentReplacement() == reader)
+ reader.markObsolete();
+ }
+ catch (Throwable t)
+ {
+ accumulate = merge(accumulate, t);
+ }
}
+ accumulate = Refs.release(Refs.selfRefs(discard), accumulate);
}
else
{
- dataTracker.replaceEarlyOpenedFiles(discard, finished);
- dataTracker.unmarkCompacting(discard);
+ try
+ {
+ dataTracker.replaceEarlyOpenedFiles(discard, finished);
+ }
+ catch (Throwable t)
+ {
+ accumulate = merge(accumulate, t);
+ }
+ try
+ {
+ dataTracker.unmarkCompacting(discard);
+ }
+ catch (Throwable t)
+ {
+ accumulate = merge(accumulate, t);
+ }
}
discard.clear();
+ return accumulate;
}
private static final class Finished
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 5998044..d6ab940 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
@@ -157,6 +157,7 @@ public class SSTableSimpleUnsortedWriter extends AbstractSSTableSimpleWriter
{
throw new RuntimeException(e);
}
+ checkForWriterException();
}
// This is overridden by CQLSSTableWriter to hold off replacing column family until the next iteration through
@@ -215,39 +216,40 @@ public class SSTableSimpleUnsortedWriter extends AbstractSSTableSimpleWriter
public void run()
{
- SSTableWriter writer = null;
-
- while (true)
{
- try
+ while (true)
{
- Buffer b = writeQueue.take();
- if (b == SENTINEL)
- return;
-
- writer = getWriter();
- boolean first = true;
- for (Map.Entry<DecoratedKey, ColumnFamily> entry : b.entrySet())
+ try
{
- if (entry.getValue().getColumnCount() > 0)
- writer.append(entry.getKey(), entry.getValue());
- else if (!first)
- throw new AssertionError("Empty partition");
- first = false;
+ Buffer b = writeQueue.take();
+ if (b == SENTINEL)
+ return;
+
+ try (SSTableWriter writer = getWriter();)
+ {
+ boolean first = true;
+ for (Map.Entry<DecoratedKey, ColumnFamily> entry : b.entrySet())
+ {
+ if (entry.getValue().getColumnCount() > 0)
+ writer.append(entry.getKey(), entry.getValue());
+ else if (!first)
+ throw new AssertionError("Empty partition");
+ first = false;
+ }
+
+ writer.finish(false);
+ }
+ }
+ catch (Throwable e)
+ {
+ JVMStabilityInspector.inspectThrowable(e);
+ // Keep only the first exception
+ if (exception == null)
+ exception = e;
}
- writer.close();
- }
- catch (Throwable e)
- {
- JVMStabilityInspector.inspectThrowable(e);
- if (writer != null)
- writer.abort();
- // Keep only the first exception
- if (exception == null)
- exception = e;
}
- }
+ }
}
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 3417d68..f206969 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
@@ -19,6 +19,8 @@ package org.apache.cassandra.io.sstable;
import java.io.File;
+import com.google.common.base.Throwables;
+
import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.db.*;
import org.apache.cassandra.db.marshal.AbstractType;
@@ -72,12 +74,11 @@ public class SSTableSimpleWriter extends AbstractSSTableSimpleWriter
{
if (currentKey != null)
writeRow(currentKey, columnFamily);
- writer.close();
+ writer.finish(false);
}
- catch (FSError e)
+ catch (Throwable t)
{
- writer.abort();
- throw e;
+ throw Throwables.propagate(writer.abort(t));
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 4411ca7..23c27b0 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -864,12 +864,21 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
* @param ibuilder
* @param dbuilder
*/
+
public void saveSummary(SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder)
{
- saveSummary(ibuilder, dbuilder, indexSummary);
+ saveSummary(this.descriptor, this.first, this.last, ibuilder, dbuilder, indexSummary);
}
- private void saveSummary(SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder, IndexSummary summary)
+ private void saveSummary(SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder, IndexSummary newSummary)
+ {
+ saveSummary(this.descriptor, this.first, this.last, ibuilder, dbuilder, newSummary);
+ }
+ /**
+ * Save index summary to Summary.db file.
+ */
+ public static void saveSummary(Descriptor descriptor, DecoratedKey first, DecoratedKey last,
+ SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder, IndexSummary summary)
{
File summariesFile = new File(descriptor.filenameFor(Component.SUMMARY));
if (summariesFile.exists())
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8704006b/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 baacb5a..f99292e 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
@@ -31,16 +31,17 @@ 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.metadata.MetadataCollector;
+import org.apache.cassandra.io.sstable.metadata.MetadataComponent;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.utils.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.cassandra.utils.concurrent.Transactional;
import java.io.DataInput;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashSet;
+import java.util.Map;
import java.util.Set;
/**
@@ -49,30 +50,24 @@ import java.util.Set;
* TableWriter.create() is the primary way to create a writer for a particular format.
* The format information is part of the Descriptor.
*/
-public abstract class SSTableWriter extends SSTable
+public abstract class SSTableWriter extends SSTable implements Transactional
{
- private static final Logger logger = LoggerFactory.getLogger(SSTableWriter.class);
-
- public static enum FinishType
- {
- CLOSE(null, true),
- NORMAL(SSTableReader.OpenReason.NORMAL, true),
- EARLY(SSTableReader.OpenReason.EARLY, false), // no renaming
- FINISH_EARLY(SSTableReader.OpenReason.NORMAL, true); // tidy up an EARLY finish
- public final SSTableReader.OpenReason openReason;
-
- public final boolean isFinal;
- FinishType(SSTableReader.OpenReason openReason, boolean isFinal)
- {
- this.openReason = openReason;
- this.isFinal = isFinal;
- }
- }
-
- protected final long repairedAt;
+ protected long repairedAt;
+ protected long maxDataAge = -1;
protected final long keyCount;
protected final MetadataCollector metadataCollector;
protected final RowIndexEntry.IndexSerializer rowIndexEntrySerializer;
+ protected final TransactionalProxy txnProxy = txnProxy();
+
+ protected abstract TransactionalProxy txnProxy();
+
+ // due to lack of multiple inheritance, we use an inner class to proxy our Transactional implementation details
+ protected abstract class TransactionalProxy extends AbstractTransactional
+ {
+ // should be set during doPrepare()
+ protected SSTableReader finalReader;
+ protected boolean openResult;
+ }
protected SSTableWriter(Descriptor descriptor, long keyCount, long repairedAt, CFMetaData metadata, IPartitioner partitioner, MetadataCollector metadataCollector)
{
@@ -164,28 +159,98 @@ public abstract class SSTableWriter extends SSTable
public abstract long getOnDiskFilePointer();
- public abstract void isolateReferences();
-
public abstract void resetAndTruncate();
- public SSTableReader closeAndOpenReader()
+ public SSTableWriter setRepairedAt(long repairedAt)
+ {
+ if (repairedAt > 0)
+ this.repairedAt = repairedAt;
+ return this;
+ }
+
+ public SSTableWriter setMaxDataAge(long maxDataAge)
+ {
+ this.maxDataAge = maxDataAge;
+ return this;
+ }
+
+ public SSTableWriter setOpenResult(boolean openResult)
+ {
+ txnProxy.openResult = openResult;
+ return this;
+ }
+
+ /**
+ * Open the resultant SSTableReader before it has been fully written
+ */
+ public abstract SSTableReader openEarly();
+
+ /**
+ * Open the resultant SSTableReader once it has been fully written, but before the
+ * _set_ of tables that are being written together as one atomic operation are all ready
+ */
+ public abstract SSTableReader openFinalEarly();
+
+ public SSTableReader finish(long repairedAt, long maxDataAge, boolean openResult)
{
- return closeAndOpenReader(System.currentTimeMillis());
+ if (repairedAt > 0)
+ this.repairedAt = repairedAt;
+ this.maxDataAge = maxDataAge;
+ return finish(openResult);
}
- public SSTableReader closeAndOpenReader(long maxDataAge)
+ public SSTableReader finish(boolean openResult)
{
- return finish(FinishType.NORMAL, maxDataAge, repairedAt);
+ txnProxy.openResult = openResult;
+ txnProxy.finish();
+ return finished();
}
- public abstract SSTableReader finish(FinishType finishType, long maxDataAge, long repairedAt);
+ /**
+ * Open the resultant SSTableReader once it has been fully written, and all related state
+ * is ready to be finalised including other sstables being written involved in the same operation
+ */
+ public SSTableReader finished()
+ {
+ return txnProxy.finalReader;
+ }
+
+ // finalise our state on disk, including renaming
+ public final void prepareToCommit()
+ {
+ txnProxy.prepareToCommit();
+ }
+
+ public final Throwable commit(Throwable accumulate)
+ {
+ return txnProxy.commit(accumulate);
+ }
+
+ public final Throwable abort(Throwable accumulate)
+ {
+ return txnProxy.abort(accumulate);
+ }
- public abstract SSTableReader openEarly(long maxDataAge);
+ public final void close()
+ {
+ txnProxy.close();
+ }
- // Close the writer and return the descriptor to the new sstable and it's metadata
- public abstract Pair<Descriptor, StatsMetadata> close();
+ public final void abort()
+ {
+ txnProxy.abort();
+ }
+ protected Map<MetadataType, MetadataComponent> finalizeMetadata()
+ {
+ return metadataCollector.finalizeMetadata(partitioner.getClass().getCanonicalName(),
+ metadata.getBloomFilterFpChance(), repairedAt);
+ }
+ protected StatsMetadata statsMetadata()
+ {
+ return (StatsMetadata) finalizeMetadata().get(MetadataType.STATS);
+ }
public static Descriptor rename(Descriptor tmpdesc, Set<Component> components)
{
@@ -209,12 +274,6 @@ public abstract class SSTableWriter extends SSTable
}
- /**
- * After failure, attempt to close the index writer and data file before deleting all temp components for the sstable
- */
- public abstract void abort();
-
-
public static abstract class Factory
{
public abstract SSTableWriter open(Descriptor descriptor, long keyCount, long repairedAt, CFMetaData metadata, IPartitioner partitioner, MetadataCollector metadataCollector);