blambov commented on code in PR #4191:
URL: https://github.com/apache/cassandra/pull/4191#discussion_r2123492545


##########
test/unit/org/apache/cassandra/db/compaction/CompactionTaskTest.java:
##########
@@ -136,6 +150,91 @@ public void compactionInterruption() throws Exception
         Assert.assertEquals(Transactional.AbstractTransactional.State.ABORTED, 
txn.state());
     }
 
+    /**
+     * Test that even some SSTables are fully expired, we can still select and 
reference them
+     * while they are part of compaction.
+     *
+     * @see <a 
href="https://issues.apache.org/jira/browse/CASSANDRA-19776>CASSANDRA-19776</a>
+     */
+    @Test
+    public void testFullyExpiredSSTablesAreNotReleasedPrematurely()
+    {
+        Assert.assertEquals(0, gcGraceCfs.getLiveSSTables().size());
+        gcGraceCfs.getCompactionStrategyManager().disable();
+
+        // similar technique to get fully expired SSTables as in 
TTLExpiryTest#testAggressiveFullyExpired
+        QueryProcessor.executeInternal("INSERT INTO ks.tbl2 (k, col1) VALUES 
(1, 1) USING TIMESTAMP 1 AND TTL 1;");
+        QueryProcessor.executeInternal("INSERT INTO ks.tbl2 (k, col2) VALUES 
(1, 1) USING TIMESTAMP 3 AND TTL 1;");
+        Util.flush(gcGraceCfs);
+        QueryProcessor.executeInternal("INSERT INTO ks.tbl2 (k, col1) VALUES 
(1, 1) USING TIMESTAMP 2 AND TTL 1;");
+        QueryProcessor.executeInternal("INSERT INTO ks.tbl2 (k, col2) VALUES 
(1, 1) USING TIMESTAMP 5 AND TTL 1;");
+        Util.flush(gcGraceCfs);
+        Set<SSTableReader> toBeObsolete = new 
HashSet<>(gcGraceCfs.getLiveSSTables());
+        Assert.assertEquals(2, toBeObsolete.size());
+        QueryProcessor.executeInternal("INSERT INTO ks.tbl2 (k, col1) VALUES 
(1, 1) USING TIMESTAMP 4 AND TTL 1;");
+        QueryProcessor.executeInternal("INSERT INTO ks.tbl2 (k, col3) VALUES 
(1, 1) USING TIMESTAMP 7 AND TTL 1;");
+        Util.flush(gcGraceCfs);
+        QueryProcessor.executeInternal("INSERT INTO ks.tbl2 (k, col3) VALUES 
(1, 1) USING TIMESTAMP 6 AND TTL 3;");
+        QueryProcessor.executeInternal("INSERT INTO ks.tbl2 (k, col2) VALUES 
(1, 1) USING TIMESTAMP 8 AND TTL 1;");
+        Util.flush(gcGraceCfs);
+
+        Set<SSTableReader> sstables = gcGraceCfs.getLiveSSTables();
+        Assert.assertEquals(4, sstables.size());
+
+        // hook to transaction's checkpoint and commit methods to verify that 
after checkpointing, we can still reference
+        ILifecycleTransaction txn = new 
WrappedLifecycleTransaction(gcGraceCfs.getTracker().tryModify(sstables, 
OperationType.COMPACTION))
+        {
+            @Override
+            public void checkpoint()
+            {
+                for (SSTableReader r : toBeObsolete)
+                    Assert.assertTrue(this.isObsolete(r));
+
+                Assert.assertEquals(4, getNumberOfReferences());
+
+                super.checkpoint();
+
+                Assert.assertEquals(4, getNumberOfReferences());
+            }
+
+            @Override
+            public Throwable commit(Throwable accumulate)
+            {
+                int referencesBeforeCommit = getNumberOfReferences();
+                Throwable commit = super.commit(accumulate);
+                int referencesAfterCommit = getNumberOfReferences();
+
+                Assert.assertTrue(referencesBeforeCommit > 
referencesAfterCommit);
+
+                return commit;
+            }
+
+            private int getNumberOfReferences()
+            {
+                // this will be used e.g. in EstimatedPartitionCount metric 
and similar.
+                // it is crucial it does not loop and returns hence we do not 
use ColumnFamilyStore's selectAndReference
+                // method which does that, we just quickly assert that refs 
are not null (they would be null if
+                // not all SSTables would be referenceable).
+                ColumnFamilyStore.ViewFragment view = 
gcGraceCfs.select(View.selectFunction(SSTableSet.CANONICAL));
+                Refs<SSTableReader> refs = Refs.tryRef(view.sstables);
+                Assert.assertNotNull(refs);
+                int size = refs.size();
+                refs.close();
+
+                return size;
+            }
+        };
+
+        CompactionTask task = new CompactionTask(gcGraceCfs, txn, 
FBUtilities.nowInSeconds() + 2);
+
+        try (CompactionController compactionController = 
task.getCompactionController(task.inputSSTables()))
+        {
+            Set<SSTableReader> fullyExpiredSSTables = 
compactionController.getFullyExpiredSSTables();
+            Assert.assertEquals(2, fullyExpiredSSTables.size());
+            task.execute(null);

Review Comment:
   To make sure we are testing the right thing, could you make sure 
`checkpoint` is being called from the lambda in 
`SSTableRewriter.maybeReopenEarly` at least once? 
   
   Ideally, we'd also like a call from `switchWriter` but I'd accept either as 
sufficient, as long as it is not just the one in `doPrepare`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscr...@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscr...@cassandra.apache.org
For additional commands, e-mail: pr-h...@cassandra.apache.org

Reply via email to