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

Jack Money commented on CASSANDRA-10961:
----------------------------------------

I also have problem with add node to cluster in 2.2.4

HostB joining cluster and immediate get error
{code}

ERROR [STREAM-IN-/HostA] 2016-01-03 00:21:04,741 StreamSession.java:524 - 
[Stream #d5c419e0-b1af-11e5-9c1e-b5506ad414a4] Streaming error occurred
java.nio.channels.ClosedChannelException: null
        at 
sun.nio.ch.SocketChannelImpl.ensureReadOpen(SocketChannelImpl.java:257) 
~[na:1.8.0_65]
        at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:300) 
~[na:1.8.0_65]
        at 
org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:53)
 ~[apache-cassandra-2.2.4.jar:2.2.4]
        at 
org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:261)
 ~[apache-cassandra-2.2.4.jar:2.2.4]
        at java.lang.Thread.run(Thread.java:745) [na:1.8.0_65]
DEBUG [STREAM-OUT-/HostA] 2016-01-03 00:21:04,741 ConnectionHandler.java:334 - 
[Stream #d5c419e0-b1af-11e5-9c1e-b5506ad414a4] Sending Session Failed
ERROR [Thread-5] 2016-01-03 00:21:04,741 CassandraDaemon.java:185 - Exception 
in thread Thread[Thread-5,5,main]
java.lang.RuntimeException: java.lang.InterruptedException
        at com.google.common.base.Throwables.propagate(Throwables.java:160) 
~[guava-16.0.jar:na]
        at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32) 
~[apache-cassandra-2.2.4.jar:2.2.4]
        at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_65]
Caused by: java.lang.InterruptedException: null
        at 
java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireInterruptibly(AbstractQueuedSynchronizer.java:1220)
 ~[na:1.8.0_65]
        at 
java.util.concurrent.locks.ReentrantLock.lockInterruptibly(ReentrantLock.java:335)
 ~[na:1.8.0_65]
        at 
java.util.concurrent.ArrayBlockingQueue.put(ArrayBlockingQueue.java:350) 
~[na:1.8.0_65]
        at 
org.apache.cassandra.streaming.compress.CompressedInputStream$Reader.runMayThrow(CompressedInputStream.java:176)
 ~[apache-cassandra-2.2.4.jar:2.2.4]
        at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) 
~[apache-cassandra-2.2.4.jar:2.2.4]
        ... 1 common frames omitted
DEBUG [STREAM-IN-/HostA] 2016-01-03 00:21:04,777 ConnectionHandler.java:110 - 
[Stream #d5c419e0-b1af-11e5-9c1e-b5506ad414a4] Closing stream connection 
handler on /HostA
INFO  [STREAM-IN-/HostA] 2016-01-03 00:21:04,777 StreamResultFuture.java:182 - 
[Stream #d5c419e0-b1af-11e5-9c1e-b5506ad414a4] Session with /HostA is complete
WARN  [STREAM-IN-/HostA] 2016-01-03 00:21:04,778 StreamResultFuture.java:209 - 
[Stream #d5c419e0-b1af-11e5-9c1e-b5506ad414a4] Stream failed
ERROR [main] 2016-01-03 00:21:04,780 StorageService.java:1245 - Error while 
waiting on bootstrap to complete. Bootstrap will have to be restarted.
java.util.concurrent.ExecutionException: 
org.apache.cassandra.streaming.StreamException: Stream failed
        at 
com.google.common.util.concurrent.AbstractFuture$Sync.getValue(AbstractFuture.java:299)
 ~[guava-16.0.jar:na]
        at 
com.google.common.util.concurrent.AbstractFuture$Sync.get(AbstractFuture.java:286)
 ~[guava-16.0.jar:na]
        at 
com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:116) 
~[guava-16.0.jar:na]
        at 
org.apache.cassandra.service.StorageService.bootstrap(StorageService.java:1240) 
[apache-cassandra-2.2.4.jar:2.2.4]
        at 
org.apache.cassandra.service.StorageService.joinTokenRing(StorageService.java:920)
 [apache-cassandra-2.2.4.jar:2.2.4]
        at 
org.apache.cassandra.service.StorageService.initServer(StorageService.java:705) 
[apache-cassandra-2.2.4.jar:2.2.4]
        at 
org.apache.cassandra.service.StorageService.initServer(StorageService.java:581) 
[apache-cassandra-2.2.4.jar:2.2.4]
        at 
org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:315) 
[apache-cassandra-2.2.4.jar:2.2.4]
        at 
org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:529) 
[apache-cassandra-2.2.4.jar:2.2.4]
        at 
org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:638) 
[apache-cassandra-2.2.4.jar:2.2.4]
Caused by: org.apache.cassandra.streaming.StreamException: Stream failed
        at 
org.apache.cassandra.streaming.management.StreamEventJMXNotifier.onFailure(StreamEventJMXNotifier.java:85)
 ~[apache-cassandra-2.2.4.jar:2.2.4]
        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:210)
 ~[apache-cassandra-2.2.4.jar:2.2.4]
        at 
org.apache.cassandra.streaming.StreamResultFuture.handleSessionComplete(StreamResultFuture.java:186)
 ~[apache-cassandra-2.2.4.jar:2.2.4]
        at 
org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:434)
 ~[apache-cassandra-2.2.4.jar:2.2.4]
        at 
org.apache.cassandra.streaming.StreamSession.onError(StreamSession.java:529) 
~[apache-cassandra-2.2.4.jar:2.2.4]
        at 
org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:279)
 ~[apache-cassandra-2.2.4.jar:2.2.4]
        at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_65]
WARN  [main] 2016-01-03 00:21:04,785 StorageService.java:975 - Some data 
streaming failed. Use nodetool to check bootstrap state and resume. For more, 
see `nodetool help bootstrap`. IN_PROGRESS
{code}

Error on hostA

{code}
ERROR [STREAM-OUT-/HostB] 2016-01-03 00:21:05,085 StreamSession.java:524 - 
[Stream #d5c419e0-b1af-11e5-9c1e-b5506ad414a4] Streaming error occurred
org.apache.cassandra.io.FSReadError: java.io.IOException: Broken pipe
        at 
org.apache.cassandra.io.util.ChannelProxy.transferTo(ChannelProxy.java:144) 
~[apache-cassandra-2.2.4.jar:2.2.4]
        at 
org.apache.cassandra.streaming.compress.CompressedStreamWriter$1.apply(CompressedStreamWriter.java:79)
 ~[apache-cassandra-2.2.4.jar:2.2.4]
        at 
org.apache.cassandra.streaming.compress.CompressedStreamWriter$1.apply(CompressedStreamWriter.java:76)
 ~[apache-cassandra-2.2.4.jar:2.2.4]
        at 
org.apache.cassandra.io.util.BufferedDataOutputStreamPlus.applyToChannel(BufferedDataOutputStreamPlus.java:297)
 ~[apache-cassandra-2.2.4.jar:2.2.4]
        at 
org.apache.cassandra.streaming.compress.CompressedStreamWriter.write(CompressedStreamWriter.java:75)
 ~[apache-cassandra-2.2.4.jar:2.2.4]
        at 
org.apache.cassandra.streaming.messages.OutgoingFileMessage.serialize(OutgoingFileMessage.java:90)
 ~[apache-cassandra-2.2.4.jar:2.2.4]
        at 
org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:48)
 ~[apache-cassandra-2.2.4.jar:2.2.4]
        at 
org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:40)
 ~[apache-cassandra-2.2.4.jar:2.2.4]
        at 
org.apache.cassandra.streaming.messages.StreamMessage.serialize(StreamMessage.java:47)
 ~[apache-cassandra-2.2.4.jar:2.2.4]
        at 
org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.sendMessage(ConnectionHandler.java:363)
 ~[apache-cassandra-2.2.4.jar:2.2.4]
        at 
org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.run(ConnectionHandler.java:335)
 ~[apache-cassandra-2.2.4.jar:2.2.4]
        at java.lang.Thread.run(Thread.java:745) [na:1.8.0_65]
Caused by: java.io.IOException: Broken pipe
        at sun.nio.ch.FileChannelImpl.transferTo0(Native Method) ~[na:1.8.0_65]
        at 
sun.nio.ch.FileChannelImpl.transferToDirectlyInternal(FileChannelImpl.java:427) 
~[na:1.8.0_65]
        at 
sun.nio.ch.FileChannelImpl.transferToDirectly(FileChannelImpl.java:492) 
~[na:1.8.0_65]
        at sun.nio.ch.FileChannelImpl.transferTo(FileChannelImpl.java:607) 
~[na:1.8.0_65]
        at 
org.apache.cassandra.io.util.ChannelProxy.transferTo(ChannelProxy.java:140) 
~[apache-cassandra-2.2.4.jar:2.2.4]
        ... 11 common frames omitted
INFO  [STREAM-IN-/HostB] 2016-01-03 00:21:05,085 StreamResultFuture.java:182 - 
[Stream #d5c419e0-b1af-11e5-9c1e-b5506ad414a4] Session with /HostB is complete
WARN  [STREAM-IN-/HostB] 2016-01-03 00:21:05,086 StreamResultFuture.java:209 - 
[Stream #d5c419e0-b1af-11e5-9c1e-b5506ad414a4] Stream failed
{code}

On stackoverflow other people also report problems
http://stackoverflow.com/questions/34551975/broken-pipe-when-bootstrapping-or-repair-node
http://stackoverflow.com/questions/34555653/cassandra-bootstrap-fail-with-streaming-error-occurred


> Not enough bytes error when add nodes to cluster
> ------------------------------------------------
>
>                 Key: CASSANDRA-10961
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-10961
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Streaming and Messaging
>            Reporter: xiaost
>
> we got the same problem all the time when we add nodes to cluster.
> netstats:
> on HostA
> {noformat}
> /la-38395-big-Data.db 14792091851/14792091851 bytes(100%) sent to idx:0/HostB
> {noformat}
> on HostB
> {noformat}
> tmp-la-4-big-Data.db 2667087450/14792091851 bytes(18%) received from 
> idx:0/HostA
> {noformat}
> After a while, Error on HostB
> {noformat}
> WARN  [STREAM-IN-/HostA] 2016-01-02 12:08:14,737 StreamSession.java:644 - 
> [Stream #b91a4e90-b105-11e5-bd57-dd0cc3b4634c] Retrying for following error
> java.lang.IllegalArgumentException: Not enough bytes
>         at 
> org.apache.cassandra.db.composites.AbstractCType.checkRemaining(AbstractCType.java:362)
>  ~[apache-cassandra-2.2.4.jar:2.2.4]
>         at 
> org.apache.cassandra.db.composites.AbstractCompoundCellNameType.fromByteBuffer(AbstractCompoundCellNameType.java:98)
>  ~[apache-cassandra-2.2.4.jar:2.2.4]
>         at 
> org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:381)
>  ~[apache-cassandra-2.2.4.jar:2.2.4]
>         at 
> org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:365)
>  ~[apache-cassandra-2.2.4.jar:2.2.4]
>         at 
> org.apache.cassandra.db.OnDiskAtom$Serializer.deserializeFromSSTable(OnDiskAtom.java:75)
>  ~[apache-cassandra-2.2.4.jar:2.2.4]
>         at 
> org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:52) 
> ~[apache-cassandra-2.2.4.jar:2.2.4]
>         at 
> org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:46) 
> ~[apache-cassandra-2.2.4.jar:2.2.4]
>         at 
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
>  ~[guava-16.0.jar:na]
>         at 
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138) 
> ~[guava-16.0.jar:na]
>         at 
> org.apache.cassandra.io.sstable.format.big.BigTableWriter.appendFromStream(BigTableWriter.java:243)
>  ~[apache-cassandra-2.2.4.jar:2.2.4]
>         at 
> org.apache.cassandra.streaming.StreamReader.writeRow(StreamReader.java:173) 
> ~[apache-cassandra-2.2.4.jar:2.2.4]
>         at 
> org.apache.cassandra.streaming.compress.CompressedStreamReader.read(CompressedStreamReader.java:95)
>  ~[apache-cassandra-2.2.4.jar:2.2.4]
>         at 
> org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:49)
>  [apache-cassandra-2.2.4.jar:2.2.4]
>         at 
> org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:38)
>  [apache-cassandra-2.2.4.jar:2.2.4]
>         at 
> org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:58)
>  [apache-cassandra-2.2.4.jar:2.2.4]
>         at 
> org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:261)
>  [apache-cassandra-2.2.4.jar:2.2.4]
>         at java.lang.Thread.run(Thread.java:745) [na:1.8.0_66-internal]
> ERROR [Thread-28] 2016-01-02 12:08:14,737 CassandraDaemon.java:185 - 
> Exception in thread Thread[Thread-28,5,main]
> java.lang.RuntimeException: java.lang.InterruptedException
>         at com.google.common.base.Throwables.propagate(Throwables.java:160) 
> ~[guava-16.0.jar:na]
>         at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32) 
> ~[apache-cassandra-2.2.4.jar:2.2.4]
>         at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_66-internal]
> Caused by: java.lang.InterruptedException: null
>         at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireInterruptibly(AbstractQueuedSynchronizer.java:1220)
>  ~[na:1.8.0_66-internal]
>         at 
> java.util.concurrent.locks.ReentrantLock.lockInterruptibly(ReentrantLock.java:335)
>  ~[na:1.8.0_66-internal]
>         at 
> java.util.concurrent.ArrayBlockingQueue.put(ArrayBlockingQueue.java:350) 
> ~[na:1.8.0_66-internal]
>         at 
> org.apache.cassandra.streaming.compress.CompressedInputStream$Reader.runMayThrow(CompressedInputStream.java:176)
>  ~[apache-cassandra-2.2.4.jar:2.2.4]
>         at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) 
> ~[apache-cassandra-2.2.4.jar:2.2.4]
>         ... 1 common frames omitted
> ERROR [STREAM-IN-/HostA] 2016-01-02 12:08:14,737 StreamSession.java:524 - 
> [Stream #b91a4e90-b105-11e5-bd57-dd0cc3b4634c] Streaming error occurred
> java.nio.channels.ClosedChannelException: null
>         at 
> sun.nio.ch.SocketChannelImpl.ensureReadOpen(SocketChannelImpl.java:257) 
> ~[na:1.8.0_66-internal]
>         at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:300) 
> ~[na:1.8.0_66-internal]
>         at 
> org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:53)
>  ~[apache-cassandra-2.2.4.jar:2.2.4]
>         at 
> org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:261)
>  ~[apache-cassandra-2.2.4.jar:2.2.4]
>         at java.lang.Thread.run(Thread.java:745) [na:1.8.0_66-internal]
> INFO  [STREAM-IN-/HostA] 2016-01-02 12:08:15,875 StreamResultFuture.java:182 
> - [Stream #b91a4e90-b105-11e5-bd57-dd0cc3b4634c] Session with /HostA is 
> complete
> {noformat}
> Errors on HostA
> {noformat}
> WARN  [STREAM-IN-/HostB] 2016-01-02 12:08:14,763 StreamResultFuture.java:209 
> - [Stream #b91a4e90-b105-11e5-bd57-dd0cc3b4634c] Stream failed
> ERROR [STREAM-OUT-/HostB] 2016-01-02 12:08:14,777 StreamSession.java:524 - 
> [Stream #b91a4e90-b105-11e5-bd57-dd0cc3b4634c] Streaming error occurred
> org.apache.cassandra.io.FSReadError: java.io.IOException: Broken pipe
>         at 
> org.apache.cassandra.io.util.ChannelProxy.transferTo(ChannelProxy.java:144) 
> ~[apache-cassandra-2.2.4.jar:2.2.4]
>         at 
> org.apache.cassandra.streaming.compress.CompressedStreamWriter$1.apply(CompressedStreamWriter.java:79)
>  ~[apache-cassandra-2.2.4.jar:2.2.4]
>         at 
> org.apache.cassandra.streaming.compress.CompressedStreamWriter$1.apply(CompressedStreamWriter.java:76)
>  ~[apache-cassandra-2.2.4.jar:2.2.4]
>         at 
> org.apache.cassandra.io.util.BufferedDataOutputStreamPlus.applyToChannel(BufferedDataOutputStreamPlus.java:297)
>  ~[apache-cassandra-2.2.4.jar:2.2.4]
>         at 
> org.apache.cassandra.streaming.compress.CompressedStreamWriter.write(CompressedStreamWriter.java:75)
>  ~[apache-cassandra-2.2.4.jar:2.2.4]
>         at 
> org.apache.cassandra.streaming.messages.OutgoingFileMessage.serialize(OutgoingFileMessage.java:90)
>  ~[apache-cassandra-2.2.4.jar:2.2.4]
>         at 
> org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:48)
>  ~[apache-cassandra-2.2.4.jar:2.2.4]
>         at 
> org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:40)
>  ~[apache-cassandra-2.2.4.jar:2.2.4]
>         at 
> org.apache.cassandra.streaming.messages.StreamMessage.serialize(StreamMessage.java:47)
>  ~[apache-cassandra-2.2.4.jar:2.2.4]
>         at 
> org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.sendMessage(ConnectionHandler.java:363)
>  ~[apache-cassandra-2.2.4.jar:2.2.4]
>         at 
> org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.run(ConnectionHandler.java:335)
>  ~[apache-cassandra-2.2.4.jar:2.2.4]
>         at java.lang.Thread.run(Thread.java:745) [na:1.8.0_66-internal]
> Caused by: java.io.IOException: Broken pipe
>         at sun.nio.ch.FileChannelImpl.transferTo0(Native Method) 
> ~[na:1.8.0_66-internal]
>         at 
> sun.nio.ch.FileChannelImpl.transferToDirectlyInternal(FileChannelImpl.java:427)
>  ~[na:1.8.0_66-internal]
>         at 
> sun.nio.ch.FileChannelImpl.transferToDirectly(FileChannelImpl.java:492) 
> ~[na:1.8.0_66-internal]
>         at sun.nio.ch.FileChannelImpl.transferTo(FileChannelImpl.java:607) 
> ~[na:1.8.0_66-internal]
>         at 
> org.apache.cassandra.io.util.ChannelProxy.transferTo(ChannelProxy.java:140) 
> ~[apache-cassandra-2.2.4.jar:2.2.4]
>         ... 11 common frames omitted
> {noformat}



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

Reply via email to