You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Eric Falcao (Created) (JIRA)" <ji...@apache.org> on 2011/11/09 22:33:52 UTC

[jira] [Created] (CASSANDRA-3481) During repair, "incorrect data size" & "Connection reset" errors. Repair unable to complete.

During repair, "incorrect data size" & "Connection reset" errors. Repair unable to complete.
--------------------------------------------------------------------------------------------

                 Key: CASSANDRA-3481
                 URL: https://issues.apache.org/jira/browse/CASSANDRA-3481
             Project: Cassandra
          Issue Type: Bug
    Affects Versions: 1.0.2
            Reporter: Eric Falcao


This has been happening since 1.0.2. I wasn't on 1.0 for very long but I'm fairly certain repair was working ok. Repair worked decently for me in 0.8 (data bloat sucked). All my SSTables are version h.

On one node:

java.lang.AssertionError: incorrect row data size 596045 written to /mnt/cassandra/data/TRProd/Metrics1m-tmp-h-25036-Data.db; correct is 586675
	at org.apache.cassandra.io.sstable.SSTableWriter.appendFromStream(SSTableWriter.java:253)
	at org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:146)
	at org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:87)
	at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:184)
	at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:81)

On the other node:

4999 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-24953-Data.db sections=1707 progress=0/1513497639 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25000-Data.db sections=635 progress=0/53400713 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25002-Data.db sections=570 progress=0/709993 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25003-Data.db sections=550 progress=0/449498 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25005-Data.db sections=516 progress=0/316301 - 0%], 6 sstables.
 INFO [StreamStage:1] 2011-11-09 19:45:22,795 StreamOutSession.java (line 203) Streaming to /10.38.69.192
ERROR [Streaming:1] 2011-11-09 19:47:47,964 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[Streaming:1,1,main]
java.lang.RuntimeException: java.net.SocketException: Connection reset
	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
	at java.lang.Thread.run(Thread.java:619)
Caused by: java.net.SocketException: Connection reset
	at java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:96)
	at java.net.SocketOutputStream.write(SocketOutputStream.java:136)
	at com.ning.compress.lzf.ChunkEncoder.encodeAndWriteChunk(ChunkEncoder.java:133)
	at com.ning.compress.lzf.LZFOutputStream.writeCompressedBlock(LZFOutputStream.java:203)
	at com.ning.compress.lzf.LZFOutputStream.write(LZFOutputStream.java:97)
	at org.apache.cassandra.streaming.FileStreamTask.write(FileStreamTask.java:181)
	at org.apache.cassandra.streaming.FileStreamTask.stream(FileStreamTask.java:145)
	at org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91)
	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
	... 3 more
ERROR [Streaming:1] 2011-11-09 19:47:47,970 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[Streaming:1,1,main]
java.lang.RuntimeException: java.net.SocketException: Connection reset
	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
	at java.lang.Thread.run(Thread.java:619)
Caused by: java.net.SocketException: Connection reset
	at java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:96)
	at java.net.SocketOutputStream.write(SocketOutputStream.java:136)
	at com.ning.compress.lzf.ChunkEncoder.encodeAndWriteChunk(ChunkEncoder.java:133)
	at com.ning.compress.lzf.LZFOutputStream.writeCompressedBlock(LZFOutputStream.java:203)
	at com.ning.compress.lzf.LZFOutputStream.write(LZFOutputStream.java:97)
	at org.apache.cassandra.streaming.FileStreamTask.write(FileStreamTask.java:181)
	at org.apache.cassandra.streaming.FileStreamTask.stream(FileStreamTask.java:145)
	at org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91)
	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
	... 3 more


--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (CASSANDRA-3481) During repair, "incorrect data size" & "Connection reset" errors. Repair unable to complete.

Posted by "Sylvain Lebresne (Updated) (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-3481?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Sylvain Lebresne updated CASSANDRA-3481:
----------------------------------------

    Attachment: 3481.patch

We do have a problem with counters that leads to that exact problem.

During streaming, we must ensure that whatever we deserialize from the wire has the same size once reserialized. For counters, whenever we should have clean deltas, instead we just 'mark' the delta to be clean in a way that doesn't change the size and let them be clean later. Problem is, if you stream a file that still have those 'marked' delta, it will remove them, changing the size of the data and thus triggering the exception at end.

Fixing this require knowing at deserialization that we are in this 'preserve the size' mode. Attaching a patch for that. Instead of adding a new flag to deserialization, I replaced 'fromRemote' boolean by an enum flag. The patch includes a unit test.

                
> During repair, "incorrect data size" & "Connection reset" errors. Repair unable to complete.
> --------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-3481
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-3481
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: 1.0.2
>            Reporter: Eric Falcao
>            Assignee: Sylvain Lebresne
>              Labels: connection, repair
>             Fix For: 1.0.3
>
>         Attachments: 3481.patch
>
>
> This has been happening since 1.0.2. I wasn't on 1.0 for very long but I'm fairly certain repair was working ok. Repair worked decently for me in 0.8 (data bloat sucked). All my SSTables are version h.
> On one node:
> java.lang.AssertionError: incorrect row data size 596045 written to /mnt/cassandra/data/TRProd/Metrics1m-tmp-h-25036-Data.db; correct is 586675
> 	at org.apache.cassandra.io.sstable.SSTableWriter.appendFromStream(SSTableWriter.java:253)
> 	at org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:146)
> 	at org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:87)
> 	at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:184)
> 	at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:81)
> On the other node:
> 4999 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-24953-Data.db sections=1707 progress=0/1513497639 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25000-Data.db sections=635 progress=0/53400713 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25002-Data.db sections=570 progress=0/709993 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25003-Data.db sections=550 progress=0/449498 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25005-Data.db sections=516 progress=0/316301 - 0%], 6 sstables.
>  INFO [StreamStage:1] 2011-11-09 19:45:22,795 StreamOutSession.java (line 203) Streaming to /10.38.69.192
> ERROR [Streaming:1] 2011-11-09 19:47:47,964 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[Streaming:1,1,main]
> java.lang.RuntimeException: java.net.SocketException: Connection reset
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> 	at java.lang.Thread.run(Thread.java:619)
> Caused by: java.net.SocketException: Connection reset
> 	at java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:96)
> 	at java.net.SocketOutputStream.write(SocketOutputStream.java:136)
> 	at com.ning.compress.lzf.ChunkEncoder.encodeAndWriteChunk(ChunkEncoder.java:133)
> 	at com.ning.compress.lzf.LZFOutputStream.writeCompressedBlock(LZFOutputStream.java:203)
> 	at com.ning.compress.lzf.LZFOutputStream.write(LZFOutputStream.java:97)
> 	at org.apache.cassandra.streaming.FileStreamTask.write(FileStreamTask.java:181)
> 	at org.apache.cassandra.streaming.FileStreamTask.stream(FileStreamTask.java:145)
> 	at org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91)
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> 	... 3 more
> ERROR [Streaming:1] 2011-11-09 19:47:47,970 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[Streaming:1,1,main]
> java.lang.RuntimeException: java.net.SocketException: Connection reset
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> 	at java.lang.Thread.run(Thread.java:619)
> Caused by: java.net.SocketException: Connection reset
> 	at java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:96)
> 	at java.net.SocketOutputStream.write(SocketOutputStream.java:136)
> 	at com.ning.compress.lzf.ChunkEncoder.encodeAndWriteChunk(ChunkEncoder.java:133)
> 	at com.ning.compress.lzf.LZFOutputStream.writeCompressedBlock(LZFOutputStream.java:203)
> 	at com.ning.compress.lzf.LZFOutputStream.write(LZFOutputStream.java:97)
> 	at org.apache.cassandra.streaming.FileStreamTask.write(FileStreamTask.java:181)
> 	at org.apache.cassandra.streaming.FileStreamTask.stream(FileStreamTask.java:145)
> 	at org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91)
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> 	... 3 more

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (CASSANDRA-3481) During repair, "incorrect data size" & "Connection reset" errors. Repair unable to complete.

Posted by "Jonathan Ellis (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-3481?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13147888#comment-13147888 ] 

Jonathan Ellis commented on CASSANDRA-3481:
-------------------------------------------

+1
                
> During repair, "incorrect data size" & "Connection reset" errors. Repair unable to complete.
> --------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-3481
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-3481
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: 1.0.2
>            Reporter: Eric Falcao
>            Assignee: Sylvain Lebresne
>              Labels: connection, repair
>             Fix For: 1.0.3
>
>         Attachments: 3481-v2.patch, 3481.patch
>
>
> This has been happening since 1.0.2. I wasn't on 1.0 for very long but I'm fairly certain repair was working ok. Repair worked decently for me in 0.8 (data bloat sucked). All my SSTables are version h.
> On one node:
> java.lang.AssertionError: incorrect row data size 596045 written to /mnt/cassandra/data/TRProd/Metrics1m-tmp-h-25036-Data.db; correct is 586675
> 	at org.apache.cassandra.io.sstable.SSTableWriter.appendFromStream(SSTableWriter.java:253)
> 	at org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:146)
> 	at org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:87)
> 	at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:184)
> 	at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:81)
> On the other node:
> 4999 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-24953-Data.db sections=1707 progress=0/1513497639 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25000-Data.db sections=635 progress=0/53400713 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25002-Data.db sections=570 progress=0/709993 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25003-Data.db sections=550 progress=0/449498 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25005-Data.db sections=516 progress=0/316301 - 0%], 6 sstables.
>  INFO [StreamStage:1] 2011-11-09 19:45:22,795 StreamOutSession.java (line 203) Streaming to /10.38.69.192
> ERROR [Streaming:1] 2011-11-09 19:47:47,964 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[Streaming:1,1,main]
> java.lang.RuntimeException: java.net.SocketException: Connection reset
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> 	at java.lang.Thread.run(Thread.java:619)
> Caused by: java.net.SocketException: Connection reset
> 	at java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:96)
> 	at java.net.SocketOutputStream.write(SocketOutputStream.java:136)
> 	at com.ning.compress.lzf.ChunkEncoder.encodeAndWriteChunk(ChunkEncoder.java:133)
> 	at com.ning.compress.lzf.LZFOutputStream.writeCompressedBlock(LZFOutputStream.java:203)
> 	at com.ning.compress.lzf.LZFOutputStream.write(LZFOutputStream.java:97)
> 	at org.apache.cassandra.streaming.FileStreamTask.write(FileStreamTask.java:181)
> 	at org.apache.cassandra.streaming.FileStreamTask.stream(FileStreamTask.java:145)
> 	at org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91)
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> 	... 3 more
> ERROR [Streaming:1] 2011-11-09 19:47:47,970 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[Streaming:1,1,main]
> java.lang.RuntimeException: java.net.SocketException: Connection reset
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> 	at java.lang.Thread.run(Thread.java:619)
> Caused by: java.net.SocketException: Connection reset
> 	at java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:96)
> 	at java.net.SocketOutputStream.write(SocketOutputStream.java:136)
> 	at com.ning.compress.lzf.ChunkEncoder.encodeAndWriteChunk(ChunkEncoder.java:133)
> 	at com.ning.compress.lzf.LZFOutputStream.writeCompressedBlock(LZFOutputStream.java:203)
> 	at com.ning.compress.lzf.LZFOutputStream.write(LZFOutputStream.java:97)
> 	at org.apache.cassandra.streaming.FileStreamTask.write(FileStreamTask.java:181)
> 	at org.apache.cassandra.streaming.FileStreamTask.stream(FileStreamTask.java:145)
> 	at org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91)
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> 	... 3 more

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Resolved] (CASSANDRA-3481) During repair, "incorrect data size" & "Connection reset" errors. Repair unable to complete.

Posted by "Sylvain Lebresne (Resolved) (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-3481?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Sylvain Lebresne resolved CASSANDRA-3481.
-----------------------------------------

       Resolution: Fixed
    Fix Version/s:     (was: 1.0.4)
                   1.0.3
         Reviewer: jbellis

Forgot to close this one but it's been committed already.
                
> During repair, "incorrect data size" & "Connection reset" errors. Repair unable to complete.
> --------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-3481
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-3481
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: 1.0.2
>            Reporter: Eric Falcao
>            Assignee: Sylvain Lebresne
>              Labels: connection, repair
>             Fix For: 1.0.3
>
>         Attachments: 3481-v2.patch, 3481.patch
>
>
> This has been happening since 1.0.2. I wasn't on 1.0 for very long but I'm fairly certain repair was working ok. Repair worked decently for me in 0.8 (data bloat sucked). All my SSTables are version h.
> On one node:
> java.lang.AssertionError: incorrect row data size 596045 written to /mnt/cassandra/data/TRProd/Metrics1m-tmp-h-25036-Data.db; correct is 586675
> 	at org.apache.cassandra.io.sstable.SSTableWriter.appendFromStream(SSTableWriter.java:253)
> 	at org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:146)
> 	at org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:87)
> 	at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:184)
> 	at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:81)
> On the other node:
> 4999 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-24953-Data.db sections=1707 progress=0/1513497639 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25000-Data.db sections=635 progress=0/53400713 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25002-Data.db sections=570 progress=0/709993 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25003-Data.db sections=550 progress=0/449498 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25005-Data.db sections=516 progress=0/316301 - 0%], 6 sstables.
>  INFO [StreamStage:1] 2011-11-09 19:45:22,795 StreamOutSession.java (line 203) Streaming to /10.38.69.192
> ERROR [Streaming:1] 2011-11-09 19:47:47,964 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[Streaming:1,1,main]
> java.lang.RuntimeException: java.net.SocketException: Connection reset
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> 	at java.lang.Thread.run(Thread.java:619)
> Caused by: java.net.SocketException: Connection reset
> 	at java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:96)
> 	at java.net.SocketOutputStream.write(SocketOutputStream.java:136)
> 	at com.ning.compress.lzf.ChunkEncoder.encodeAndWriteChunk(ChunkEncoder.java:133)
> 	at com.ning.compress.lzf.LZFOutputStream.writeCompressedBlock(LZFOutputStream.java:203)
> 	at com.ning.compress.lzf.LZFOutputStream.write(LZFOutputStream.java:97)
> 	at org.apache.cassandra.streaming.FileStreamTask.write(FileStreamTask.java:181)
> 	at org.apache.cassandra.streaming.FileStreamTask.stream(FileStreamTask.java:145)
> 	at org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91)
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> 	... 3 more
> ERROR [Streaming:1] 2011-11-09 19:47:47,970 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[Streaming:1,1,main]
> java.lang.RuntimeException: java.net.SocketException: Connection reset
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> 	at java.lang.Thread.run(Thread.java:619)
> Caused by: java.net.SocketException: Connection reset
> 	at java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:96)
> 	at java.net.SocketOutputStream.write(SocketOutputStream.java:136)
> 	at com.ning.compress.lzf.ChunkEncoder.encodeAndWriteChunk(ChunkEncoder.java:133)
> 	at com.ning.compress.lzf.LZFOutputStream.writeCompressedBlock(LZFOutputStream.java:203)
> 	at com.ning.compress.lzf.LZFOutputStream.write(LZFOutputStream.java:97)
> 	at org.apache.cassandra.streaming.FileStreamTask.write(FileStreamTask.java:181)
> 	at org.apache.cassandra.streaming.FileStreamTask.stream(FileStreamTask.java:145)
> 	at org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91)
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> 	... 3 more

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (CASSANDRA-3481) During repair, "incorrect data size" & "Connection reset" errors. Repair unable to complete.

Posted by "Eric Falcao (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-3481?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13147763#comment-13147763 ] 

Eric Falcao commented on CASSANDRA-3481:
----------------------------------------

Metrics1m is a Counter CF. I originally tried CounterColumns with TTL's but was advised against it. They have no expiration.

Let me know if I can be any more help. I might be able to try the patch a bit later in the day.
                
> During repair, "incorrect data size" & "Connection reset" errors. Repair unable to complete.
> --------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-3481
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-3481
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: 1.0.2
>            Reporter: Eric Falcao
>            Assignee: Sylvain Lebresne
>              Labels: connection, repair
>             Fix For: 1.0.3
>
>         Attachments: 3481.patch
>
>
> This has been happening since 1.0.2. I wasn't on 1.0 for very long but I'm fairly certain repair was working ok. Repair worked decently for me in 0.8 (data bloat sucked). All my SSTables are version h.
> On one node:
> java.lang.AssertionError: incorrect row data size 596045 written to /mnt/cassandra/data/TRProd/Metrics1m-tmp-h-25036-Data.db; correct is 586675
> 	at org.apache.cassandra.io.sstable.SSTableWriter.appendFromStream(SSTableWriter.java:253)
> 	at org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:146)
> 	at org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:87)
> 	at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:184)
> 	at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:81)
> On the other node:
> 4999 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-24953-Data.db sections=1707 progress=0/1513497639 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25000-Data.db sections=635 progress=0/53400713 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25002-Data.db sections=570 progress=0/709993 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25003-Data.db sections=550 progress=0/449498 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25005-Data.db sections=516 progress=0/316301 - 0%], 6 sstables.
>  INFO [StreamStage:1] 2011-11-09 19:45:22,795 StreamOutSession.java (line 203) Streaming to /10.38.69.192
> ERROR [Streaming:1] 2011-11-09 19:47:47,964 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[Streaming:1,1,main]
> java.lang.RuntimeException: java.net.SocketException: Connection reset
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> 	at java.lang.Thread.run(Thread.java:619)
> Caused by: java.net.SocketException: Connection reset
> 	at java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:96)
> 	at java.net.SocketOutputStream.write(SocketOutputStream.java:136)
> 	at com.ning.compress.lzf.ChunkEncoder.encodeAndWriteChunk(ChunkEncoder.java:133)
> 	at com.ning.compress.lzf.LZFOutputStream.writeCompressedBlock(LZFOutputStream.java:203)
> 	at com.ning.compress.lzf.LZFOutputStream.write(LZFOutputStream.java:97)
> 	at org.apache.cassandra.streaming.FileStreamTask.write(FileStreamTask.java:181)
> 	at org.apache.cassandra.streaming.FileStreamTask.stream(FileStreamTask.java:145)
> 	at org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91)
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> 	... 3 more
> ERROR [Streaming:1] 2011-11-09 19:47:47,970 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[Streaming:1,1,main]
> java.lang.RuntimeException: java.net.SocketException: Connection reset
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> 	at java.lang.Thread.run(Thread.java:619)
> Caused by: java.net.SocketException: Connection reset
> 	at java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:96)
> 	at java.net.SocketOutputStream.write(SocketOutputStream.java:136)
> 	at com.ning.compress.lzf.ChunkEncoder.encodeAndWriteChunk(ChunkEncoder.java:133)
> 	at com.ning.compress.lzf.LZFOutputStream.writeCompressedBlock(LZFOutputStream.java:203)
> 	at com.ning.compress.lzf.LZFOutputStream.write(LZFOutputStream.java:97)
> 	at org.apache.cassandra.streaming.FileStreamTask.write(FileStreamTask.java:181)
> 	at org.apache.cassandra.streaming.FileStreamTask.stream(FileStreamTask.java:145)
> 	at org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91)
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> 	... 3 more

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (CASSANDRA-3481) During repair, "incorrect data size" & "Connection reset" errors. Repair unable to complete.

Posted by "Jonathan Ellis (Updated) (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-3481?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Jonathan Ellis updated CASSANDRA-3481:
--------------------------------------

    Fix Version/s: 1.0.3
         Assignee: Sylvain Lebresne
    
> During repair, "incorrect data size" & "Connection reset" errors. Repair unable to complete.
> --------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-3481
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-3481
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: 1.0.2
>            Reporter: Eric Falcao
>            Assignee: Sylvain Lebresne
>              Labels: connection, repair
>             Fix For: 1.0.3
>
>
> This has been happening since 1.0.2. I wasn't on 1.0 for very long but I'm fairly certain repair was working ok. Repair worked decently for me in 0.8 (data bloat sucked). All my SSTables are version h.
> On one node:
> java.lang.AssertionError: incorrect row data size 596045 written to /mnt/cassandra/data/TRProd/Metrics1m-tmp-h-25036-Data.db; correct is 586675
> 	at org.apache.cassandra.io.sstable.SSTableWriter.appendFromStream(SSTableWriter.java:253)
> 	at org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:146)
> 	at org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:87)
> 	at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:184)
> 	at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:81)
> On the other node:
> 4999 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-24953-Data.db sections=1707 progress=0/1513497639 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25000-Data.db sections=635 progress=0/53400713 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25002-Data.db sections=570 progress=0/709993 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25003-Data.db sections=550 progress=0/449498 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25005-Data.db sections=516 progress=0/316301 - 0%], 6 sstables.
>  INFO [StreamStage:1] 2011-11-09 19:45:22,795 StreamOutSession.java (line 203) Streaming to /10.38.69.192
> ERROR [Streaming:1] 2011-11-09 19:47:47,964 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[Streaming:1,1,main]
> java.lang.RuntimeException: java.net.SocketException: Connection reset
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> 	at java.lang.Thread.run(Thread.java:619)
> Caused by: java.net.SocketException: Connection reset
> 	at java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:96)
> 	at java.net.SocketOutputStream.write(SocketOutputStream.java:136)
> 	at com.ning.compress.lzf.ChunkEncoder.encodeAndWriteChunk(ChunkEncoder.java:133)
> 	at com.ning.compress.lzf.LZFOutputStream.writeCompressedBlock(LZFOutputStream.java:203)
> 	at com.ning.compress.lzf.LZFOutputStream.write(LZFOutputStream.java:97)
> 	at org.apache.cassandra.streaming.FileStreamTask.write(FileStreamTask.java:181)
> 	at org.apache.cassandra.streaming.FileStreamTask.stream(FileStreamTask.java:145)
> 	at org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91)
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> 	... 3 more
> ERROR [Streaming:1] 2011-11-09 19:47:47,970 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[Streaming:1,1,main]
> java.lang.RuntimeException: java.net.SocketException: Connection reset
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> 	at java.lang.Thread.run(Thread.java:619)
> Caused by: java.net.SocketException: Connection reset
> 	at java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:96)
> 	at java.net.SocketOutputStream.write(SocketOutputStream.java:136)
> 	at com.ning.compress.lzf.ChunkEncoder.encodeAndWriteChunk(ChunkEncoder.java:133)
> 	at com.ning.compress.lzf.LZFOutputStream.writeCompressedBlock(LZFOutputStream.java:203)
> 	at com.ning.compress.lzf.LZFOutputStream.write(LZFOutputStream.java:97)
> 	at org.apache.cassandra.streaming.FileStreamTask.write(FileStreamTask.java:181)
> 	at org.apache.cassandra.streaming.FileStreamTask.stream(FileStreamTask.java:145)
> 	at org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91)
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> 	... 3 more

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (CASSANDRA-3481) During repair, "incorrect data size" & "Connection reset" errors. Repair unable to complete.

Posted by "Jonathan Ellis (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-3481?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13147872#comment-13147872 ] 

Jonathan Ellis commented on CASSANDRA-3481:
-------------------------------------------

Shouldn't IncomingStreamReader use PRESERVE_SIZE instead of FROM_REMOTE?

What does the commented-out test do, and why is it commented out?

nit: this comment needs updating:

{noformat}
             // deserialize column with fromRemote false, in order to keep size of streamed column
{noformat}

                
> During repair, "incorrect data size" & "Connection reset" errors. Repair unable to complete.
> --------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-3481
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-3481
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: 1.0.2
>            Reporter: Eric Falcao
>            Assignee: Sylvain Lebresne
>              Labels: connection, repair
>             Fix For: 1.0.3
>
>         Attachments: 3481.patch
>
>
> This has been happening since 1.0.2. I wasn't on 1.0 for very long but I'm fairly certain repair was working ok. Repair worked decently for me in 0.8 (data bloat sucked). All my SSTables are version h.
> On one node:
> java.lang.AssertionError: incorrect row data size 596045 written to /mnt/cassandra/data/TRProd/Metrics1m-tmp-h-25036-Data.db; correct is 586675
> 	at org.apache.cassandra.io.sstable.SSTableWriter.appendFromStream(SSTableWriter.java:253)
> 	at org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:146)
> 	at org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:87)
> 	at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:184)
> 	at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:81)
> On the other node:
> 4999 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-24953-Data.db sections=1707 progress=0/1513497639 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25000-Data.db sections=635 progress=0/53400713 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25002-Data.db sections=570 progress=0/709993 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25003-Data.db sections=550 progress=0/449498 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25005-Data.db sections=516 progress=0/316301 - 0%], 6 sstables.
>  INFO [StreamStage:1] 2011-11-09 19:45:22,795 StreamOutSession.java (line 203) Streaming to /10.38.69.192
> ERROR [Streaming:1] 2011-11-09 19:47:47,964 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[Streaming:1,1,main]
> java.lang.RuntimeException: java.net.SocketException: Connection reset
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> 	at java.lang.Thread.run(Thread.java:619)
> Caused by: java.net.SocketException: Connection reset
> 	at java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:96)
> 	at java.net.SocketOutputStream.write(SocketOutputStream.java:136)
> 	at com.ning.compress.lzf.ChunkEncoder.encodeAndWriteChunk(ChunkEncoder.java:133)
> 	at com.ning.compress.lzf.LZFOutputStream.writeCompressedBlock(LZFOutputStream.java:203)
> 	at com.ning.compress.lzf.LZFOutputStream.write(LZFOutputStream.java:97)
> 	at org.apache.cassandra.streaming.FileStreamTask.write(FileStreamTask.java:181)
> 	at org.apache.cassandra.streaming.FileStreamTask.stream(FileStreamTask.java:145)
> 	at org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91)
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> 	... 3 more
> ERROR [Streaming:1] 2011-11-09 19:47:47,970 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[Streaming:1,1,main]
> java.lang.RuntimeException: java.net.SocketException: Connection reset
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> 	at java.lang.Thread.run(Thread.java:619)
> Caused by: java.net.SocketException: Connection reset
> 	at java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:96)
> 	at java.net.SocketOutputStream.write(SocketOutputStream.java:136)
> 	at com.ning.compress.lzf.ChunkEncoder.encodeAndWriteChunk(ChunkEncoder.java:133)
> 	at com.ning.compress.lzf.LZFOutputStream.writeCompressedBlock(LZFOutputStream.java:203)
> 	at com.ning.compress.lzf.LZFOutputStream.write(LZFOutputStream.java:97)
> 	at org.apache.cassandra.streaming.FileStreamTask.write(FileStreamTask.java:181)
> 	at org.apache.cassandra.streaming.FileStreamTask.stream(FileStreamTask.java:145)
> 	at org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91)
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> 	... 3 more

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (CASSANDRA-3481) During repair, "incorrect data size" & "Connection reset" errors. Repair unable to complete.

Posted by "Sylvain Lebresne (Updated) (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-3481?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Sylvain Lebresne updated CASSANDRA-3481:
----------------------------------------

    Attachment: 3481-v2.patch

v2 attached

bq. Shouldn't IncomingStreamReader use PRESERVE_SIZE instead of FROM_REMOTE?

That would work too but we don't need to, because we won't echo the columns like in the other branch. So it's better to use FROM_REMOTE and have the counter delta cleaned right away rather than later (as this is slightly more efficient). I added a comment.

bq. What does the commented-out test do, and why is it commented out?

Oups, attached the wrong patch. I just copied a previous test but commented it out when I realized it was simpler to just add a few lines into the existing test. v2 removes the commented-out one.

                
> During repair, "incorrect data size" & "Connection reset" errors. Repair unable to complete.
> --------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-3481
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-3481
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: 1.0.2
>            Reporter: Eric Falcao
>            Assignee: Sylvain Lebresne
>              Labels: connection, repair
>             Fix For: 1.0.3
>
>         Attachments: 3481-v2.patch, 3481.patch
>
>
> This has been happening since 1.0.2. I wasn't on 1.0 for very long but I'm fairly certain repair was working ok. Repair worked decently for me in 0.8 (data bloat sucked). All my SSTables are version h.
> On one node:
> java.lang.AssertionError: incorrect row data size 596045 written to /mnt/cassandra/data/TRProd/Metrics1m-tmp-h-25036-Data.db; correct is 586675
> 	at org.apache.cassandra.io.sstable.SSTableWriter.appendFromStream(SSTableWriter.java:253)
> 	at org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:146)
> 	at org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:87)
> 	at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:184)
> 	at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:81)
> On the other node:
> 4999 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-24953-Data.db sections=1707 progress=0/1513497639 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25000-Data.db sections=635 progress=0/53400713 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25002-Data.db sections=570 progress=0/709993 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25003-Data.db sections=550 progress=0/449498 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25005-Data.db sections=516 progress=0/316301 - 0%], 6 sstables.
>  INFO [StreamStage:1] 2011-11-09 19:45:22,795 StreamOutSession.java (line 203) Streaming to /10.38.69.192
> ERROR [Streaming:1] 2011-11-09 19:47:47,964 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[Streaming:1,1,main]
> java.lang.RuntimeException: java.net.SocketException: Connection reset
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> 	at java.lang.Thread.run(Thread.java:619)
> Caused by: java.net.SocketException: Connection reset
> 	at java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:96)
> 	at java.net.SocketOutputStream.write(SocketOutputStream.java:136)
> 	at com.ning.compress.lzf.ChunkEncoder.encodeAndWriteChunk(ChunkEncoder.java:133)
> 	at com.ning.compress.lzf.LZFOutputStream.writeCompressedBlock(LZFOutputStream.java:203)
> 	at com.ning.compress.lzf.LZFOutputStream.write(LZFOutputStream.java:97)
> 	at org.apache.cassandra.streaming.FileStreamTask.write(FileStreamTask.java:181)
> 	at org.apache.cassandra.streaming.FileStreamTask.stream(FileStreamTask.java:145)
> 	at org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91)
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> 	... 3 more
> ERROR [Streaming:1] 2011-11-09 19:47:47,970 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[Streaming:1,1,main]
> java.lang.RuntimeException: java.net.SocketException: Connection reset
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> 	at java.lang.Thread.run(Thread.java:619)
> Caused by: java.net.SocketException: Connection reset
> 	at java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:96)
> 	at java.net.SocketOutputStream.write(SocketOutputStream.java:136)
> 	at com.ning.compress.lzf.ChunkEncoder.encodeAndWriteChunk(ChunkEncoder.java:133)
> 	at com.ning.compress.lzf.LZFOutputStream.writeCompressedBlock(LZFOutputStream.java:203)
> 	at com.ning.compress.lzf.LZFOutputStream.write(LZFOutputStream.java:97)
> 	at org.apache.cassandra.streaming.FileStreamTask.write(FileStreamTask.java:181)
> 	at org.apache.cassandra.streaming.FileStreamTask.stream(FileStreamTask.java:145)
> 	at org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91)
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> 	... 3 more

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (CASSANDRA-3481) During repair, "incorrect data size" & "Connection reset" errors. Repair unable to complete.

Posted by "Sylvain Lebresne (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-3481?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13147698#comment-13147698 ] 

Sylvain Lebresne commented on CASSANDRA-3481:
---------------------------------------------

It's 1.x only as far as the bug is concerned (it's really linked to single-pass streaming). That being said, the bulk of the patch, i.e, replacing the fromRemote boolean by a Flag, could be backported in 0.8 for purpose of making future merge easier. But I'm skeptical this is worth it.
                
> During repair, "incorrect data size" & "Connection reset" errors. Repair unable to complete.
> --------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-3481
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-3481
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: 1.0.2
>            Reporter: Eric Falcao
>            Assignee: Sylvain Lebresne
>              Labels: connection, repair
>             Fix For: 1.0.3
>
>         Attachments: 3481.patch
>
>
> This has been happening since 1.0.2. I wasn't on 1.0 for very long but I'm fairly certain repair was working ok. Repair worked decently for me in 0.8 (data bloat sucked). All my SSTables are version h.
> On one node:
> java.lang.AssertionError: incorrect row data size 596045 written to /mnt/cassandra/data/TRProd/Metrics1m-tmp-h-25036-Data.db; correct is 586675
> 	at org.apache.cassandra.io.sstable.SSTableWriter.appendFromStream(SSTableWriter.java:253)
> 	at org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:146)
> 	at org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:87)
> 	at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:184)
> 	at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:81)
> On the other node:
> 4999 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-24953-Data.db sections=1707 progress=0/1513497639 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25000-Data.db sections=635 progress=0/53400713 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25002-Data.db sections=570 progress=0/709993 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25003-Data.db sections=550 progress=0/449498 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25005-Data.db sections=516 progress=0/316301 - 0%], 6 sstables.
>  INFO [StreamStage:1] 2011-11-09 19:45:22,795 StreamOutSession.java (line 203) Streaming to /10.38.69.192
> ERROR [Streaming:1] 2011-11-09 19:47:47,964 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[Streaming:1,1,main]
> java.lang.RuntimeException: java.net.SocketException: Connection reset
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> 	at java.lang.Thread.run(Thread.java:619)
> Caused by: java.net.SocketException: Connection reset
> 	at java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:96)
> 	at java.net.SocketOutputStream.write(SocketOutputStream.java:136)
> 	at com.ning.compress.lzf.ChunkEncoder.encodeAndWriteChunk(ChunkEncoder.java:133)
> 	at com.ning.compress.lzf.LZFOutputStream.writeCompressedBlock(LZFOutputStream.java:203)
> 	at com.ning.compress.lzf.LZFOutputStream.write(LZFOutputStream.java:97)
> 	at org.apache.cassandra.streaming.FileStreamTask.write(FileStreamTask.java:181)
> 	at org.apache.cassandra.streaming.FileStreamTask.stream(FileStreamTask.java:145)
> 	at org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91)
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> 	... 3 more
> ERROR [Streaming:1] 2011-11-09 19:47:47,970 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[Streaming:1,1,main]
> java.lang.RuntimeException: java.net.SocketException: Connection reset
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> 	at java.lang.Thread.run(Thread.java:619)
> Caused by: java.net.SocketException: Connection reset
> 	at java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:96)
> 	at java.net.SocketOutputStream.write(SocketOutputStream.java:136)
> 	at com.ning.compress.lzf.ChunkEncoder.encodeAndWriteChunk(ChunkEncoder.java:133)
> 	at com.ning.compress.lzf.LZFOutputStream.writeCompressedBlock(LZFOutputStream.java:203)
> 	at com.ning.compress.lzf.LZFOutputStream.write(LZFOutputStream.java:97)
> 	at org.apache.cassandra.streaming.FileStreamTask.write(FileStreamTask.java:181)
> 	at org.apache.cassandra.streaming.FileStreamTask.stream(FileStreamTask.java:145)
> 	at org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91)
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> 	... 3 more

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (CASSANDRA-3481) During repair, "incorrect data size" & "Connection reset" errors. Repair unable to complete.

Posted by "Jonathan Ellis (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-3481?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13147693#comment-13147693 ] 

Jonathan Ellis commented on CASSANDRA-3481:
-------------------------------------------

is this 1.x only or 0.8 too?
                
> During repair, "incorrect data size" & "Connection reset" errors. Repair unable to complete.
> --------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-3481
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-3481
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: 1.0.2
>            Reporter: Eric Falcao
>            Assignee: Sylvain Lebresne
>              Labels: connection, repair
>             Fix For: 1.0.3
>
>         Attachments: 3481.patch
>
>
> This has been happening since 1.0.2. I wasn't on 1.0 for very long but I'm fairly certain repair was working ok. Repair worked decently for me in 0.8 (data bloat sucked). All my SSTables are version h.
> On one node:
> java.lang.AssertionError: incorrect row data size 596045 written to /mnt/cassandra/data/TRProd/Metrics1m-tmp-h-25036-Data.db; correct is 586675
> 	at org.apache.cassandra.io.sstable.SSTableWriter.appendFromStream(SSTableWriter.java:253)
> 	at org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:146)
> 	at org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:87)
> 	at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:184)
> 	at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:81)
> On the other node:
> 4999 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-24953-Data.db sections=1707 progress=0/1513497639 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25000-Data.db sections=635 progress=0/53400713 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25002-Data.db sections=570 progress=0/709993 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25003-Data.db sections=550 progress=0/449498 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25005-Data.db sections=516 progress=0/316301 - 0%], 6 sstables.
>  INFO [StreamStage:1] 2011-11-09 19:45:22,795 StreamOutSession.java (line 203) Streaming to /10.38.69.192
> ERROR [Streaming:1] 2011-11-09 19:47:47,964 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[Streaming:1,1,main]
> java.lang.RuntimeException: java.net.SocketException: Connection reset
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> 	at java.lang.Thread.run(Thread.java:619)
> Caused by: java.net.SocketException: Connection reset
> 	at java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:96)
> 	at java.net.SocketOutputStream.write(SocketOutputStream.java:136)
> 	at com.ning.compress.lzf.ChunkEncoder.encodeAndWriteChunk(ChunkEncoder.java:133)
> 	at com.ning.compress.lzf.LZFOutputStream.writeCompressedBlock(LZFOutputStream.java:203)
> 	at com.ning.compress.lzf.LZFOutputStream.write(LZFOutputStream.java:97)
> 	at org.apache.cassandra.streaming.FileStreamTask.write(FileStreamTask.java:181)
> 	at org.apache.cassandra.streaming.FileStreamTask.stream(FileStreamTask.java:145)
> 	at org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91)
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> 	... 3 more
> ERROR [Streaming:1] 2011-11-09 19:47:47,970 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[Streaming:1,1,main]
> java.lang.RuntimeException: java.net.SocketException: Connection reset
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> 	at java.lang.Thread.run(Thread.java:619)
> Caused by: java.net.SocketException: Connection reset
> 	at java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:96)
> 	at java.net.SocketOutputStream.write(SocketOutputStream.java:136)
> 	at com.ning.compress.lzf.ChunkEncoder.encodeAndWriteChunk(ChunkEncoder.java:133)
> 	at com.ning.compress.lzf.LZFOutputStream.writeCompressedBlock(LZFOutputStream.java:203)
> 	at com.ning.compress.lzf.LZFOutputStream.write(LZFOutputStream.java:97)
> 	at org.apache.cassandra.streaming.FileStreamTask.write(FileStreamTask.java:181)
> 	at org.apache.cassandra.streaming.FileStreamTask.stream(FileStreamTask.java:145)
> 	at org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91)
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> 	... 3 more

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (CASSANDRA-3481) During repair, "incorrect data size" & "Connection reset" errors. Repair unable to complete.

Posted by "Sylvain Lebresne (Commented) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-3481?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13147561#comment-13147561 ] 

Sylvain Lebresne commented on CASSANDRA-3481:
---------------------------------------------

What is in this 'Metrics' column family? Is it normal columns, some expiring ones, super columns, counters?
                
> During repair, "incorrect data size" & "Connection reset" errors. Repair unable to complete.
> --------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-3481
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-3481
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: 1.0.2
>            Reporter: Eric Falcao
>            Assignee: Sylvain Lebresne
>              Labels: connection, repair
>             Fix For: 1.0.3
>
>
> This has been happening since 1.0.2. I wasn't on 1.0 for very long but I'm fairly certain repair was working ok. Repair worked decently for me in 0.8 (data bloat sucked). All my SSTables are version h.
> On one node:
> java.lang.AssertionError: incorrect row data size 596045 written to /mnt/cassandra/data/TRProd/Metrics1m-tmp-h-25036-Data.db; correct is 586675
> 	at org.apache.cassandra.io.sstable.SSTableWriter.appendFromStream(SSTableWriter.java:253)
> 	at org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:146)
> 	at org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:87)
> 	at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:184)
> 	at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:81)
> On the other node:
> 4999 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-24953-Data.db sections=1707 progress=0/1513497639 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25000-Data.db sections=635 progress=0/53400713 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25002-Data.db sections=570 progress=0/709993 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25003-Data.db sections=550 progress=0/449498 - 0%, /mnt/cassandra/data/TRProd/Metrics1m-h-25005-Data.db sections=516 progress=0/316301 - 0%], 6 sstables.
>  INFO [StreamStage:1] 2011-11-09 19:45:22,795 StreamOutSession.java (line 203) Streaming to /10.38.69.192
> ERROR [Streaming:1] 2011-11-09 19:47:47,964 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[Streaming:1,1,main]
> java.lang.RuntimeException: java.net.SocketException: Connection reset
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> 	at java.lang.Thread.run(Thread.java:619)
> Caused by: java.net.SocketException: Connection reset
> 	at java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:96)
> 	at java.net.SocketOutputStream.write(SocketOutputStream.java:136)
> 	at com.ning.compress.lzf.ChunkEncoder.encodeAndWriteChunk(ChunkEncoder.java:133)
> 	at com.ning.compress.lzf.LZFOutputStream.writeCompressedBlock(LZFOutputStream.java:203)
> 	at com.ning.compress.lzf.LZFOutputStream.write(LZFOutputStream.java:97)
> 	at org.apache.cassandra.streaming.FileStreamTask.write(FileStreamTask.java:181)
> 	at org.apache.cassandra.streaming.FileStreamTask.stream(FileStreamTask.java:145)
> 	at org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91)
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> 	... 3 more
> ERROR [Streaming:1] 2011-11-09 19:47:47,970 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[Streaming:1,1,main]
> java.lang.RuntimeException: java.net.SocketException: Connection reset
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> 	at java.lang.Thread.run(Thread.java:619)
> Caused by: java.net.SocketException: Connection reset
> 	at java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:96)
> 	at java.net.SocketOutputStream.write(SocketOutputStream.java:136)
> 	at com.ning.compress.lzf.ChunkEncoder.encodeAndWriteChunk(ChunkEncoder.java:133)
> 	at com.ning.compress.lzf.LZFOutputStream.writeCompressedBlock(LZFOutputStream.java:203)
> 	at com.ning.compress.lzf.LZFOutputStream.write(LZFOutputStream.java:97)
> 	at org.apache.cassandra.streaming.FileStreamTask.write(FileStreamTask.java:181)
> 	at org.apache.cassandra.streaming.FileStreamTask.stream(FileStreamTask.java:145)
> 	at org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91)
> 	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> 	... 3 more

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira