[ 
https://issues.apache.org/jira/browse/CASSANDRA-5418?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sylvain Lebresne updated CASSANDRA-5418:
----------------------------------------

    Attachment: 5418-v4.txt

I agree on the source of the problem. On the patch however, since the goal 
should be to write only what we get from the stream (since we've used the 
dataSize from the stream), it would feel more natural to me to just skip 
tombstoneTracker.writeOpenedMarker (in which case we really can skip the 
tombstone tracker completely and save a few CPU cycles). I'm attaching a v5 
patch that implement this (imo simpler) alternative.

Now as was noted above, this fix (whatever version of the patch we use) has the 
small downside that if the source and destination don't have the same 
column_index_size_in_kb, we'll be screwed. This is definitively a much less 
problem that this issue and so we should still fix this, but for 2.0, once 
CASSANRA-4180 gets in, then we should more or less revert this fix because it 
won't be necessary anymore. I've create CASSANRA-5454 so we don't forget about 
it.

                
> streaming fails
> ---------------
>
>                 Key: CASSANDRA-5418
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-5418
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>    Affects Versions: 1.2.2, 1.2.3
>         Environment: 5 nodes, vnodes enabled, encryption disabled, 
> compression enabled, RackInferring snitch, Centos 6, Oracle JVM with JNA 
> enabled.
>            Reporter: Igor Ivanov
>            Priority: Critical
>         Attachments: 5418-1.2.txt, 5418-1.2-v2.txt, 5418-1.2-v3.txt, 
> 5418-v4.txt
>
>
> When I run *nodetool repair* on cas01 node it get's stuck at some point.
> I see following exceptions in cas01 system.log:
> {quote}
> ERROR [Streaming to /10.10.45.60:28] 2013-04-02 09:03:55,353 
> CassandraDaemon.java (line 132) Exception in thread Thread[Streaming to 
> /10.10.45.60:28,5,main]
> java.lang.RuntimeException: java.io.EOFException
>       at com.google.common.base.Throwables.propagate(Throwables.java:160)
>       at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32)
>       at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown 
> Source)
>       at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
>       at java.lang.Thread.run(Unknown Source)
> Caused by: java.io.EOFException
>       at java.io.DataInputStream.readInt(Unknown Source)
>       at 
> org.apache.cassandra.streaming.FileStreamTask.receiveReply(FileStreamTask.java:193)
>       at 
> org.apache.cassandra.streaming.compress.CompressedFileStreamTask.stream(CompressedFileStreamTask.java:114)
>       at 
> org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91)
>       at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>       ... 3 more
> ERROR [Thread-2076] 2013-04-02 09:07:12,261 CassandraDaemon.java (line 132) 
> Exception in thread Thread[Thread-2076,5,main]
> java.lang.AssertionError: incorrect row data size 130921 written to 
> /var/lib/cassandra/data/EDITED/content_list/footballsite-content_list-tmp-ib-3660-Data.db;
>  correct is 131074
>       at 
> org.apache.cassandra.io.sstable.SSTableWriter.appendFromStream(SSTableWriter.java:285)
>       at 
> org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:179)
>       at 
> org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:122)
>       at 
> org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:238)
>       at 
> org.apache.cassandra.net.IncomingTcpConnection.handleStream(IncomingTcpConnection.java:178)
>       at 
> org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:78)
> {quote}
> On other machines there are some exceptions too:
> {quote}
> ERROR [Thread-1424] 2013-04-02 09:07:12,248 CassandraDaemon.java (line 132) 
> Exception in thread Thread[Thread-1424,5,main]
> java.lang.AssertionError: incorrect row data size 130921 written to 
> /var/lib/cassandra/data/EDITED/content_list/footballsite-content_list-tmp-ib-2268-Data.db;
>  correct is 131074
>       at 
> org.apache.cassandra.io.sstable.SSTableWriter.appendFromStream(SSTableWriter.java:285)
>       at 
> org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:179)
>       at 
> org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:122)
>       at 
> org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:238)
>       at 
> org.apache.cassandra.net.IncomingTcpConnection.handleStream(IncomingTcpConnection.java:178)
>       at 
> org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:78)
> ERROR [Streaming to /10.10.45.58:55] 2013-04-02 09:07:12,263 
> CassandraDaemon.java (line 132) Exception in thread Thread[Streaming to 
> /10.10.45.58:55,5,main]
> java.lang.RuntimeException: java.io.EOFException
>       at com.google.common.base.Throwables.propagate(Throwables.java:160)
>       at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32)
>       at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown 
> Source)
>       at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
>       at java.lang.Thread.run(Unknown Source)
> Caused by: java.io.EOFException
>       at java.io.DataInputStream.readInt(Unknown Source)
>       at 
> org.apache.cassandra.streaming.FileStreamTask.receiveReply(FileStreamTask.java:193)
>       at 
> org.apache.cassandra.streaming.compress.CompressedFileStreamTask.stream(CompressedFileStreamTask.java:114)
>       at 
> org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91)
>       at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>       ... 3 more
> {quote}
> Then I see frozen status in *nodetool netstats* and repair never completes.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

Reply via email to