cassandra git commit: Remove ref counting in SSTableScanner, fix CompactionTask ordering
Repository: cassandra Updated Branches: refs/heads/cassandra-2.1 ddca610c9 - bdbb071f4 Remove ref counting in SSTableScanner, fix CompactionTask ordering Patch by jmckenzie; reviewed by belliottsmith as a follow-up for CASSANDRA-8399 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/bdbb071f Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/bdbb071f Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/bdbb071f Branch: refs/heads/cassandra-2.1 Commit: bdbb071f4f87131d6996aac52f2b75a5833d5238 Parents: ddca610 Author: Joshua McKenzie jmcken...@apache.org Authored: Wed Jan 7 13:05:31 2015 -0600 Committer: Joshua McKenzie jmcken...@apache.org Committed: Wed Jan 7 14:05:40 2015 -0600 -- .../cassandra/db/compaction/CompactionTask.java | 82 ++-- .../cassandra/io/sstable/SSTableScanner.java| 8 +- 2 files changed, 45 insertions(+), 45 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/bdbb071f/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 4885bc8..d215b4c 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java @@ -140,7 +140,6 @@ public class CompactionTask extends AbstractCompactionTask try (CompactionController controller = getCompactionController(sstables);) { - SetSSTableReader actuallyCompact = Sets.difference(sstables, controller.getFullyExpiredSSTables()); long estimatedTotalKeys = Math.max(cfs.metadata.getMinIndexInterval(), SSTableReader.getApproximateKeyCount(actuallyCompact)); @@ -149,11 +148,16 @@ public class CompactionTask extends AbstractCompactionTask long expectedSSTableSize = Math.min(getExpectedWriteSize(), strategy.getMaxSSTableBytes()); logger.debug(Expected bloom filter size : {}, keysPerSSTable); +ListSSTableReader newSStables; +AbstractCompactionIterable ci; + +// SSTableScanners need to be closed before markCompactedSSTablesReplaced call as scanners contain references +// to both ifile and dfile and SSTR will throw deletion errors on Windows if it tries to delete before scanner is closed. +// See CASSANDRA-8019 and CASSANDRA-8399 try (AbstractCompactionStrategy.ScannerList scanners = strategy.getScanners(actuallyCompact)) { -AbstractCompactionIterable ci = new CompactionIterable(compactionType, scanners.scanners, controller); +ci = new CompactionIterable(compactionType, scanners.scanners, controller); IteratorAbstractCompactedRow iter = ci.iterator(); -ListSSTableReader newSStables; // we can't preheat until the tracker has been set. This doesn't happen until we tell the cfs to // replace the old entries. Track entries to preheat here until then. long minRepairedAt = getMinRepairedAt(actuallyCompact); @@ -215,44 +219,44 @@ public class CompactionTask extends AbstractCompactionTask if (collector != null) collector.finishCompaction(ci); } +} -CollectionSSTableReader oldSStables = this.sstables; -if (!offline) - cfs.getDataTracker().markCompactedSSTablesReplaced(oldSStables, newSStables, compactionType); - -// log a bunch of statistics about the result and save to system table compaction_history -long dTime = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start); -long startsize = SSTableReader.getTotalBytes(oldSStables); -long endsize = SSTableReader.getTotalBytes(newSStables); -double ratio = (double) endsize / (double) startsize; - -StringBuilder newSSTableNames = new StringBuilder(); -for (SSTableReader reader : newSStables) - newSSTableNames.append(reader.descriptor.baseFilename()).append(,); - -double mbps = dTime 0 ? (double) endsize / (1024 * 1024) / ((double) dTime / 1000) : 0; -long totalSourceRows = 0; -long[] counts = ci.getMergedRowCounts(); -StringBuilder mergeSummary = new StringBuilder(counts.length * 10); -MapInteger, Long mergedRows = new HashMap(); -for (int i = 0; i counts.length; i++) -{ -
[1/2] cassandra git commit: Remove ref counting in SSTableScanner, fix CompactionTask ordering
Repository: cassandra Updated Branches: refs/heads/trunk 493859bf6 - 65ffc39b1 Remove ref counting in SSTableScanner, fix CompactionTask ordering Patch by jmckenzie; reviewed by belliottsmith as a follow-up for CASSANDRA-8399 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/bdbb071f Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/bdbb071f Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/bdbb071f Branch: refs/heads/trunk Commit: bdbb071f4f87131d6996aac52f2b75a5833d5238 Parents: ddca610 Author: Joshua McKenzie jmcken...@apache.org Authored: Wed Jan 7 13:05:31 2015 -0600 Committer: Joshua McKenzie jmcken...@apache.org Committed: Wed Jan 7 14:05:40 2015 -0600 -- .../cassandra/db/compaction/CompactionTask.java | 82 ++-- .../cassandra/io/sstable/SSTableScanner.java| 8 +- 2 files changed, 45 insertions(+), 45 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/bdbb071f/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 4885bc8..d215b4c 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java @@ -140,7 +140,6 @@ public class CompactionTask extends AbstractCompactionTask try (CompactionController controller = getCompactionController(sstables);) { - SetSSTableReader actuallyCompact = Sets.difference(sstables, controller.getFullyExpiredSSTables()); long estimatedTotalKeys = Math.max(cfs.metadata.getMinIndexInterval(), SSTableReader.getApproximateKeyCount(actuallyCompact)); @@ -149,11 +148,16 @@ public class CompactionTask extends AbstractCompactionTask long expectedSSTableSize = Math.min(getExpectedWriteSize(), strategy.getMaxSSTableBytes()); logger.debug(Expected bloom filter size : {}, keysPerSSTable); +ListSSTableReader newSStables; +AbstractCompactionIterable ci; + +// SSTableScanners need to be closed before markCompactedSSTablesReplaced call as scanners contain references +// to both ifile and dfile and SSTR will throw deletion errors on Windows if it tries to delete before scanner is closed. +// See CASSANDRA-8019 and CASSANDRA-8399 try (AbstractCompactionStrategy.ScannerList scanners = strategy.getScanners(actuallyCompact)) { -AbstractCompactionIterable ci = new CompactionIterable(compactionType, scanners.scanners, controller); +ci = new CompactionIterable(compactionType, scanners.scanners, controller); IteratorAbstractCompactedRow iter = ci.iterator(); -ListSSTableReader newSStables; // we can't preheat until the tracker has been set. This doesn't happen until we tell the cfs to // replace the old entries. Track entries to preheat here until then. long minRepairedAt = getMinRepairedAt(actuallyCompact); @@ -215,44 +219,44 @@ public class CompactionTask extends AbstractCompactionTask if (collector != null) collector.finishCompaction(ci); } +} -CollectionSSTableReader oldSStables = this.sstables; -if (!offline) - cfs.getDataTracker().markCompactedSSTablesReplaced(oldSStables, newSStables, compactionType); - -// log a bunch of statistics about the result and save to system table compaction_history -long dTime = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start); -long startsize = SSTableReader.getTotalBytes(oldSStables); -long endsize = SSTableReader.getTotalBytes(newSStables); -double ratio = (double) endsize / (double) startsize; - -StringBuilder newSSTableNames = new StringBuilder(); -for (SSTableReader reader : newSStables) - newSSTableNames.append(reader.descriptor.baseFilename()).append(,); - -double mbps = dTime 0 ? (double) endsize / (1024 * 1024) / ((double) dTime / 1000) : 0; -long totalSourceRows = 0; -long[] counts = ci.getMergedRowCounts(); -StringBuilder mergeSummary = new StringBuilder(counts.length * 10); -MapInteger, Long mergedRows = new HashMap(); -for (int i = 0; i counts.length; i++) -{ -long count =