[ https://issues.apache.org/jira/browse/CASSANDRA-6565?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13931732#comment-13931732 ]
Pas commented on CASSANDRA-6565: -------------------------------- Hello! Slightly similar issue, I didn't want to further pollute the issue tracker, so: adding plus one node to a 2-node ring resulted in (all three on 2.0.6): # grep 192.168.36.171 system.log {code} INFO [HANDSHAKE-/192.168.36.173] 2014-03-12 13:45:57,801 OutboundTcpConnection.java (line 386) Handshaking version with /192.168.36.173 INFO [GossipStage:1] 2014-03-12 13:46:00,170 Gossiper.java (line 883) Node /192.168.36.173 is now part of the cluster INFO [HANDSHAKE-/192.168.36.173] 2014-03-12 13:46:00,172 OutboundTcpConnection.java (line 386) Handshaking version with /192.168.36.173 INFO [RequestResponseStage:9] 2014-03-12 13:46:00,185 Gossiper.java (line 849) InetAddress /192.168.36.173 is now UP INFO [STREAM-INIT-/192.168.36.173:34610] 2014-03-12 13:46:48,608 StreamResultFuture.java (line 121) [Stream #2e0c6a10-a9e4-11e3-9c59-5340fa68c75f] Received streaming plan for Bootstrap INFO [STREAM-IN-/192.168.36.173] 2014-03-12 13:47:28,974 ColumnFamilyStore.java (line 785) Enqueuing flush of Memtable-TStore@119522965(23215181/50011094 serialized/live bytes, 84365 ops) INFO [STREAM-IN-/192.168.36.173] 2014-03-12 13:47:52,156 StreamResultFuture.java (line 173) [Stream #2e0c6a10-a9e4-11e3-9c59-5340fa68c75f] Prepare completed. Receiving 0 files(0 bytes), sending 1971 files(6355271334 bytes) ERROR [STREAM-OUT-/192.168.36.173] 2014-03-12 13:51:21,163 StreamSession.java (line 418) [Stream #2e0c6a10-a9e4-11e3-9c59-5340fa68c75f] Streaming error occurred INFO [STREAM-OUT-/192.168.36.173] 2014-03-12 13:51:21,181 StreamResultFuture.java (line 186) [Stream #2e0c6a10-a9e4-11e3-9c59-5340fa68c75f] Session with /192.168.36.173 is complete INFO [STREAM-IN-/192.168.36.173] 2014-03-12 13:51:21,181 StreamResultFuture.java (line 186) [Stream #2e0c6a10-a9e4-11e3-9c59-5340fa68c75f] Session with /192.168.36.173 is complete WARN [STREAM-OUT-/192.168.36.173] 2014-03-12 13:51:21,184 StreamResultFuture.java (line 215) [Stream #2e0c6a10-a9e4-11e3-9c59-5340fa68c75f] Stream failed WARN [STREAM-IN-/192.168.36.173] 2014-03-12 13:51:21,185 StreamResultFuture.java (line 215) [Stream #2e0c6a10-a9e4-11e3-9c59-5340fa68c75f] Stream failed ERROR [STREAM-OUT-/192.168.36.173] 2014-03-12 13:51:21,185 StreamSession.java (line 418) [Stream #2e0c6a10-a9e4-11e3-9c59-5340fa68c75f] Streaming error occurred [....] WARN [STREAM-OUT-/192.168.36.173] 2014-03-12 13:51:21,461 StreamResultFuture.java (line 215) [Stream #2e0c6a10-a9e4-11e3-9c59-5340fa68c75f] Stream failed ERROR [STREAM-OUT-/192.168.36.173] 2014-03-12 13:51:21,462 StreamSession.java (line 418) [Stream #2e0c6a10-a9e4-11e3-9c59-5340fa68c75f] Streaming error occurred INFO [STREAM-OUT-/192.168.36.173] 2014-03-12 13:51:21,462 StreamResultFuture.java (line 186) [Stream #2e0c6a10-a9e4-11e3-9c59-5340fa68c75f] Session with /192.168.36.173 is complete WARN [STREAM-OUT-/192.168.36.173] 2014-03-12 13:51:21,462 StreamResultFuture.java (line 215) [Stream #2e0c6a10-a9e4-11e3-9c59-5340fa68c75f] Stream failed INFO [GossipStage:1] 2014-03-12 13:51:23,158 Gossiper.java (line 863) InetAddress /192.168.36.173 is now DOWN INFO [GossipTasks:1] 2014-03-12 13:51:51,522 Gossiper.java (line 633) FatClient /192.168.36.173 has been silent for 30000ms, removing from gossip {code} And there is always a Broken Pipe IOException after Streaming error occured {code} java.io.IOException: Broken pipe at sun.nio.ch.FileDispatcherImpl.write0(Native Method) at sun.nio.ch.SocketDispatcher.write(SocketDispatcher.java:47) at sun.nio.ch.IOUtil.writeFromNativeBuffer(IOUtil.java:93) at sun.nio.ch.IOUtil.write(IOUtil.java:65) at sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:487) at org.apache.cassandra.streaming.messages.StreamMessage.serialize(StreamMessage.java:44) at org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.sendMessage(ConnectionHandler.java:383) at org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.run(ConnectionHandler.java:363) at java.lang.Thread.run(Thread.java:744) {code} Whereas on 192.168.36.173 (the new node) {code} INFO [main] 2014-03-12 13:45:24,140 StorageService.java (line 985) JOINING: Starting to bootstrap... INFO [main] 2014-03-12 13:45:32,319 StreamResultFuture.java (line 87) [Stream #2e0c6a10-a9e4-11e3-9c59-5340fa68c75f] Executing streaming plan for Bootstrap INFO [main] 2014-03-12 13:45:32,319 StreamResultFuture.java (line 91) [Stream #2e0c6a10-a9e4-11e3-9c59-5340fa68c75f] Beginning stream session with /192.168.36.172 INFO [main] 2014-03-12 13:45:32,319 StreamResultFuture.java (line 91) [Stream #2e0c6a10-a9e4-11e3-9c59-5340fa68c75f] Beginning stream session with /192.168.36.171 INFO [STREAM-IN-/192.168.36.172] 2014-03-12 13:45:32,370 StreamResultFuture.java (line 173) [Stream #2e0c6a10-a9e4-11e3-9c59-5340fa68c75f] Prepare completed. Receiving 1 files(64 bytes), send ing 0 files(0 bytes) [...] WARN [STREAM-IN-/192.168.36.171] 2014-03-12 13:50:04,882 StreamSession.java (line 532) [Stream #2e0c6a10-a9e4-11e3-9c59-5340fa68c75f] Retrying for following error java.lang.NullPointerException at org.apache.cassandra.streaming.compress.CompressedStreamReader.read(CompressedStreamReader.java:63) at org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:47) at org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:37) at org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:55) at org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:283) at java.lang.Thread.run(Thread.java:744) ERROR [STREAM-IN-/192.168.36.171] 2014-03-12 13:50:04,944 StreamSession.java (line 418) [Stream #2e0c6a10-a9e4-11e3-9c59-5340fa68c75f] Streaming error occurred java.lang.IllegalArgumentException: Unknown type -128 at org.apache.cassandra.streaming.messages.StreamMessage$Type.get(StreamMessage.java:89) at org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:54) at org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:283) at java.lang.Thread.run(Thread.java:744) INFO [STREAM-IN-/192.168.36.171] 2014-03-12 13:50:04,946 StreamResultFuture.java (line 186) [Stream #2e0c6a10-a9e4-11e3-9c59-5340fa68c75f] Session with /192.168.36.171 is complete WARN [STREAM-IN-/192.168.36.171] 2014-03-12 13:50:04,947 StreamResultFuture.java (line 215) [Stream #2e0c6a10-a9e4-11e3-9c59-5340fa68c75f] Stream failed ERROR [main] 2014-03-12 13:50:04,947 CassandraDaemon.java (line 488) Exception encountered during startup java.lang.RuntimeException: Error during boostrap: Stream failed at org.apache.cassandra.dht.BootStrapper.bootstrap(BootStrapper.java:86) at org.apache.cassandra.service.StorageService.bootstrap(StorageService.java:1013) at org.apache.cassandra.service.StorageService.joinTokenRing(StorageService.java:775) at org.apache.cassandra.service.StorageService.initServer(StorageService.java:610) at org.apache.cassandra.service.StorageService.initServer(StorageService.java:501) at org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:353) at org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:471) at org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:560) Caused by: org.apache.cassandra.streaming.StreamException: Stream failed at org.apache.cassandra.streaming.management.StreamEventJMXNotifier.onFailure(StreamEventJMXNotifier.java:85) at com.google.common.util.concurrent.Futures$4.run(Futures.java:1160) at com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297) at com.google.common.util.concurrent.ExecutionList.executeListener(ExecutionList.java:156) at com.google.common.util.concurrent.ExecutionList.execute(ExecutionList.java:145) at com.google.common.util.concurrent.AbstractFuture.setException(AbstractFuture.java:202) at org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamResultFuture.java:216) at org.apache.cassandra.streaming.StreamResultFuture.handleSessionComplete(StreamResultFuture.java:191) at org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:329) at org.apache.cassandra.streaming.StreamSession.onError(StreamSession.java:423) at org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:300) at java.lang.Thread.run(Thread.java:744) INFO [StorageServiceShutdownHook] 2014-03-12 13:50:04,955 Gossiper.java (line 1251) Announcing shutdown INFO [StorageServiceShutdownHook] 2014-03-12 13:50:06,955 MessagingService.java (line 667) Waiting for messaging service to quiesce INFO [ACCEPT-/192.168.36.173] 2014-03-12 13:50:06,956 MessagingService.java (line 907) MessagingService has terminated the accept() thread {code} Thanks. > New node refuses to join the ring. > ---------------------------------- > > Key: CASSANDRA-6565 > URL: https://issues.apache.org/jira/browse/CASSANDRA-6565 > Project: Cassandra > Issue Type: Bug > Reporter: Shao-Chuan Wang > > We have 30 nodes in one DC, 25 nodes in another. We are running 2.0.1. > Two nodes are joining the ring, but one of them failed > ARN [STREAM-IN-/10.4.197.53] 2014-01-09 19:41:40,418 StreamResultFuture.java > (line 209) [Stream #e515d6e0-795d-11e3-b74a-b72892248056] Stream failed > ERROR [main] 2014-01-09 19:41:40,418 CassandraDaemon.java (line 459) > Exception encountered during startup > java.lang.RuntimeException: Error during boostrap: Stream failed > at > org.apache.cassandra.dht.BootStrapper.bootstrap(BootStrapper.java:86) > at > org.apache.cassandra.service.StorageService.bootstrap(StorageService.java:901) > at > org.apache.cassandra.service.StorageService.joinTokenRing(StorageService.java:670) > at > org.apache.cassandra.service.StorageService.initServer(StorageService.java:529) > at > org.apache.cassandra.service.StorageService.initServer(StorageService.java:428) > at > org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:343) > at > org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:442) > at > org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:485) > Caused by: org.apache.cassandra.streaming.StreamException: Stream failed > at > org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamResultFuture.java:210) > at > org.apache.cassandra.streaming.StreamResultFuture.handleSessionComplete(StreamResultFuture.java:185) > at > org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:321) > at > org.apache.cassandra.streaming.StreamSession.complete(StreamSession.java:501) > at org.apache.cassandra.streaming.StreamSession.messageReceived(Stre > amSession.java:376) at > org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:293) > at java.lang.Thread.run(Thread.java:744) > ERROR [StorageServiceShutdownHook] 2014-01-09 19:41:40,428 > CassandraDaemon.java (line 185) Exception in thread > Thread[StorageServiceShutdownHook,5,main] > java.lang.NullPointerException > at > org.apache.cassandra.service.StorageService.stopRPCServer(StorageService.java:312) > at > org.apache.cassandra.service.StorageService.shutdownClientServers(StorageService.java:361) > at > org.apache.cassandra.service.StorageService.access$000(StorageService.java:96) > at > org.apache.cassandra.service.StorageService$1.runMayThrow(StorageService.java:494) > at > org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) > at java.lang.Thread.run(Thread.java:744) -- This message was sent by Atlassian JIRA (v6.2#6252)