[ 
https://issues.apache.org/jira/browse/CASSANDRA-19101?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17936954#comment-17936954
 ] 

Dmitry Konstantinov commented on CASSANDRA-19101:
-------------------------------------------------

I have adjusted the test to drain + added a check for unused segments and it is 
failing :)
{code:java}
        final Mutation m = new RowUpdateBuilder(cfs1.metadata.get(), 0, "k")
                           .clustering("bytes")
                           .add("val", ByteBuffer.wrap(entropy))
                           .build();

        // force creating several commitlog files
        for (int i = 0; i < 10; i++)
        {
            m.apply();
        }

        StorageService.instance.drain();

        for (CommitLogSegment segment : 
CommitLog.instance.segmentManager.getActiveSegments())
        {
            Assert.assertFalse("An unused segment is left after drain: " + 
segment.getName()
                               + ", dirty tables: " + segment.dirtyString()
                               + ", total segments: " + 
CommitLog.instance.segmentManager.getActiveSegments().size()
                               + ", commit log files: " + Arrays.toString(new 
File(DatabaseDescriptor.getCommitLogLocation()).listFiles()),
                               segment.isUnused());
        }
    }
}
{code}
{code}
java.lang.AssertionError: An unused segment is left after drain: 
CommitLog-7-1742418190647.log, dirty tables: , total segments: 1, commit log 
files: [build/test/cassandra/commitlog/CommitLog-7-1742418190647.log]

        at org.junit.Assert.fail(Assert.java:88)
        at org.junit.Assert.assertTrue(Assert.java:41)
        at org.junit.Assert.assertFalse(Assert.java:64)
        at 
org.apache.cassandra.db.commitlog.CommitlogShutdownTest.testShutdownWithPendingTasks(CommitlogShutdownTest.java:100)
{code}


We have 1 remaining unused segment which is remaining after drain (in a queue 
and on disk).
drain() invokes 
org.apache.cassandra.db.commitlog.CommitLog#forceRecycleAllSegments()
forceRecycleAllSegments method iterates over all active segments to check if 
they are unused:
{code}
            for (CommitLogSegment segment : activeSegments)
            {
                if (segment.isUnused())
                    archiveAndDiscard(segment);
            }
{code}
but for the last segment: isStillAllocating() = allocatePosition.get() < 
endOfBuffer returns true when segment.isUnused() is invoked in this loop, so it 
is skipped here.
Later during 
org.apache.cassandra.db.commitlog.AbstractCommitLogSegmentManager#awaitTermination
we invoke CommitLogSegment#close for the last remaining segment which as a part 
of discardUnusedTail() logic and it changes allocatePosition, so 
isStillAllocating() now is false. As a result when we check segment.isUnused() 
after drain - it now returns true.

So, it looks like we need to skip at least the last segment in the test check...



> Test Failure: org.apache.cassandra.db.commitlog.CommitlogShutdownTest failed 
> on trunk
> -------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-19101
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-19101
>             Project: Apache Cassandra
>          Issue Type: Bug
>          Components: Test/unit
>            Reporter: Jacek Lewandowski
>            Assignee: Dmitry Konstantinov
>            Priority: Normal
>             Fix For: 4.0.x, 4.1.x, 5.x
>
>         Attachments: Cassandra-trunk_2057_CommitlogShutdownTest.log, 
> diff_discard_logic.png
>
>
> {noformat}
> java.lang.AssertionError
>       at org.junit.Assert.fail(Assert.java:86)
>       at org.junit.Assert.assertTrue(Assert.java:41)
>       at org.junit.Assert.assertTrue(Assert.java:52)
>       at 
> org.apache.cassandra.db.commitlog.CommitlogShutdownTest.testShutdownWithPendingTasks(CommitlogShutdownTest.java:96)
>       at 
> java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>       at 
> java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>       at 
> java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>       at java.base/java.lang.reflect.Method.invoke(Method.java:566)
>       at 
> org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
>       at 
> org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
>       at 
> org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
>       at 
> org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
>       at 
> org.jboss.byteman.contrib.bmunit.BMUnitRunner$10.evaluate(BMUnitRunner.java:393)
>       at 
> org.jboss.byteman.contrib.bmunit.BMUnitRunner$6.evaluate(BMUnitRunner.java:263)
>       at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325)
>       at 
> org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78)
>       at 
> org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57)
>       at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
>       at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
>       at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
>       at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
>       at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
>       at 
> org.jboss.byteman.contrib.bmunit.BMUnitRunner$1.evaluate(BMUnitRunner.java:97)
>       at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
>       at org.junit.runner.JUnitCore.run(JUnitCore.java:137)
>       at 
> com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69)
>       at 
> com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38)
>       at 
> com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11)
>       at 
> com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35)
>       at 
> com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232)
>       at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55)
> {noformat}
> Manual testing to confirm issues found by CircleCI when testing 
> CASSANDRA-18464. Run with Java 11 / IntelliJ



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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

Reply via email to