cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Stefania (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-12457) dtest failure in upgrade_tests.cql_tests.TestCQLNodes2RF1_Upgrade_current_2_1_x_To_indev_2_2_x.bug_5732_test
Date Thu, 01 Sep 2016 08:44:20 GMT

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

Stefania commented on CASSANDRA-12457:
--------------------------------------

The analysis of the latest failure logs revealed two different scenarios:

* In one case the instance tidier for the sstable with leaks did run, so the reference is
not leaked, but the runnable in the non periodic tasks that is launched asynchronously in
the instance tidier did not run at all, explaining the leaked resources. 

* In the other case, not only did the the non periodic task runnable not run, but also another
runnable task did not complete.  

Both could be explained by the shutdown call of the non periodic tasks and the fact that the
execute-delayed-tasks policy is currently set to false, so I will schedule another run with
this policy set to true. Still, it is really odd that there is no interrupted exception or
trace of a canceled future (I did have runs with these logs enabled and never saw anything).

For one of the two cases, we cannot rule out a deadlock whilst executing the runnable that
did non complete. I noticed that the sstable global tidier calls {{SystemKeyspace.clearSSTableReadMeter()}},
which *applies a mutation to {{sstable_activity}} after all the system tables have been flushed
and the commitlog has been stopped*. So regardless of whether there is a deadlock or not,
I think we should stop the compactions before we flush the system sstables and stop the commit
log, so that any obsoleted sstable can update {{sstable_activity}}.

> dtest failure in upgrade_tests.cql_tests.TestCQLNodes2RF1_Upgrade_current_2_1_x_To_indev_2_2_x.bug_5732_test
> ------------------------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-12457
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-12457
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Craig Kodman
>            Assignee: Stefania
>              Labels: dtest
>             Fix For: 2.2.x
>
>         Attachments: 12457_2.1_logs_with_allocation_stacks.tar.gz, 12457_2.2_logs_with_allocation_stacks_1.tar.gz,
12457_2.2_logs_with_allocation_stacks_2.tar.gz, 12457_2.2_logs_with_allocation_stacks_3.tar.gz,
12457_2.2_logs_with_allocation_stacks_4.tar.gz, node1.log, node1_debug.log, node1_gc.log,
node2.log
>
>
> example failure:
> http://cassci.datastax.com/job/cassandra-2.2_dtest_upgrade/16/testReport/upgrade_tests.cql_tests/TestCQLNodes2RF1_Upgrade_current_2_1_x_To_indev_2_2_x/bug_5732_test
> {code}
> Stacktrace
>   File "/usr/lib/python2.7/unittest/case.py", line 358, in run
>     self.tearDown()
>   File "/home/automaton/cassandra-dtest/upgrade_tests/upgrade_base.py", line 216, in
tearDown
>     super(UpgradeTester, self).tearDown()
>   File "/home/automaton/cassandra-dtest/dtest.py", line 666, in tearDown
>     raise AssertionError('Unexpected error in log, see stdout')
> "Unexpected error in log, see stdout\n-------------------- >> begin captured logging
<< --------------------\ndtest: DEBUG: Upgrade test beginning, setting CASSANDRA_VERSION
to 2.1.15, and jdk to 8. (Prior values will be restored after test).\ndtest: DEBUG: cluster
ccm directory: /mnt/tmp/dtest-D8UF3i\ndtest: DEBUG: Done setting configuration options:\n{
  'initial_token': None,\n    'num_tokens': '32',\n    'phi_convict_threshold': 5,\n    'range_request_timeout_in_ms':
10000,\n    'read_request_timeout_in_ms': 10000,\n    'request_timeout_in_ms': 10000,\n  
 'truncate_request_timeout_in_ms': 10000,\n    'write_request_timeout_in_ms': 10000}\ndtest:
DEBUG: [[Row(table_name=u'ks', index_name=u'test.testindex')], [Row(table_name=u'ks', index_name=u'test.testindex')]]\ndtest:
DEBUG: upgrading node1 to git:91f7387e1f785b18321777311a5c3416af0663c2\nccm: INFO: Fetching
Cassandra updates...\ndtest: DEBUG: Querying upgraded node\ndtest: DEBUG: Querying old node\ndtest:
DEBUG: removing ccm cluster test at: /mnt/tmp/dtest-D8UF3i\ndtest: DEBUG: clearing ssl stores
from [/mnt/tmp/dtest-D8UF3i] directory\n--------------------- >> end captured logging
<< ---------------------"
> {code}
> {code}
> Standard Output
> http://git-wip-us.apache.org/repos/asf/cassandra.git git:91f7387e1f785b18321777311a5c3416af0663c2
> Unexpected error in node1 log, error: 
> ERROR [Reference-Reaper:1] 2016-08-13 01:34:34,581 Ref.java:199 - LEAK DETECTED: a reference
(org.apache.cassandra.utils.concurrent.Ref$State@73deb57f) to class org.apache.cassandra.io.sstable.SSTableReader$DescriptorTypeTidy@2098812276:/mnt/tmp/dtest-D8UF3i/test/node1/data1/system/schema_columns-296e9c049bec3085827dc17d3df2122a/system-schema_columns-ka-4
was not released before the reference was garbage collected
> Unexpected error in node1 log, error: 
> ERROR [Reference-Reaper:1] 2016-08-13 01:34:34,581 Ref.java:199 - LEAK DETECTED: a reference
(org.apache.cassandra.utils.concurrent.Ref$State@7926de0f) to class org.apache.cassandra.utils.concurrent.WrappedSharedCloseable$1@1009016655:[[OffHeapBitSet]]
was not released before the reference was garbage collected
> Unexpected error in node1 log, error: 
> ERROR [Reference-Reaper:1] 2016-08-13 01:34:34,581 Ref.java:199 - LEAK DETECTED: a reference
(org.apache.cassandra.utils.concurrent.Ref$State@3a5760f9) to class org.apache.cassandra.io.util.MmappedSegmentedFile$Cleanup@223486002:/mnt/tmp/dtest-D8UF3i/test/node1/data0/system/schema_columns-296e9c049bec3085827dc17d3df2122a/system-schema_columns-ka-3-Index.db
was not released before the reference was garbage collected
> Unexpected error in node1 log, error: 
> ERROR [Reference-Reaper:1] 2016-08-13 01:34:34,582 Ref.java:199 - LEAK DETECTED: a reference
(org.apache.cassandra.utils.concurrent.Ref$State@42cb4131) to class org.apache.cassandra.utils.concurrent.WrappedSharedCloseable$1@1544265728:[Memory@[0..4),
Memory@[0..a)] was not released before the reference was garbage collected
> Unexpected error in node1 log, error: 
> ERROR [Reference-Reaper:1] 2016-08-13 01:34:34,582 Ref.java:199 - LEAK DETECTED: a reference
(org.apache.cassandra.utils.concurrent.Ref$State@5dda43d0) to class org.apache.cassandra.io.util.MmappedSegmentedFile$Cleanup@1100327913:/mnt/tmp/dtest-D8UF3i/test/node1/data1/system/schema_columns-296e9c049bec3085827dc17d3df2122a/system-schema_columns-ka-4-Index.db
was not released before the reference was garbage collected
> Unexpected error in node1 log, error: 
> ERROR [Reference-Reaper:1] 2016-08-13 01:34:34,582 Ref.java:199 - LEAK DETECTED: a reference
(org.apache.cassandra.utils.concurrent.Ref$State@59cfa823) to class org.apache.cassandra.utils.concurrent.WrappedSharedCloseable$1@1480923322:[Memory@[0..4),
Memory@[0..a)] was not released before the reference was garbage collected
> Unexpected error in node1 log, error: 
> ERROR [Reference-Reaper:1] 2016-08-13 01:34:34,601 Ref.java:199 - LEAK DETECTED: a reference
(org.apache.cassandra.utils.concurrent.Ref$State@570e14a1) to class org.apache.cassandra.io.util.CompressedPoolingSegmentedFile$Cleanup@992487242:/mnt/tmp/dtest-D8UF3i/test/node1/data1/system/schema_columns-296e9c049bec3085827dc17d3df2122a/system-schema_columns-ka-4-Data.db
was not released before the reference was garbage collected
> Unexpected error in node1 log, error: 
> ERROR [Reference-Reaper:1] 2016-08-13 01:34:34,602 Ref.java:199 - LEAK DETECTED: a reference
(org.apache.cassandra.utils.concurrent.Ref$State@1f021ebc) to class org.apache.cassandra.utils.concurrent.WrappedSharedCloseable$1@1878148398:[Memory@[0..4),
Memory@[0..e)] was not released before the reference was garbage collected
> Unexpected error in node1 log, error: 
> ERROR [Reference-Reaper:1] 2016-08-13 01:34:34,604 Ref.java:199 - LEAK DETECTED: a reference
(org.apache.cassandra.utils.concurrent.Ref$State@48feef6d) to class org.apache.cassandra.io.sstable.SSTableReader$DescriptorTypeTidy@848724815:/mnt/tmp/dtest-D8UF3i/test/node1/data0/system/schema_columns-296e9c049bec3085827dc17d3df2122a/system-schema_columns-ka-3
was not released before the reference was garbage collected
> Unexpected error in node1 log, error: 
> ERROR [Reference-Reaper:1] 2016-08-13 01:34:34,605 Ref.java:199 - LEAK DETECTED: a reference
(org.apache.cassandra.utils.concurrent.Ref$State@3dee8c5f) to class org.apache.cassandra.io.sstable.SSTableReader$DescriptorTypeTidy@1078490617:/mnt/tmp/dtest-D8UF3i/test/node1/data1/system/schema_columns-296e9c049bec3085827dc17d3df2122a/system-schema_columns-ka-2
was not released before the reference was garbage collected
> Unexpected error in node1 log, error: 
> ERROR [Reference-Reaper:1] 2016-08-13 01:34:34,614 Ref.java:199 - LEAK DETECTED: a reference
(org.apache.cassandra.utils.concurrent.Ref$State@7f726f1a) to class org.apache.cassandra.io.util.CompressedPoolingSegmentedFile$Cleanup@2037913408:/mnt/tmp/dtest-D8UF3i/test/node1/data0/system/schema_columns-296e9c049bec3085827dc17d3df2122a/system-schema_columns-ka-3-Data.db
was not released before the reference was garbage collected
> Unexpected error in node1 log, error: 
> ERROR [Reference-Reaper:1] 2016-08-13 01:34:34,615 Ref.java:199 - LEAK DETECTED: a reference
(org.apache.cassandra.utils.concurrent.Ref$State@303df044) to class org.apache.cassandra.io.util.CompressedPoolingSegmentedFile$Cleanup@861514759:/mnt/tmp/dtest-D8UF3i/test/node1/data1/system/schema_columns-296e9c049bec3085827dc17d3df2122a/system-schema_columns-ka-2-Data.db
was not released before the reference was garbage collected
> Unexpected error in node1 log, error: 
> ERROR [Reference-Reaper:1] 2016-08-13 01:34:34,616 Ref.java:199 - LEAK DETECTED: a reference
(org.apache.cassandra.utils.concurrent.Ref$State@5fbf0fc9) to class org.apache.cassandra.utils.concurrent.WrappedSharedCloseable$1@1715786089:[[OffHeapBitSet]]
was not released before the reference was garbage collected
> Unexpected error in node1 log, error: 
> ERROR [Reference-Reaper:1] 2016-08-13 01:34:34,616 Ref.java:199 - LEAK DETECTED: a reference
(org.apache.cassandra.utils.concurrent.Ref$State@3924b235) to class org.apache.cassandra.io.util.MmappedSegmentedFile$Cleanup@1197672578:/mnt/tmp/dtest-D8UF3i/test/node1/data1/system/schema_columns-296e9c049bec3085827dc17d3df2122a/system-schema_columns-ka-2-Index.db
was not released before the reference was garbage collected
> Unexpected error in node1 log, error: 
> ERROR [Reference-Reaper:1] 2016-08-13 01:34:34,616 Ref.java:199 - LEAK DETECTED: a reference
(org.apache.cassandra.utils.concurrent.Ref$State@600596e0) to class org.apache.cassandra.utils.concurrent.WrappedSharedCloseable$1@545967120:[[OffHeapBitSet]]
was not released before the reference was garbage collected
> {code}



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

Mime
View raw message