Skip to content

Commit

Permalink
Avoids unmarking compacting sstable prematurely during cleanup
Browse files Browse the repository at this point in the history
patch by slebresne; reviewed by jbellis for CASSANDRA-2769


git-svn-id: https://svn.apache.org/repos/asf/cassandra/branches/cassandra-0.8@1136135 13f79535-47bb-0310-9956-ffa450edef68
  • Loading branch information
pcmanus committed Jun 15, 2011
1 parent 948a238 commit 27f189c
Show file tree
Hide file tree
Showing 2 changed files with 32 additions and 37 deletions.
1 change: 1 addition & 0 deletions CHANGES.txt
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,7 @@
(CASSANDRA-2767)
* use threadsafe collections for StreamInSession (CASSANDRA-2766)
* avoid infinite loop when creating merkle tree (CASSANDRA-2758)
* avoids unmarking compacting sstable prematurely in cleanup (CASSANDRA-2769)


0.8.0-final
Expand Down
68 changes: 31 additions & 37 deletions src/java/org/apache/cassandra/db/compaction/CompactionManager.java
Original file line number Diff line number Diff line change
Expand Up @@ -845,56 +845,50 @@ private void doCleanupCompaction(ColumnFamilyStore cfs, Collection<SSTableReader
logger.debug("Expected bloom filter size : " + expectedBloomFilterSize);

SSTableWriter writer = null;

logger.info("Cleaning up " + sstable);
// Calculate the expected compacted filesize
long expectedRangeFileSize = cfs.getExpectedCompactedFileSize(Arrays.asList(sstable)) / 2;
String compactionFileLocation = table.getDataFileLocation(expectedRangeFileSize);
if (compactionFileLocation == null)
throw new IOException("disk full");

SSTableScanner scanner = sstable.getDirectScanner(CompactionIterator.FILE_BUFFER_SIZE);
SortedSet<ByteBuffer> indexedColumns = cfs.getIndexedColumns();
CleanupInfo ci = new CleanupInfo(sstable, scanner);
executor.beginCompaction(ci);
try
{
logger.info("Cleaning up " + sstable);
// Calculate the expected compacted filesize
long expectedRangeFileSize = cfs.getExpectedCompactedFileSize(Arrays.asList(sstable)) / 2;
String compactionFileLocation = table.getDataFileLocation(expectedRangeFileSize);
if (compactionFileLocation == null)
throw new IOException("disk full");

SSTableScanner scanner = sstable.getDirectScanner(CompactionIterator.FILE_BUFFER_SIZE);
SortedSet<ByteBuffer> indexedColumns = cfs.getIndexedColumns();
CleanupInfo ci = new CleanupInfo(sstable, scanner);
executor.beginCompaction(ci);
try
while (scanner.hasNext())
{
while (scanner.hasNext())
SSTableIdentityIterator row = (SSTableIdentityIterator) scanner.next();
if (Range.isTokenInRanges(row.getKey().token, ranges))
{
SSTableIdentityIterator row = (SSTableIdentityIterator) scanner.next();
if (Range.isTokenInRanges(row.getKey().token, ranges))
{
writer = maybeCreateWriter(cfs, compactionFileLocation, expectedBloomFilterSize, writer, Collections.singletonList(sstable));
writer.append(controller.getCompactedRow(row));
totalkeysWritten++;
}
else
writer = maybeCreateWriter(cfs, compactionFileLocation, expectedBloomFilterSize, writer, Collections.singletonList(sstable));
writer.append(controller.getCompactedRow(row));
totalkeysWritten++;
}
else
{
cfs.invalidateCachedRow(row.getKey());
if (!indexedColumns.isEmpty() || isCommutative)
{
cfs.invalidateCachedRow(row.getKey());
if (!indexedColumns.isEmpty() || isCommutative)
while (row.hasNext())
{
while (row.hasNext())
{
IColumn column = row.next();
if (column instanceof CounterColumn)
renewer.maybeRenew((CounterColumn) column);
if (indexedColumns.contains(column.name()))
Table.cleanupIndexEntry(cfs, row.getKey().key, column);
}
IColumn column = row.next();
if (column instanceof CounterColumn)
renewer.maybeRenew((CounterColumn) column);
if (indexedColumns.contains(column.name()))
Table.cleanupIndexEntry(cfs, row.getKey().key, column);
}
}
}
}
finally
{
scanner.close();
executor.finishCompaction(ci);
}
}
finally
{
cfs.getDataTracker().unmarkCompacting(Arrays.asList(sstable));
scanner.close();
executor.finishCompaction(ci);
}

List<SSTableReader> results = new ArrayList<SSTableReader>();
Expand Down

0 comments on commit 27f189c

Please sign in to comment.