cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Paulo Motta (JIRA)" <j...@apache.org>
Subject [jira] [Comment Edited] (CASSANDRA-10961) Not enough bytes error when add nodes to cluster
Date Thu, 07 Jan 2016 20:26:40 GMT

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

Paulo Motta edited comment on CASSANDRA-10961 at 1/7/16 8:25 PM:
-----------------------------------------------------------------

I might have found the culprit on {{CompressedStreamReader}}, in this snippet:
{noformat}
int sectionLength = (int) (section.right - section.left);
{noformat}

This might cause sections larger than 2.14GB to have negative length due to {{int}} overflow
and be skipped, what may break deserealization of next sections. I changed to use {{long}}
type instead.

I also reverted CASSANDRA-10005, since interrupting a thread waiting on a {{SocketChannel}}
will cause the channel to be closed what will make retry fail.

Furthermore I added more debugging statements, so in case this is not the problem we will
have more elements to investigate further. If that's the case, please attach new debug logs
in the next run.

[~xiaost] I attached a new [jar|https://issues.apache.org/jira/secure/attachment/12781053/apache-cassandra-2.2.4-SNAPSHOT.jar]
that you can test, and in case you want to build your own you can do it from this [branch|https://github.com/pauloricardomg/cassandra/tree/2.2-10961]


was (Author: pauloricardomg):
I might have found the culprit on {{CompressedStreamReader}}, in this snippet:
{noformat}
int sectionLength = (int) (section.right - section.left);
{noformat}

This might cause sections larger than 2.14GB to have negative length and be skipped, what
may break deserealization of next sections.

I also reverted CASSANDRA-10005, since interrupting a thread waiting on a {{SocketChannel}}
will cause the channel to be closed what will make retry fail.

Furthermore I added more debugging statements, so in case this is not the problem we will
have more elements to investigate further. If that's the case, please attach new debug logs
in the next run.

[~xiaost] I attached a new [jar|https://issues.apache.org/jira/secure/attachment/12781053/apache-cassandra-2.2.4-SNAPSHOT.jar]
that you can test, and in case you want to build your own you can do it from this [branch|https://github.com/pauloricardomg/cassandra/tree/2.2-10961]

> 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
>            Assignee: Paulo Motta
>         Attachments: apache-cassandra-2.2.4-SNAPSHOT.jar, debug.1.log, debug.logs.zip,
netstats.1.log
>
>
> 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)

Mime
View raw message