You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ma...@apache.org on 2016/09/27 07:23:12 UTC
[2/3] cassandra git commit: Make sure repaired tombstones are dropped
when only_purge_repaired_tombstones is enabled
Make sure repaired tombstones are dropped when only_purge_repaired_tombstones is enabled
Patch by Sharvanath Pathak; reviewed by marcuse for CASSANDRA-12703
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/b7da0031
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/b7da0031
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/b7da0031
Branch: refs/heads/trunk
Commit: b7da0031fad4f7ce6493bcbd059c58d61c589a3b
Parents: 405a828
Author: sharvanath <sh...@rubrik.com>
Authored: Mon Sep 26 09:53:07 2016 +0200
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Tue Sep 27 09:09:29 2016 +0200
----------------------------------------------------------------------
.../db/compaction/CompactionIterator.java | 2 +-
.../db/RepairedDataTombstonesTest.java | 80 ++++++++++++--------
2 files changed, 51 insertions(+), 31 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b7da0031/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java b/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java
index d39da2a..f8f620c 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java
@@ -266,7 +266,7 @@ public class CompactionIterator extends CompactionInfo.Holder implements Unfilte
private Purger(boolean isForThrift, CompactionController controller, int nowInSec)
{
- super(isForThrift, nowInSec, controller.gcBefore, controller.compactingRepaired() ? Integer.MIN_VALUE : Integer.MAX_VALUE, controller.cfs.getCompactionStrategyManager().onlyPurgeRepairedTombstones());
+ super(isForThrift, nowInSec, controller.gcBefore, controller.compactingRepaired() ? Integer.MAX_VALUE : Integer.MIN_VALUE, controller.cfs.getCompactionStrategyManager().onlyPurgeRepairedTombstones());
this.controller = controller;
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b7da0031/test/unit/org/apache/cassandra/db/RepairedDataTombstonesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RepairedDataTombstonesTest.java b/test/unit/org/apache/cassandra/db/RepairedDataTombstonesTest.java
index 3a74029..e0d68a4 100644
--- a/test/unit/org/apache/cassandra/db/RepairedDataTombstonesTest.java
+++ b/test/unit/org/apache/cassandra/db/RepairedDataTombstonesTest.java
@@ -60,10 +60,9 @@ public class RepairedDataTombstonesTest extends CQLTester
Thread.sleep(1000);
// at this point we have 2 sstables, one repaired and one unrepaired. Both sstables contain expired tombstones, but we should only drop the tombstones from the repaired sstable.
getCurrentColumnFamilyStore().forceMajorCompaction();
- verify();
- verify2(1);
+ verifyIncludingPurgeable();
+ verify2IncludingPurgeable(1);
assertEquals(2, Iterables.size(getCurrentColumnFamilyStore().getSSTables(SSTableSet.LIVE)));
-
}
@Test
@@ -85,8 +84,8 @@ public class RepairedDataTombstonesTest extends CQLTester
flush();
Thread.sleep(1000);
getCurrentColumnFamilyStore().forceMajorCompaction();
- verify();
- verify2(1);
+ verifyIncludingPurgeable();
+ verify2IncludingPurgeable(1);
assertEquals(1, Iterables.size(getCurrentColumnFamilyStore().getSSTables(SSTableSet.LIVE)));
assertFalse(getCurrentColumnFamilyStore().getSSTables(SSTableSet.LIVE).iterator().next().isRepaired());
@@ -129,8 +128,8 @@ public class RepairedDataTombstonesTest extends CQLTester
// allow gcgrace to properly expire:
Thread.sleep(1000);
- verify();
- verify2(123);
+ verifyIncludingPurgeable();
+ verify2IncludingPurgeable(123);
}
@@ -219,34 +218,44 @@ public class RepairedDataTombstonesTest extends CQLTester
Thread.sleep(2000);
// we will keep all tombstones since the oldest tombstones are unrepaired:
- verify(30, 0, 30);
- verify2(1, 30, 0, 30);
+ verify(30, 0, 30, false);
+ verify2(1, 30, 0, 30, false);
}
private void verify()
{
- verify(10, 10, 20);
+ verify(10, 10, 20, false);
+ }
+
+ private void verifyIncludingPurgeable()
+ {
+ verify(10, 10, 20, true);
}
- private void verify(int expectedRows, int minVal, int maxVal)
+ private void verify(int expectedRows, int minVal, int maxVal, boolean includePurgeable)
{
ReadCommand cmd = Util.cmd(getCurrentColumnFamilyStore()).build();
int foundRows = 0;
- try (ReadOrderGroup orderGroup = cmd.startOrderGroup(); UnfilteredPartitionIterator iterator = cmd.executeLocally(orderGroup))
+ try (ReadOrderGroup orderGroup = cmd.startOrderGroup();
+ UnfilteredPartitionIterator iterator =
+ includePurgeable ? cmd.queryStorage(getCurrentColumnFamilyStore(), orderGroup) :
+ cmd.executeLocally(orderGroup))
{
while (iterator.hasNext())
{
- UnfilteredRowIterator rowIter = iterator.next();
- if (!rowIter.partitionKey().equals(Util.dk(ByteBufferUtil.bytes(999)))) // partition key 999 is 'live' and used to avoid sstables from being dropped
+ try (UnfilteredRowIterator rowIter = iterator.next())
{
- while (rowIter.hasNext())
+ if (!rowIter.partitionKey().equals(Util.dk(ByteBufferUtil.bytes(999)))) // partition key 999 is 'live' and used to avoid sstables from being dropped
{
- AbstractRow row = (AbstractRow) rowIter.next();
- for (int i = 0; i < row.clustering().size(); i++)
+ while (rowIter.hasNext())
{
- foundRows++;
- int val = ByteBufferUtil.toInt(row.clustering().get(i));
- assertTrue("val=" + val, val >= minVal && val < maxVal);
+ AbstractRow row = (AbstractRow) rowIter.next();
+ for (int i = 0; i < row.clustering().size(); i++)
+ {
+ foundRows++;
+ int val = ByteBufferUtil.toInt(row.clustering().get(i));
+ assertTrue("val=" + val, val >= minVal && val < maxVal);
+ }
}
}
}
@@ -254,28 +263,39 @@ public class RepairedDataTombstonesTest extends CQLTester
}
assertEquals(expectedRows, foundRows);
}
+
private void verify2(int key)
{
- verify2(key, 10, 10, 20);
+ verify2(key, 10, 10, 20, false);
+ }
+
+ private void verify2IncludingPurgeable(int key)
+ {
+ verify2(key, 10, 10, 20, true);
}
- private void verify2(int key, int expectedRows, int minVal, int maxVal)
+ private void verify2(int key, int expectedRows, int minVal, int maxVal, boolean includePurgeable)
{
ReadCommand cmd = Util.cmd(getCurrentColumnFamilyStore(), Util.dk(ByteBufferUtil.bytes(key))).build();
int foundRows = 0;
- try (ReadOrderGroup orderGroup = cmd.startOrderGroup(); UnfilteredPartitionIterator iterator = cmd.executeLocally(orderGroup))
+ try (ReadOrderGroup orderGroup = cmd.startOrderGroup();
+ UnfilteredPartitionIterator iterator =
+ includePurgeable ? cmd.queryStorage(getCurrentColumnFamilyStore(), orderGroup) :
+ cmd.executeLocally(orderGroup))
{
while (iterator.hasNext())
{
- UnfilteredRowIterator rowIter = iterator.next();
- while (rowIter.hasNext())
+ try (UnfilteredRowIterator rowIter = iterator.next())
{
- AbstractRow row = (AbstractRow) rowIter.next();
- for (int i = 0; i < row.clustering().size(); i++)
+ while (rowIter.hasNext())
{
- foundRows++;
- int val = ByteBufferUtil.toInt(row.clustering().get(i));
- assertTrue("val=" + val, val >= minVal && val < maxVal);
+ AbstractRow row = (AbstractRow) rowIter.next();
+ for (int i = 0; i < row.clustering().size(); i++)
+ {
+ foundRows++;
+ int val = ByteBufferUtil.toInt(row.clustering().get(i));
+ assertTrue("val=" + val, val >= minVal && val < maxVal);
+ }
}
}
}