Repository: cassandra Updated Branches: refs/heads/trunk 0841353e9 -> f100024eb
fix handleRepairStatusChangedNotification to remove first then add Patch by ZhaoYang reviewed by marcuse for CASSANDRA-14720 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/f100024e Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/f100024e Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/f100024e Branch: refs/heads/trunk Commit: f100024eb3becf53042823ce1008d3d5ec4e5f86 Parents: 0841353 Author: Zhao Yang <[email protected]> Authored: Wed Sep 12 11:32:46 2018 +0800 Committer: Marcus Eriksson <[email protected]> Committed: Wed Sep 12 08:32:51 2018 +0200 ---------------------------------------------------------------------- CHANGES.txt | 1 + .../compaction/CompactionStrategyManager.java | 5 +- .../LongLeveledCompactionStrategyTest.java | 99 ++++++++++++++------ 3 files changed, 73 insertions(+), 32 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/f100024e/CHANGES.txt ---------------------------------------------------------------------- diff --git a/CHANGES.txt b/CHANGES.txt index ef285e0..3cfdcff 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0 + * fix handleRepairStatusChangedNotification to remove first then add (CASSANDRA-14720) * Allow transient node to serve as a repair coordinator (CASSANDRA-14693) * DecayingEstimatedHistogramReservoir.EstimatedHistogramReservoirSnapshot returns wrong value for size() and incorrectly calculates count (CASSANDRA-14696) * AbstractReplicaCollection equals and hash code should throw due to conflict between order sensitive/insensitive uses (CASSANDRA-14700) http://git-wip-us.apache.org/repos/asf/cassandra/blob/f100024e/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java index afe628b..45ccbe2 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java @@ -735,13 +735,16 @@ public class CompactionStrategyManager implements INotificationConsumer continue; AbstractStrategyHolder dstHolder = holders.get(i); - dstHolder.addSSTables(group); for (AbstractStrategyHolder holder : holders) { if (holder != dstHolder) holder.removeSSTables(group); } + + // adding sstables into another strategy may change its level, + // thus it won't be removed from original LCS. We have to remove sstables first + dstHolder.addSSTables(group); } } finally http://git-wip-us.apache.org/repos/asf/cassandra/blob/f100024e/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java ---------------------------------------------------------------------- diff --git a/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java b/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java index 56eef17..3bcd9d1 100644 --- a/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java +++ b/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java @@ -20,6 +20,7 @@ package org.apache.cassandra.db.compaction; import java.nio.ByteBuffer; import java.util.*; import java.util.concurrent.*; +import java.util.stream.Collectors; import com.google.common.collect.Lists; @@ -38,8 +39,11 @@ import org.apache.cassandra.db.*; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.schema.CompactionParams; import org.apache.cassandra.schema.KeyspaceParams; +import org.apache.cassandra.service.ActiveRepairService; +import org.apache.cassandra.service.StorageService; import org.apache.cassandra.utils.FBUtilities; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; public class LongLeveledCompactionStrategyTest @@ -75,22 +79,7 @@ public class LongLeveledCompactionStrategyTest ByteBuffer value = ByteBuffer.wrap(new byte[100 * 1024]); // 100 KB value, make it easy to have multiple files - // Enough data to have a level 1 and 2 - int rows = 128; - int columns = 10; - - // Adds enough data to trigger multiple sstable per level - for (int r = 0; r < rows; r++) - { - DecoratedKey key = Util.dk(String.valueOf(r)); - UpdateBuilder builder = UpdateBuilder.create(store.metadata(), key); - for (int c = 0; c < columns; c++) - builder.newRow("column" + c).add("val", value); - - Mutation rm = new Mutation(builder.build()); - rm.apply(); - store.forceBlockingFlush(); - } + populateSSTables(store); // Execute LCS in parallel ExecutorService executor = new ThreadPoolExecutor(4, 4, @@ -153,22 +142,8 @@ public class LongLeveledCompactionStrategyTest ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF_STANDARDLVL2); ByteBuffer value = ByteBuffer.wrap(new byte[100 * 1024]); // 100 KB value, make it easy to have multiple files - // Enough data to have a level 1 and 2 - int rows = 128; - int columns = 10; - - // Adds enough data to trigger multiple sstable per level - for (int r = 0; r < rows; r++) - { - DecoratedKey key = Util.dk(String.valueOf(r)); - UpdateBuilder builder = UpdateBuilder.create(store.metadata(), key); - for (int c = 0; c < columns; c++) - builder.newRow("column" + c).add("val", value); + populateSSTables(store); - Mutation rm = new Mutation(builder.build()); - rm.apply(); - store.forceBlockingFlush(); - } LeveledCompactionStrategyTest.waitForLeveling(store); store.disableAutoCompaction(); CompactionStrategyManager mgr = store.getCompactionStrategyManager(); @@ -229,4 +204,66 @@ public class LongLeveledCompactionStrategyTest } + + @Test + public void testRepairStatusChanges() throws Exception + { + String ksname = KEYSPACE1; + String cfname = "StandardLeveled"; + Keyspace keyspace = Keyspace.open(ksname); + ColumnFamilyStore store = keyspace.getColumnFamilyStore(cfname); + store.disableAutoCompaction(); + + CompactionStrategyManager mgr = store.getCompactionStrategyManager(); + LeveledCompactionStrategy repaired = (LeveledCompactionStrategy) mgr.getStrategies().get(0).get(0); + LeveledCompactionStrategy unrepaired = (LeveledCompactionStrategy) mgr.getStrategies().get(1).get(0); + + // populate repaired sstables + populateSSTables(store); + assertTrue(repaired.getSSTables().isEmpty()); + assertFalse(unrepaired.getSSTables().isEmpty()); + mgr.mutateRepaired(store.getLiveSSTables(), FBUtilities.nowInSeconds(), null, false); + assertFalse(repaired.getSSTables().isEmpty()); + assertTrue(unrepaired.getSSTables().isEmpty()); + + // populate unrepaired sstables + populateSSTables(store); + assertFalse(repaired.getSSTables().isEmpty()); + assertFalse(unrepaired.getSSTables().isEmpty()); + + // compact them into upper levels + store.forceMajorCompaction(); + assertFalse(repaired.getSSTables().isEmpty()); + assertFalse(unrepaired.getSSTables().isEmpty()); + + // mark unrepair + mgr.mutateRepaired(store.getLiveSSTables().stream().filter(s -> s.isRepaired()).collect(Collectors.toList()), + ActiveRepairService.UNREPAIRED_SSTABLE, + null, + false); + assertTrue(repaired.getSSTables().isEmpty()); + assertFalse(unrepaired.getSSTables().isEmpty()); + } + + private void populateSSTables(ColumnFamilyStore store) + { + ByteBuffer value = ByteBuffer.wrap(new byte[100 * 1024]); // 100 KB value, make it easy to have multiple files + + // Enough data to have a level 1 and 2 + int rows = 128; + int columns = 10; + + // Adds enough data to trigger multiple sstable per level + for (int r = 0; r < rows; r++) + { + DecoratedKey key = Util.dk(String.valueOf(r)); + UpdateBuilder builder = UpdateBuilder.create(store.metadata(), key); + for (int c = 0; c < columns; c++) + builder.newRow("column" + c).add("val", value); + + Mutation rm = new Mutation(builder.build()); + rm.apply(); + store.forceBlockingFlush(); + } + } } --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
