jmckenzie-dev commented on code in PR #3996:
URL: https://github.com/apache/cassandra/pull/3996#discussion_r2010742397


##########
test/unit/org/apache/cassandra/db/commitlog/CommitlogShutdownTest.java:
##########
@@ -85,15 +87,23 @@ public void testShutdownWithPendingTasks() throws Exception
         // force creating several commitlog files
         for (int i = 0; i < 10; i++)
         {
-            CommitLog.instance.add(m);
+            m.apply();
         }
 
-        // schedule discarding completed segments and immediately issue a 
shutdown
-        TableId tableId = m.getTableIds().iterator().next();
-        CommitLog.instance.discardCompletedSegments(tableId, 
CommitLogPosition.NONE, CommitLog.instance.getCurrentPosition());
-        CommitLog.instance.shutdownBlocking();
+        StorageService.instance.drain();
 
-        // the shutdown should block until all logs except the currently 
active one and perhaps a new, empty one are gone
-        Assert.assertTrue(new 
File(DatabaseDescriptor.getCommitLogLocation()).listFiles().length <= 2);
+        List<CommitLogSegment> segmentsToCheck = new 
ArrayList<>(CommitLog.instance.segmentManager.getActiveSegments());
+        // remove the last, potentially active segment from the check
+        if (!segmentsToCheck.isEmpty())
+            segmentsToCheck.remove(segmentsToCheck.size() - 1);

Review Comment:
   So to test my understanding, we're saying that at worst we'll allow 
accumulation of 1 inactive CL Segment on node shutdown here. If a node were 
cycling up and down and in the pathological worst-case of having a single 
segment left alive, this would no longer accumulate (the problem in 
CASSANDRA-13123) so we would then instead only have the 1 CLSegment to contend 
with, replay on restart, and keep on moving.



##########
test/unit/org/apache/cassandra/db/commitlog/CommitlogShutdownTest.java:
##########
@@ -85,15 +87,23 @@ public void testShutdownWithPendingTasks() throws Exception
         // force creating several commitlog files
         for (int i = 0; i < 10; i++)
         {
-            CommitLog.instance.add(m);
+            m.apply();
         }
 
-        // schedule discarding completed segments and immediately issue a 
shutdown
-        TableId tableId = m.getTableIds().iterator().next();
-        CommitLog.instance.discardCompletedSegments(tableId, 
CommitLogPosition.NONE, CommitLog.instance.getCurrentPosition());
-        CommitLog.instance.shutdownBlocking();
+        StorageService.instance.drain();

Review Comment:
   This moves us further away from testing the CommitLog's shutdown in 
isolation as a unit test (unit being the "module" / package in this case, 
but... guess we work with what we have?) to more of an integration or smoke 
test to ensure that the `StorageService.instance.drain()` call works as 
expected.
   
   All that said - the original point of 
[CASSANDRA-13123](https://issues.apache.org/jira/browse/CASSANDRA-13123) looks 
like it was about making sure all segments got removed on a `drain` call, so in 
reality I think you're moving this test _towards_ a better fit for purpose for 
the original goal.
   
   I'd love to see a world where we unit tested the components of the 
`CommitLog`, including segment allocation, in isolation so we had confidence in 
them. All that said, we're not there yet and a flaky mis-targeted test isn't 
helping anyone.
   
   



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to