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

Yuki Morishita commented on CASSANDRA-13178:
--------------------------------------------

After CASSANDRA-8838 (v2.2+), when bootstrap streaming failed, the node should 
keep running (but not joinning to the cluster) and user can manually resume 
bootstrap streaming (nodetool bootstrap resume).

I'm afraid we are not backporting this feature to 2.1.

> Cassandra crash if a table is dropped while joining
> ---------------------------------------------------
>
>                 Key: CASSANDRA-13178
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-13178
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Streaming and Messaging
>         Environment: Cassandra-2.1.9
>            Reporter: Brian Gallew
>
> I just had three different nodes crash while joining a cluster.  They've been 
> streaming data for 2 days now, and each one exited with this error:
> {code}
> ERROR [STREAM-IN-/XX.XX.144.63] 2017-02-02 20:42:45,275 
> StreamSession.java:505 - [Stream #b4279df0-e7de-11e6-aa5a-452e7f2ad12b] 
> Streaming error occurred
> java.io.IOException: CF 8280bcb0-cdf7-11e6-b227-b97a72c7aa0d was dropped 
> during streaming
>       at 
> org.apache.cassandra.streaming.compress.CompressedStreamReader.read(CompressedStreamReader.java:71)
>  ~[apache-cassandra-2.1.9.jar:2.1.9-SNAPSHOT]
>       at 
> org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:48)
>  ~[apache-cassandra-2.1.9.jar:2.1.9-SNAPSHOT]
>       at 
> org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:38)
>  ~[apache-cassandra-2.1.9.jar:2.1.9-SNAPSHOT]
>       at 
> org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:56)
>  ~[apache-cassandra-2.1.9.jar:2.1.9-SNAPSHOT]
>       at 
> org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:250)
>  ~[apache-cassandra-2.1.9.jar:2.1.9-SNAPSHOT]
>       at java.lang.Thread.run(Thread.java:745) [na:1.8.0_91]
> INFO  [STREAM-IN-/XX.XX.144.63] 2017-02-02 20:42:45,275 
> StreamResultFuture.java:180 - [Stream #b4279df0-e7de-11e6-aa5a-452e7f2ad12b] 
> Session with /XX.XX.144.63 is complete
> WARN  [STREAM-IN-/XX.XX.144.63] 2017-02-02 20:42:45,277 
> StreamResultFuture.java:207 - [Stream #b4279df0-e7de-11e6-aa5a-452e7f2ad12b] 
> Stream failed
> ERROR [main] 2017-02-02 20:42:45,278 CassandraDaemon.java:579 - Exception 
> encountered during startup
> java.lang.RuntimeException: Error during boostrap: Stream failed
>       at 
> org.apache.cassandra.dht.BootStrapper.bootstrap(BootStrapper.java:86) 
> ~[apache-cassandra-2.1.9.jar:2.1.9-SNAPSHOT]
>       at 
> org.apache.cassandra.service.StorageService.bootstrap(StorageService.java:1148)
>  ~[apache-cassandra-2.1.9.jar:2.1.9-SNAPSHOT]
>       at 
> org.apache.cassandra.service.StorageService.joinTokenRing(StorageService.java:938)
>  ~[apache-cassandra-2.1.9.jar:2.1.9-SNAPSHOT]
>       at 
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:734)
>  ~[apache-cassandra-2.1.9.jar:2.1.9-SNAPSHOT]
>       at 
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:611)
>  ~[apache-cassandra-2.1.9.jar:2.1.9-SNAPSHOT]
>       at 
> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:387) 
> [apache-cassandra-2.1.9.jar:2.1.9-SNAPSHOT]
>       at 
> org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:562)
>  [apache-cassandra-2.1.9.jar:2.1.9-SNAPSHOT]
>       at 
> org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:651) 
> [apache-cassandra-2.1.9.jar:2.1.9-SNAPSHOT]
> Caused by: org.apache.cassandra.streaming.StreamException: Stream failed
>       at 
> org.apache.cassandra.streaming.management.StreamEventJMXNotifier.onFailure(StreamEventJMXNotifier.java:85)
>  ~[apache-cassandra-2.1.9.jar:2.1.9-SNAPSHOT]
>       at com.google.common.util.concurrent.Futures$4.run(Futures.java:1172) 
> ~[guava-16.0.jar:na]
>       at 
> com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297)
>  ~[guava-16.0.jar:na]
>       at 
> com.google.common.util.concurrent.ExecutionList.executeListener(ExecutionList.java:156)
>  ~[guava-16.0.jar:na]
>       at 
> com.google.common.util.concurrent.ExecutionList.execute(ExecutionList.java:145)
>  ~[guava-16.0.jar:na]
>       at 
> com.google.common.util.concurrent.AbstractFuture.setException(AbstractFuture.java:202)
>  ~[guava-16.0.jar:na]
>       at 
> org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamResultFuture.java:208)
>  ~[apache-cassandra-2.1.9.jar:2.1.9-SNAPSHOT]
>       at 
> org.apache.cassandra.streaming.StreamResultFuture.handleSessionComplete(StreamResultFuture.java:184)
>  ~[apache-cassandra-2.1.9.jar:2.1.9-SNAPSHOT]
>       at 
> org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:415)
>  ~[apache-cassandra-2.1.9.jar:2.1.9-SNAPSHOT]
>       at 
> org.apache.cassandra.streaming.StreamSession.onError(StreamSession.java:510) 
> ~[apache-cassandra-2.1.9.jar:2.1.9-SNAPSHOT]
>       at 
> org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:268)
>  ~[apache-cassandra-2.1.9.jar:2.1.9-SNAPSHOT]
>       at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_91]
> WARN  [StorageServiceShutdownHook] 2017-02-02 20:42:45,286 Gossiper.java:1450 
> - No local state or state is in silent shutdown, not announcing shutdown
> INFO  [StorageServiceShutdownHook] 2017-02-02 20:42:45,286 
> MessagingService.java:735 - Waiting for messaging service to quiesce
> INFO  [ACCEPT-/XX.XX.146.177] 2017-02-02 20:42:45,293 
> MessagingService.java:1011 - MessagingService has terminated the accept() 
> thread
> INFO  [CompactionExecutor:6837] 2017-02-02 20:42:45,323 
> ColumnFamilyStore.java:900 - Enqueuing flush of compactions_in_progress: 1687 
> (0%) on-heap, 0 (0%) off-heap
> INFO  [MemtableFlushWriter:5703] 2017-02-02 20:42:45,324 Memtable.java:347 - 
> Writing Memtable-compactions_in_progress@689094870(0.184KiB serialized bytes, 
> 11 ops, 0%/0% of on/off-heap limit)
> INFO  [MemtableFlushWriter:5703] 2017-02-02 20:42:45,324 Memtable.java:382 - 
> Completed flushing 
> /mnt/cassandra/data/system/compactions_in_progress-55080ab05d9c388690a4acb25fe1f77b/system-compactions_in_progress-tmp-ka-5779-Data.db
>  (0.000KiB) for commitlog position ReplayPosition(segmentId=1485885417287, 
> position=23644862)
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to