[ https://issues.apache.org/jira/browse/CASSANDRA-2792?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13053104#comment-13053104 ]
Hudson commented on CASSANDRA-2792: ----------------------------------- Integrated in Cassandra-0.7 #507 (See [https://builds.apache.org/job/Cassandra-0.7/507/]) Improve thread safety in StreamOutSession patch by slebresne; reviewed by jbellis for CASSANDRA-2792 slebresne : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1138148 Files : * /cassandra/branches/cassandra-0.7/CHANGES.txt * /cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/streaming/StreamOutSession.java > Bootstrapping node stalls. Bootstrapper thinks it is still streaming some > sstables. The source nodes do not. Caused by IllegalStateException on source > nodes. > ------------------------------------------------------------------------------------------------------------------------------------------------------------- > > Key: CASSANDRA-2792 > URL: https://issues.apache.org/jira/browse/CASSANDRA-2792 > Project: Cassandra > Issue Type: Bug > Components: Core > Affects Versions: 0.7.6 > Environment: Ubuntu > Reporter: Dominic Williams > Assignee: Sylvain Lebresne > Fix For: 0.7.7 > > Attachments: 0001-Make-StreamOutSession-threadSafe.patch > > Original Estimate: 4h > Remaining Estimate: 4h > > I am bootstrapping a node into a 4 node cluster with RF3 (1 node is currently > down due to sstable issues, but the cluster is running without issues). > There are two keyspaces FightMyMonster and FMM_Studio. The first keyspace > successfully streams and the whole operation is probably at 99% when it > stalls on some sstables in the much smaller FMM_Studio keyspace. > Netstats on the bootstrapping node reports it is still streaming: > Mode: Bootstrapping > Not sending any streams. > Streaming from: /192.168.1.4 > FMM_Studio: /var/opt/cassandra/data/FMM_Studio/PartsData-f-101-Data.db > sections=1 progress=0/76453 - 0% > FMM_Studio: /var/opt/cassandra/data/FMM_Studio/PartsData-f-103-Data.db > sections=1 progress=0/90475 - 0% > FMM_Studio: /var/opt/cassandra/data/FMM_Studio/PartsData-f-102-Data.db > sections=1 progress=0/4304182 - 0% > Streaming from: /192.168.1.3 > FMM_Studio: /var/opt/cassandra/data/FMM_Studio/PartsData-f-158-Data.db > sections=2 progress=0/146990 - 0% > FMM_Studio: /var/opt/cassandra/data/FMM_Studio/AuthorClasses-f-81-Data.db > sections=1 progress=0/3992 - 0% > FMM_Studio: /var/opt/cassandra/data/FMM_Studio/Studio-f-70-Data.db > sections=1 progress=0/1776 - 0% > FMM_Studio: /var/opt/cassandra/data/FMM_Studio/PartsData-f-159-Data.db > sections=2 progress=0/136829 - 0% > FMM_Studio: /var/opt/cassandra/data/FMM_Studio/PartsData-f-157-Data.db > sections=2 progress=0/5779597 - 0% > FMM_Studio: /var/opt/cassandra/data/FMM_Studio/AuthorClasses-f-82-Data.db > sections=1 progress=0/161 - 0% > FMM_Studio: /var/opt/cassandra/data/FMM_Studio/Studio-f-71-Data.db > sections=1 progress=0/135 - 0% > Pool Name Active Pending Completed > Commands n/a 0 334 > Responses n/a 0 421957 > However, running netstats on the source nodes reports they are not streaming: > Mode: Normal > Nothing streaming to /192.168.1.9 > Not receiving any streams. > Pool Name Active Pending Completed > Commands n/a 0 1949476 > Responses n/a 1 1778768 > Examination of the logs on the source nodes show an IllegalStateException > that has likely interrupted/broken the streaming process. > 17 22:27:05,924 StreamOut.java (line 126) Beginning transfer to /192.168.1.9 > INFO [StreamStage:1] 2011-06-17 22:27:05,925 StreamOut.java (line 100) > Flushing memtables for FMM_Studio... > INFO [StreamStage:1] 2011-06-17 22:27:06,004 StreamOut.java (line 173) > Stream context metadata > [/var/opt/cassandra/data/FMM_Studio/Classes-f-107-Data.db sections=1 > progress=0/1585378 - 0%, /var/opt/cas > sandra/data/FMM_Studio/PartsData-f-100-Data.db sections=1 progress=0/76453 - > 0%, /var/opt/cassandra/data/FMM_Studio/PartsData-f-98-Data.db sections=1 > progress=0/4309514 - 0%, /var/opt/cassandra/data/FMM > _Studio/PartsData-f-99-Data.db sections=1 progress=0/90475 - 0%], 11 sstables. > INFO [StreamStage:1] 2011-06-17 22:27:06,005 StreamOutSession.java (line > 174) Streaming to /192.168.1.9 > INFO [StreamStage:1] 2011-06-17 22:27:06,006 StreamOut.java (line 126) > Beginning transfer to /192.168.1.9 > INFO [StreamStage:1] 2011-06-17 22:27:06,007 StreamOut.java (line 100) > Flushing memtables for FightMyMonster... > INFO [StreamStage:1] 2011-06-17 22:27:06,007 ColumnFamilyStore.java (line > 1065) Enqueuing flush of Memtable-MonsterMarket_1@1054909557(338 bytes, 24 > operations) > INFO [StreamStage:1] 2011-06-17 22:27:06,007 ColumnFamilyStore.java (line > 1065) Enqueuing flush of Memtable-UserFights@239934867(1124836 bytes, 965 > operations) > INFO [FlushWriter:409] 2011-06-17 22:27:06,007 Memtable.java (line 157) > Writing Memtable-MonsterMarket_1@1054909557(338 bytes, 24 operations) > INFO [StreamStage:1] 2011-06-17 22:27:06,007 ColumnFamilyStore.java (line > 1065) Enqueuing flush of Memtable-Users_CisIndex@1758504250(242 bytes, 8 > operations) > INFO [StreamStage:1] 2011-06-17 22:27:06,008 ColumnFamilyStore.java (line > 1065) Enqueuing flush of Memtable-Tribes@1510979736(18318 bytes, 703 > operations) > INFO [StreamStage:1] 2011-06-17 22:27:06,008 ColumnFamilyStore.java (line > 1065) Enqueuing flush of Memtable-ColumnViews_TimeUUID@864545260(2073 bytes, > 63 operations) > INFO [StreamStage:1] 2011-06-17 22:27:06,008 ColumnFamilyStore.java (line > 1065) Enqueuing flush of Memtable-MonsterMarket_0@537829218(2600 bytes, 129 > operations) > INFO [FlushWriter:409] 2011-06-17 22:27:06,069 Memtable.java (line 172) > Completed flushing > /var/opt/cassandra/data/FightMyMonster/MonsterMarket_1-f-3799-Data.db (1774 > bytes) > INFO [FlushWriter:409] 2011-06-17 22:27:06,069 Memtable.java (line 157) > Writing Memtable-UserFights@239934867(1124836 bytes, 965 operations) > INFO [StreamStage:1] 2011-06-17 22:27:06,070 ColumnFamilyStore.java (line > 1065) Enqueuing flush of Memtable-UserSigninLog@1692186117(4043 bytes, 137 > operations) > INFO [FlushWriter:409] 2011-06-17 22:27:06,161 Memtable.java (line 172) > Completed flushing > /var/opt/cassandra/data/FightMyMonster/UserFights-f-8192-Data.db (1179202 > bytes) > INFO [FlushWriter:409] 2011-06-17 22:27:06,161 Memtable.java (line 157) > Writing Memtable-Users_CisIndex@1758504250(242 bytes, 8 operations) > INFO [CompactionExecutor:1] 2011-06-17 22:27:06,161 CompactionManager.java > (line 395) Compacting > [SSTableReader(path='/var/opt/cassandra/data/FightMyMonster/UserFights-f-8189-Data.db'),SSTableReader(pa > th='/var/opt/cassandra/data/FightMyMonster/UserFights-f-8190-Data.db'),SSTableReader(path='/var/opt/cassandra/data/FightMyMonster/UserFights-f-8191-Data.db'),SSTableReader(path='/var/opt/cassandra/data/ > FightMyMonster/UserFights-f-8192-Data.db')] > INFO [StreamStage:1] 2011-06-17 22:27:06,162 ColumnFamilyStore.java (line > 1065) Enqueuing flush of Memtable-TribeFights@321579649(138 bytes, 3 > operations) > ERROR [MiscStage:1] 2011-06-17 22:27:06,168 AbstractCassandraDaemon.java > (line 114) Fatal exception in thread Thread[MiscStage:1,5,main] > java.lang.IllegalStateException: target reports current file is > /var/opt/cassandra/data/FMM_Studio/Classes-f-107-Data.db but is null > at > org.apache.cassandra.streaming.StreamOutSession.validateCurrentFile(StreamOutSession.java:166) > at > org.apache.cassandra.streaming.StreamReplyVerbHandler.doVerb(StreamReplyVerbHandler.java:58) > at > org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:72) > at > java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) > at java.lang.Thread.run(Thread.java:619) > ERROR [MiscStage:1] 2011-06-17 22:27:06,168 AbstractCassandraDaemon.java > (line 114) Fatal exception in thread Thread[MiscStage:1,5,main] > java.lang.IllegalStateException: target reports current file is > /var/opt/cassandra/data/FMM_Studio/Classes-f-107-Data.db but is null > at > org.apache.cassandra.streaming.StreamOutSession.validateCurrentFile(StreamOutSession.java:166) > at > org.apache.cassandra.streaming.StreamReplyVerbHandler.doVerb(StreamReplyVerbHandler.java:58) > at > org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:72) > at > java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) > at java.lang.Thread.run(Thread.java:619) > There are two problems. Firstly the source nodes should report to the > bootstrapping node that there has been a problem, and/or the bootstrapping > node should timeout and report the the issue. > Secondly, there is an issue with what is causing IllegalStateException. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira