You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Alex Petrov (JIRA)" <ji...@apache.org> on 2016/08/01 08:46:20 UTC

[jira] [Commented] (CASSANDRA-12251) dtest failure in upgrade_tests.cql_tests.TestCQLNodes3RF3_Upgrade_current_3_x_To_indev_3_x.whole_list_conditional_test

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

Alex Petrov commented on CASSANDRA-12251:
-----------------------------------------

Unfortunately, I could not reproduce this locally. However, given the exception it's more or less clear what has happened. It's just much harder to land nodes into the same state. 

The tasks executed by {{MigrationManager}} are landing in the {{optional}} tasks of scheduled executor [here|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/MigrationManager.java]. Optional tasks are not being waited for during shutdown or drain. Although even putting the task into the {{nonPeriodicTasks}} executor won't fix the problem since post-flush executor is being shut down before the periodic executor [here|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/StorageService.java#L4251]. 

In order to fix the issue, we need to both move the migration task to non-periodic and make sure that shutdown order is correct. Since mutation stage is already shut down, I left commit log shutdown on ints place and only moved the post flush executor shutdown after the periodic tasks.

> dtest failure in upgrade_tests.cql_tests.TestCQLNodes3RF3_Upgrade_current_3_x_To_indev_3_x.whole_list_conditional_test
> ----------------------------------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-12251
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-12251
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Philip Thompson
>            Assignee: Alex Petrov
>              Labels: dtest
>         Attachments: node1.log, node1_debug.log, node1_gc.log, node2.log, node2_debug.log, node2_gc.log, node3.log, node3_debug.log, node3_gc.log
>
>
> example failure:
> http://cassci.datastax.com/job/cassandra-3.8_dtest_upgrade/1/testReport/upgrade_tests.cql_tests/TestCQLNodes3RF3_Upgrade_current_3_x_To_indev_3_x/whole_list_conditional_test
> Failed on CassCI build cassandra-3.8_dtest_upgrade #1
> Relevant error in logs is
> {code}
> Unexpected error in node1 log, error: 
> ERROR [InternalResponseStage:2] 2016-07-20 04:58:45,876 CassandraDaemon.java:217 - Exception in thread Thread[InternalResponseStage:2,5,main]
> java.util.concurrent.RejectedExecutionException: ThreadPoolExecutor has shut down
> 	at org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor$1.rejectedExecution(DebuggableThreadPoolExecutor.java:61) ~[apache-cassandra-3.7.jar:3.7]
> 	at java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:823) ~[na:1.8.0_51]
> 	at java.util.concurrent.ThreadPoolExecutor.execute(ThreadPoolExecutor.java:1369) ~[na:1.8.0_51]
> 	at org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.execute(DebuggableThreadPoolExecutor.java:165) ~[apache-cassandra-3.7.jar:3.7]
> 	at java.util.concurrent.AbstractExecutorService.submit(AbstractExecutorService.java:112) ~[na:1.8.0_51]
> 	at org.apache.cassandra.db.ColumnFamilyStore.switchMemtable(ColumnFamilyStore.java:842) ~[apache-cassandra-3.7.jar:3.7]
> 	at org.apache.cassandra.db.ColumnFamilyStore.switchMemtableIfCurrent(ColumnFamilyStore.java:822) ~[apache-cassandra-3.7.jar:3.7]
> 	at org.apache.cassandra.db.ColumnFamilyStore.forceFlush(ColumnFamilyStore.java:891) ~[apache-cassandra-3.7.jar:3.7]
> 	at org.apache.cassandra.schema.SchemaKeyspace.lambda$flush$1(SchemaKeyspace.java:279) ~[apache-cassandra-3.7.jar:3.7]
> 	at org.apache.cassandra.schema.SchemaKeyspace$$Lambda$200/1129213153.accept(Unknown Source) ~[na:na]
> 	at java.lang.Iterable.forEach(Iterable.java:75) ~[na:1.8.0_51]
> 	at org.apache.cassandra.schema.SchemaKeyspace.flush(SchemaKeyspace.java:279) ~[apache-cassandra-3.7.jar:3.7]
> 	at org.apache.cassandra.schema.SchemaKeyspace.mergeSchema(SchemaKeyspace.java:1271) ~[apache-cassandra-3.7.jar:3.7]
> 	at org.apache.cassandra.schema.SchemaKeyspace.mergeSchemaAndAnnounceVersion(SchemaKeyspace.java:1253) ~[apache-cassandra-3.7.jar:3.7]
> 	at org.apache.cassandra.service.MigrationTask$1.response(MigrationTask.java:92) ~[apache-cassandra-3.7.jar:3.7]
> 	at org.apache.cassandra.net.ResponseVerbHandler.doVerb(ResponseVerbHandler.java:53) ~[apache-cassandra-3.7.jar:3.7]
> 	at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:64) ~[apache-cassandra-3.7.jar:3.7]
> 	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[na:1.8.0_51]
> 	at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[na:1.8.0_51]
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) ~[na:1.8.0_51]
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_51]
> 	at java.lang.Thread.run(Thread.java:745) [na:1.8.0_51]
> {code}
> This is on a mixed 3.0.8, 3.8-tentative cluster



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)