This is an automated email from the ASF dual-hosted git repository. maedhroz pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/cassandra.git
commit 93b89b2e79e2fc57acd260ba4588f2448891ef14 Author: Zhao Yang <zhaoyangsingap...@gmail.com> AuthorDate: Tue Jun 13 14:02:34 2023 +0800 send SSTableAddedNotification before MemtableDiscardedNotification to fix index data visilibity issue --- src/java/org/apache/cassandra/db/lifecycle/Tracker.java | 5 +++-- test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java | 12 ++++++------ 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java index 272bbd9305..061765bd52 100644 --- a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java +++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java @@ -411,11 +411,12 @@ public class Tracker Throwable fail; fail = updateSizeTracking(emptySet(), sstables, null); - notifyDiscarded(memtable); - // TODO: if we're invalidated, should we notifyadded AND removed, or just skip both? fail = notifyAdded(sstables, false, memtable, fail); + // make sure index sees flushed index files before dicarding memtable index + notifyDiscarded(memtable); + if (!isDummy() && !cfstore.isValid()) dropSSTables(); diff --git a/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java index d6e3742dcc..7192d50031 100644 --- a/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java +++ b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java @@ -327,9 +327,9 @@ public class TrackerTest tracker.replaceFlushed(prev2, singleton(reader)); Assert.assertEquals(1, tracker.getView().sstables.size()); Assert.assertEquals(2, listener.received.size()); - Assert.assertEquals(prev2, ((MemtableDiscardedNotification) listener.received.get(0)).memtable); - Assert.assertEquals(singleton(reader), ((SSTableAddedNotification) listener.received.get(1)).added); - Assert.assertEquals(Optional.of(prev2), ((SSTableAddedNotification) listener.received.get(1)).memtable()); + Assert.assertEquals(singleton(reader), ((SSTableAddedNotification) listener.received.get(0)).added); + Assert.assertEquals(Optional.of(prev2), ((SSTableAddedNotification) listener.received.get(0)).memtable()); + Assert.assertEquals(prev2, ((MemtableDiscardedNotification) listener.received.get(1)).memtable); listener.received.clear(); if (reader instanceof KeyCacheSupport<?>) Assert.assertTrue(((KeyCacheSupport<?>) reader).getKeyCache().isEnabled()); @@ -350,9 +350,9 @@ public class TrackerTest Assert.assertEquals(0, cfs.metric.liveDiskSpaceUsed.getCount()); Assert.assertEquals(5, listener.received.size()); Assert.assertEquals(prev1, ((MemtableSwitchedNotification) listener.received.get(0)).memtable); - Assert.assertEquals(prev1, ((MemtableDiscardedNotification) listener.received.get(1)).memtable); - Assert.assertEquals(singleton(reader), ((SSTableAddedNotification) listener.received.get(2)).added); - Assert.assertEquals(Optional.of(prev1), ((SSTableAddedNotification) listener.received.get(2)).memtable()); + Assert.assertEquals(singleton(reader), ((SSTableAddedNotification) listener.received.get(1)).added); + Assert.assertEquals(Optional.of(prev1), ((SSTableAddedNotification) listener.received.get(1)).memtable()); + Assert.assertEquals(prev1, ((MemtableDiscardedNotification) listener.received.get(2)).memtable); Assert.assertTrue(listener.received.get(3) instanceof SSTableDeletingNotification); Assert.assertEquals(1, ((SSTableListChangedNotification) listener.received.get(4)).removed.size()); DatabaseDescriptor.setIncrementalBackupsEnabled(backups); --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org