[
https://issues.apache.org/jira/browse/CASSANDRA-18635?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17813524#comment-17813524
]
Berenguer Blasi edited comment on CASSANDRA-18635 at 2/2/24 7:30 AM:
---------------------------------------------------------------------
Posting my findings so far: The exception is
[thrown|https://github.com/apache/cassandra/blob/cassandra-5.0/src/java/org/apache/cassandra/db/compaction/CompactionTask.java#L204]
when the CSM is paused which can happen for number of reasons such as stopping
or forcing a compaction, acquiring sstables, etc. If a compaction is forced,
pausing the CSM, just as a compaction task is already running then it throws.
This is why it's so difficult to repro.
I didn't manage to repro but forcing to throw from the CompactionTask
reproduces the same exact stracktrace, error codes, messages and all the dots
seem to connect nicely. Here we have some log + extra stack traces, where code
lines also match test code lines, where the issued compaction pauses and
resumes the CSM, then a later {{upgradesstables}} does indeed interrupt the
compaction. Given these are async an unlucky run will hit the problem:
{noformat}
DEBUG [node1_isolatedExecutor:1] node1 2024-02-01 10:13:04,909 Cancelling
in-progress compactions for tbl
DEBUG [node1_isolatedExecutor:1] node1 2024-02-01 10:13:04,910 Bereng paused
java.lang.Exception
at
org.apache.cassandra.db.compaction.CompactionStrategyManager.pause(CompactionStrategyManager.java:306)
at
org.apache.cassandra.db.ColumnFamilyStore.pauseCompactionStrategies(ColumnFamilyStore.java:2899)
at
org.apache.cassandra.db.ColumnFamilyStore.runWithCompactionsDisabled(ColumnFamilyStore.java:2846)
at
org.apache.cassandra.db.ColumnFamilyStore.runWithCompactionsDisabled(ColumnFamilyStore.java:2815)
at
org.apache.cassandra.db.compaction.CompactionStrategyManager.getMaximalTasks(CompactionStrategyManager.java:1071)
at
org.apache.cassandra.db.compaction.CompactionManager.submitMaximal(CompactionManager.java:993)
at
org.apache.cassandra.distributed.test.UpgradeSSTablesTest.lambda$upgradeSSTablesInterruptsOngoingCompaction$53e4ed01$1(UpgradeSSTablesTest.java:84)
at
org.apache.cassandra.distributed.impl.IsolatedExecutor.lambda$async$10(IsolatedExecutor.java:156)
at org.apache.cassandra.concurrent.FutureTask$1.call(FutureTask.java:96)
at org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:61)
at org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:71)
at
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
at
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
at java.base/java.lang.Thread.run(Thread.java:829)
DEBUG [node1_isolatedExecutor:1] node1 2024-02-01 10:13:04,913 Bereng resume
java.lang.Exception
at
org.apache.cassandra.db.compaction.CompactionStrategyManager.resume(CompactionStrategyManager.java:287)
at
org.apache.cassandra.db.ColumnFamilyStore.resumeAll(ColumnFamilyStore.java:2917)
at
org.apache.cassandra.db.ColumnFamilyStore.lambda$pauseCompactionStrategies$14(ColumnFamilyStore.java:2902)
at
org.apache.cassandra.db.ColumnFamilyStore.runWithCompactionsDisabled(ColumnFamilyStore.java:2887)
at
org.apache.cassandra.db.ColumnFamilyStore.runWithCompactionsDisabled(ColumnFamilyStore.java:2815)
at
org.apache.cassandra.db.compaction.CompactionStrategyManager.getMaximalTasks(CompactionStrategyManager.java:1071)
at
org.apache.cassandra.db.compaction.CompactionManager.submitMaximal(CompactionManager.java:993)
at
org.apache.cassandra.distributed.test.UpgradeSSTablesTest.lambda$upgradeSSTablesInterruptsOngoingCompaction$53e4ed01$1(UpgradeSSTablesTest.java:84)
at
org.apache.cassandra.distributed.impl.IsolatedExecutor.lambda$async$10(IsolatedExecutor.java:156)
at org.apache.cassandra.concurrent.FutureTask$1.call(FutureTask.java:96)
at org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:61)
at org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:71)
at
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
at
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
at java.base/java.lang.Thread.run(Thread.java:829)
INFO [node1_isolatedExecutor:1] node1 2024-02-01 10:13:04,913 Major compaction
will not result in a single sstable - repaired and unrepaired data is kept
separate and compaction runs per data_file_directory.
DEBUG [node1_CompactionExecutor:1] node1 2024-02-01 10:13:04,915 FileStore /
(/dev/nvme0n1p3) has 257607498752 bytes available, checking if we can write
2639 bytes
INFO [node1_CompactionExecutor:1] node1 2024-02-01 10:13:04,916 Compacting
(1b5a7900-c0e2-11ee-a2d5-d1a8b9818275)
[/home/bereng/work/repos/bdpWS/18635/build/tmp/dtests9959175088994503563/node1/data0/distributed_test_keyspace/tbl-1a850090c0e211eea2d5d1a8b9818275/nb-4-big-Data.db:level=0,
/home/bereng/work/repos/bdpWS/18635/build/tmp/dtests9959175088994503563/node1/data0/distributed_test_keyspace/tbl-1a850090c0e211eea2d5d1a8b9818275/nb-1-big-Data.db:level=0,
/home/bereng/work/repos/bdpWS/18635/build/tmp/dtests9959175088994503563/node1/data0/distributed_test_keyspace/tbl-1a850090c0e211eea2d5d1a8b9818275/nb-13-big-Data.db:level=0,
/home/bereng/work/repos/bdpWS/18635/build/tmp/dtests9959175088994503563/node1/data0/distributed_test_keyspace/tbl-1a850090c0e211eea2d5d1a8b9818275/nb-7-big-Data.db:level=0,
/home/bereng/work/repos/bdpWS/18635/build/tmp/dtests9959175088994503563/node1/data0/distributed_test_keyspace/tbl-1a850090c0e211eea2d5d1a8b9818275/nb-10-big-Data.db:level=0,
]
INFO [node1_isolatedExecutor:2] node1 2024-02-01 10:13:04,928 Starting Upgrade
sstables on distributed_test_keyspace.[tbl]
DEBUG [node1_isolatedExecutor:2] node1 2024-02-01 10:13:04,932 Cancelling
in-progress compactions for tbl
DEBUG [node1_isolatedExecutor:2] node1 2024-02-01 10:13:04,932 Bereng paused
java.lang.Exception
at
org.apache.cassandra.db.compaction.CompactionStrategyManager.pause(CompactionStrategyManager.java:306)
at
org.apache.cassandra.db.ColumnFamilyStore.pauseCompactionStrategies(ColumnFamilyStore.java:2899)
at
org.apache.cassandra.db.ColumnFamilyStore.runWithCompactionsDisabled(ColumnFamilyStore.java:2846)
at
org.apache.cassandra.db.ColumnFamilyStore.runWithCompactionsDisabled(ColumnFamilyStore.java:2815)
at
org.apache.cassandra.db.ColumnFamilyStore.withAllSSTables(ColumnFamilyStore.java:2938)
at
org.apache.cassandra.db.compaction.CompactionManager.parallelAllSSTableOperation(CompactionManager.java:430)
at
org.apache.cassandra.db.compaction.CompactionManager.performSSTableRewrite(CompactionManager.java:584)
at
org.apache.cassandra.db.compaction.CompactionManager.performSSTableRewrite(CompactionManager.java:557)
at
org.apache.cassandra.db.ColumnFamilyStore.sstablesRewrite(ColumnFamilyStore.java:1813)
at
org.apache.cassandra.service.StorageService.rewriteSSTables(StorageService.java:4167)
at
org.apache.cassandra.service.StorageService.upgradeSSTables(StorageService.java:4145)
at
org.apache.cassandra.tools.NodeProbe.upgradeSSTables(NodeProbe.java:369)
at
org.apache.cassandra.tools.NodeProbe.lambda$upgradeSSTables$4(NodeProbe.java:424)
at org.apache.cassandra.tools.NodeProbe.perform(NodeProbe.java:435)
at
org.apache.cassandra.tools.NodeProbe.upgradeSSTables(NodeProbe.java:423)
at
org.apache.cassandra.tools.nodetool.UpgradeSSTable.execute(UpgradeSSTable.java:61)
at
org.apache.cassandra.tools.NodeTool$NodeToolCmd.runInternal(NodeTool.java:383)
at
org.apache.cassandra.tools.NodeTool$NodeToolCmd.run(NodeTool.java:368)
at org.apache.cassandra.tools.NodeTool.execute(NodeTool.java:267)
at
org.apache.cassandra.distributed.impl.Instance$DTestNodeTool.execute(Instance.java:1104)
at
org.apache.cassandra.distributed.impl.Instance.lambda$nodetoolResult$50(Instance.java:1013)
at org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:61)
at org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:71)
at
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
at
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
at java.base/java.lang.Thread.run(Thread.java:829)
INFO [node1_CompactionExecutor:1] node1 2024-02-01 10:13:04,956 Compaction
interrupted: Compaction(1b5a7900-c0e2-11ee-a2d5-d1a8b9818275, 0 / 49835
bytes)@1a850090-c0e2-11ee-a2d5-d1a8b9818275(distributed_test_keyspace, tbl)
{noformat}
So everything seems correct where the compaction is issued and then later
interrupted . I would suggest a fix where nodetool's {{upgradesstables}} would
just spin+retry for 5s if that exception is hit. My 2cts.
was (Author: bereng):
Posting my findings so far: The exception is
[thrown|https://github.com/apache/cassandra/blob/cassandra-5.0/src/java/org/apache/cassandra/db/compaction/CompactionTask.java#L204]
when the CSM is paused which can happen for number of reasons such as stopping
or forcing a compaction, acquiring sstables, etc. If a compaction is forced,
pausing the CSM, just as a compaction task is already running then it throws.
This is why it's so difficult to repro.
I didn't manage to repro but forcing to throw from the CompactionTask
reproduces the same exact stracktrace, error codes, messages and all the dots
seem to connect nicely. Here we have some log + extra stack traces where the
issued compaction pauses and resumes the CSM, then a later {{upgradesstables}}
does indeed interrupt the compaction. Given these are async and unlucky run
will hit the problem:
{noformat}
DEBUG [node1_isolatedExecutor:1] node1 2024-02-01 10:13:04,909 Cancelling
in-progress compactions for tbl
DEBUG [node1_isolatedExecutor:1] node1 2024-02-01 10:13:04,910 Bereng paused
java.lang.Exception
at
org.apache.cassandra.db.compaction.CompactionStrategyManager.pause(CompactionStrategyManager.java:306)
at
org.apache.cassandra.db.ColumnFamilyStore.pauseCompactionStrategies(ColumnFamilyStore.java:2899)
at
org.apache.cassandra.db.ColumnFamilyStore.runWithCompactionsDisabled(ColumnFamilyStore.java:2846)
at
org.apache.cassandra.db.ColumnFamilyStore.runWithCompactionsDisabled(ColumnFamilyStore.java:2815)
at
org.apache.cassandra.db.compaction.CompactionStrategyManager.getMaximalTasks(CompactionStrategyManager.java:1071)
at
org.apache.cassandra.db.compaction.CompactionManager.submitMaximal(CompactionManager.java:993)
at
org.apache.cassandra.distributed.test.UpgradeSSTablesTest.lambda$upgradeSSTablesInterruptsOngoingCompaction$53e4ed01$1(UpgradeSSTablesTest.java:84)
at
org.apache.cassandra.distributed.impl.IsolatedExecutor.lambda$async$10(IsolatedExecutor.java:156)
at org.apache.cassandra.concurrent.FutureTask$1.call(FutureTask.java:96)
at org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:61)
at org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:71)
at
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
at
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
at java.base/java.lang.Thread.run(Thread.java:829)
DEBUG [node1_isolatedExecutor:1] node1 2024-02-01 10:13:04,913 Bereng resume
java.lang.Exception
at
org.apache.cassandra.db.compaction.CompactionStrategyManager.resume(CompactionStrategyManager.java:287)
at
org.apache.cassandra.db.ColumnFamilyStore.resumeAll(ColumnFamilyStore.java:2917)
at
org.apache.cassandra.db.ColumnFamilyStore.lambda$pauseCompactionStrategies$14(ColumnFamilyStore.java:2902)
at
org.apache.cassandra.db.ColumnFamilyStore.runWithCompactionsDisabled(ColumnFamilyStore.java:2887)
at
org.apache.cassandra.db.ColumnFamilyStore.runWithCompactionsDisabled(ColumnFamilyStore.java:2815)
at
org.apache.cassandra.db.compaction.CompactionStrategyManager.getMaximalTasks(CompactionStrategyManager.java:1071)
at
org.apache.cassandra.db.compaction.CompactionManager.submitMaximal(CompactionManager.java:993)
at
org.apache.cassandra.distributed.test.UpgradeSSTablesTest.lambda$upgradeSSTablesInterruptsOngoingCompaction$53e4ed01$1(UpgradeSSTablesTest.java:84)
at
org.apache.cassandra.distributed.impl.IsolatedExecutor.lambda$async$10(IsolatedExecutor.java:156)
at org.apache.cassandra.concurrent.FutureTask$1.call(FutureTask.java:96)
at org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:61)
at org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:71)
at
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
at
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
at java.base/java.lang.Thread.run(Thread.java:829)
INFO [node1_isolatedExecutor:1] node1 2024-02-01 10:13:04,913 Major compaction
will not result in a single sstable - repaired and unrepaired data is kept
separate and compaction runs per data_file_directory.
DEBUG [node1_CompactionExecutor:1] node1 2024-02-01 10:13:04,915 FileStore /
(/dev/nvme0n1p3) has 257607498752 bytes available, checking if we can write
2639 bytes
INFO [node1_CompactionExecutor:1] node1 2024-02-01 10:13:04,916 Compacting
(1b5a7900-c0e2-11ee-a2d5-d1a8b9818275)
[/home/bereng/work/repos/bdpWS/18635/build/tmp/dtests9959175088994503563/node1/data0/distributed_test_keyspace/tbl-1a850090c0e211eea2d5d1a8b9818275/nb-4-big-Data.db:level=0,
/home/bereng/work/repos/bdpWS/18635/build/tmp/dtests9959175088994503563/node1/data0/distributed_test_keyspace/tbl-1a850090c0e211eea2d5d1a8b9818275/nb-1-big-Data.db:level=0,
/home/bereng/work/repos/bdpWS/18635/build/tmp/dtests9959175088994503563/node1/data0/distributed_test_keyspace/tbl-1a850090c0e211eea2d5d1a8b9818275/nb-13-big-Data.db:level=0,
/home/bereng/work/repos/bdpWS/18635/build/tmp/dtests9959175088994503563/node1/data0/distributed_test_keyspace/tbl-1a850090c0e211eea2d5d1a8b9818275/nb-7-big-Data.db:level=0,
/home/bereng/work/repos/bdpWS/18635/build/tmp/dtests9959175088994503563/node1/data0/distributed_test_keyspace/tbl-1a850090c0e211eea2d5d1a8b9818275/nb-10-big-Data.db:level=0,
]
INFO [node1_isolatedExecutor:2] node1 2024-02-01 10:13:04,928 Starting Upgrade
sstables on distributed_test_keyspace.[tbl]
DEBUG [node1_isolatedExecutor:2] node1 2024-02-01 10:13:04,932 Cancelling
in-progress compactions for tbl
DEBUG [node1_isolatedExecutor:2] node1 2024-02-01 10:13:04,932 Bereng paused
java.lang.Exception
at
org.apache.cassandra.db.compaction.CompactionStrategyManager.pause(CompactionStrategyManager.java:306)
at
org.apache.cassandra.db.ColumnFamilyStore.pauseCompactionStrategies(ColumnFamilyStore.java:2899)
at
org.apache.cassandra.db.ColumnFamilyStore.runWithCompactionsDisabled(ColumnFamilyStore.java:2846)
at
org.apache.cassandra.db.ColumnFamilyStore.runWithCompactionsDisabled(ColumnFamilyStore.java:2815)
at
org.apache.cassandra.db.ColumnFamilyStore.withAllSSTables(ColumnFamilyStore.java:2938)
at
org.apache.cassandra.db.compaction.CompactionManager.parallelAllSSTableOperation(CompactionManager.java:430)
at
org.apache.cassandra.db.compaction.CompactionManager.performSSTableRewrite(CompactionManager.java:584)
at
org.apache.cassandra.db.compaction.CompactionManager.performSSTableRewrite(CompactionManager.java:557)
at
org.apache.cassandra.db.ColumnFamilyStore.sstablesRewrite(ColumnFamilyStore.java:1813)
at
org.apache.cassandra.service.StorageService.rewriteSSTables(StorageService.java:4167)
at
org.apache.cassandra.service.StorageService.upgradeSSTables(StorageService.java:4145)
at
org.apache.cassandra.tools.NodeProbe.upgradeSSTables(NodeProbe.java:369)
at
org.apache.cassandra.tools.NodeProbe.lambda$upgradeSSTables$4(NodeProbe.java:424)
at org.apache.cassandra.tools.NodeProbe.perform(NodeProbe.java:435)
at
org.apache.cassandra.tools.NodeProbe.upgradeSSTables(NodeProbe.java:423)
at
org.apache.cassandra.tools.nodetool.UpgradeSSTable.execute(UpgradeSSTable.java:61)
at
org.apache.cassandra.tools.NodeTool$NodeToolCmd.runInternal(NodeTool.java:383)
at
org.apache.cassandra.tools.NodeTool$NodeToolCmd.run(NodeTool.java:368)
at org.apache.cassandra.tools.NodeTool.execute(NodeTool.java:267)
at
org.apache.cassandra.distributed.impl.Instance$DTestNodeTool.execute(Instance.java:1104)
at
org.apache.cassandra.distributed.impl.Instance.lambda$nodetoolResult$50(Instance.java:1013)
at org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:61)
at org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:71)
at
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
at
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
at java.base/java.lang.Thread.run(Thread.java:829)
INFO [node1_CompactionExecutor:1] node1 2024-02-01 10:13:04,956 Compaction
interrupted: Compaction(1b5a7900-c0e2-11ee-a2d5-d1a8b9818275, 0 / 49835
bytes)@1a850090-c0e2-11ee-a2d5-d1a8b9818275(distributed_test_keyspace, tbl)
{noformat}
So everything seems correct where the compaction is issued and then later
interrupted . I would suggest a fix where nodetool's {{upgradesstables}} would
just spin+retry for 5s if that exception is hit. My 2cts.
> Test failure: org.apache.cassandra.distributed.test.UpgradeSSTablesTest
> -----------------------------------------------------------------------
>
> Key: CASSANDRA-18635
> URL: https://issues.apache.org/jira/browse/CASSANDRA-18635
> Project: Cassandra
> Issue Type: Bug
> Components: Test/dtest/java
> Reporter: Brandon Williams
> Assignee: Josh McKenzie
> Priority: Normal
> Fix For: 5.0-rc, 5.x
>
>
> Seen here:
> https://app.circleci.com/pipelines/github/driftx/cassandra/1095/workflows/6114e2e3-8dcc-4bb0-b664-ae7d82c3349f/jobs/33405/tests
> {noformat}
> junit.framework.AssertionFailedError: expected:<0> but was:<2>
> at
> org.apache.cassandra.distributed.test.UpgradeSSTablesTest.upgradeSSTablesInterruptsOngoingCompaction(UpgradeSSTablesTest.java:86)
> {noformat}
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]