[
https://issues.apache.org/jira/browse/CASSANDRA-19101?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17911475#comment-17911475
]
Stefan Miklosovic edited comment on CASSANDRA-19101 at 1/9/25 1:07 PM:
-----------------------------------------------------------------------
I am not completely sure where it is said that we will always end up with two
commit logs only after the shutdown. Why two?
In the test, there is
{code}
// schedule discarding completed segments and immediately issue a
shutdown
TableId tableId = m.getTableIds().iterator().next();
CommitLog.instance.discardCompletedSegments(tableId,
CommitLogPosition.NONE, CommitLog.instance.getCurrentPosition());
// why do we think that after the last line, there will be 2 commit
logs only?
CommitLog.instance.shutdownBlocking();
{code}
One possible reason it fails is that discardCompletedSegments contains this:
{code}
// Don't mark or try to delete any newer segments once we've
reached the one containing the
// position of the flush.
if (segment.contains(upperBound))
break;
{code}
upperBound is "CommitLog.instance.getCurrentPosition()".
So what might happen is that it will discard only commit log segments up to
current position, but then, after returning from discardCompletedSegments and
before shutdownBlocking is done, there is yet another commit log segment
created.
If we carefully inspect the logs, we see:
{code}
[junit-timeout] DEBUG [main] 2025-01-09 09:48:00,891 CommitLog.java:369 -
Commit log segment
CommitLogSegment(build/test/cassandra/commitlog/CommitLog-9-1736416077190.log)
is unused
[junit-timeout] DEBUG [main] 2025-01-09 09:48:00,891
AbstractCommitLogSegmentManager.java:412 - Segment
CommitLogSegment(build/test/cassandra/commitlog/CommitLog-9-1736416077190.log)
is no longer active and will be deleted now
[junit-timeout] DEBUG [PERIODIC-COMMIT-LOG-SYNCER] 2025-01-09 09:48:00,966
HeapUtils.java:133 - Heap dump creation on uncaught exceptions is disabled.
[junit-timeout] INFO [main] 2025-01-09 09:48:01,065
CommitlogShutdownTest.java:110 - Content of commit log location after shutdown
is : [build/test/cassandra/commitlog/CommitLog-9-1736416077189.log,
build/test/cassandra/commitlog/CommitLog-9-1736416077191.log,
build/test/cassandra/commitlog/CommitLog-9-1736416077192.log]
{code}
So, 7190 is removed. " Commit log segment is unused" message is part of
CommitLog.discardCompletedSegments, but we also see that there are two
additional logs created, 7191 and 7192. On top of already present 7189 -
failing the assert about having two logs only.
was (Author: smiklosovic):
I am not completely sure where it is said that we will always end up with two
commit logs only after the shutdown. Why two?
In the test, there is
{code}
// schedule discarding completed segments and immediately issue a
shutdown
TableId tableId = m.getTableIds().iterator().next();
CommitLog.instance.discardCompletedSegments(tableId,
CommitLogPosition.NONE, CommitLog.instance.getCurrentPosition());
// why do we think that after the last line, there will be 2 commit
logs only?
CommitLog.instance.shutdownBlocking();
{code}
One possible reason is that discardCompletedSegments contains this:
{code}
// Don't mark or try to delete any newer segments once we've
reached the one containing the
// position of the flush.
if (segment.contains(upperBound))
break;
{code}
upperBound is "CommitLog.instance.getCurrentPosition()".
So what might happen is that it will discard only commit log segments up to
current position, but then, after returning from discardCompletedSegments and
before shutdownBlocking is done, there is yet another commit log segment
created.
If we carefully inspect the logs, we see:
{code}
[junit-timeout] DEBUG [main] 2025-01-09 09:48:00,891 CommitLog.java:369 -
Commit log segment
CommitLogSegment(build/test/cassandra/commitlog/CommitLog-9-1736416077190.log)
is unused
[junit-timeout] DEBUG [main] 2025-01-09 09:48:00,891
AbstractCommitLogSegmentManager.java:412 - Segment
CommitLogSegment(build/test/cassandra/commitlog/CommitLog-9-1736416077190.log)
is no longer active and will be deleted now
[junit-timeout] DEBUG [PERIODIC-COMMIT-LOG-SYNCER] 2025-01-09 09:48:00,966
HeapUtils.java:133 - Heap dump creation on uncaught exceptions is disabled.
[junit-timeout] INFO [main] 2025-01-09 09:48:01,065
CommitlogShutdownTest.java:110 - Content of commit log location after shutdown
is : [build/test/cassandra/commitlog/CommitLog-9-1736416077189.log,
build/test/cassandra/commitlog/CommitLog-9-1736416077191.log,
build/test/cassandra/commitlog/CommitLog-9-1736416077192.log]
{code}
So, 7190 is removed. " Commit log segment is unused" message is part of
CommitLog.discardCompletedSegments, but we also see that there are two
additional logs created, 7191 and 7192. On top of already present 7189 -
failing the assert about having two logs only.
> 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: Josh McKenzie
> Priority: Normal
> Fix For: 5.x
>
>
> {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: [email protected]
For additional commands, e-mail: [email protected]