Return-Path: X-Original-To: apmail-cassandra-commits-archive@www.apache.org Delivered-To: apmail-cassandra-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id E08F29C6D for ; Fri, 5 Oct 2012 07:23:10 +0000 (UTC) Received: (qmail 52876 invoked by uid 500); 5 Oct 2012 07:22:48 -0000 Delivered-To: apmail-cassandra-commits-archive@cassandra.apache.org Received: (qmail 52861 invoked by uid 500); 5 Oct 2012 07:22:48 -0000 Mailing-List: contact commits-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@cassandra.apache.org Delivered-To: mailing list commits@cassandra.apache.org Received: (qmail 52838 invoked by uid 99); 5 Oct 2012 07:22:47 -0000 Received: from arcas.apache.org (HELO arcas.apache.org) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 05 Oct 2012 07:22:47 +0000 Date: Fri, 5 Oct 2012 18:22:47 +1100 (NCT) From: "Yuki Morishita (JIRA)" To: commits@cassandra.apache.org Message-ID: <1509031236.4482.1349421767857.JavaMail.jiratomcat@arcas> In-Reply-To: <1637309957.140192.1348853350194.JavaMail.jiratomcat@arcas> Subject: [jira] [Commented] (CASSANDRA-4733) Last written key >= current key exception when streaming MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/CASSANDRA-4733?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13470075#comment-13470075 ] Yuki Morishita commented on CASSANDRA-4733: ------------------------------------------- I still cannot reproduce above error, but I believe this was caused by the change in CASSANDRA-4710. SSTableReader#getPositionsForRanges is used to determine the sections to transfer inside sstable, but the method returns incorrect sections for some cases. In fact, system.log file that I got from Brandon showed that the node was trying to stream sections way bigger than actual sstable file size. > Last written key >= current key exception when streaming > -------------------------------------------------------- > > Key: CASSANDRA-4733 > URL: https://issues.apache.org/jira/browse/CASSANDRA-4733 > Project: Cassandra > Issue Type: Bug > Components: Core > Affects Versions: 1.2.0 beta 1 > Reporter: Brandon Williams > Assignee: Yuki Morishita > > {noformat} > ERROR 16:52:56,260 Exception in thread Thread[Streaming to /10.179.111.137:1,5,main] > java.lang.RuntimeException: java.io.IOException: Connection reset by peer > 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(ThreadPoolExecutor.java:886) > at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) > at java.lang.Thread.run(Thread.java:662) > Caused by: java.io.IOException: Connection reset by peer > at sun.nio.ch.FileDispatcher.write0(Native Method) > at sun.nio.ch.SocketDispatcher.write(SocketDispatcher.java:29) > at sun.nio.ch.IOUtil.writeFromNativeBuffer(IOUtil.java:72) > at sun.nio.ch.IOUtil.write(IOUtil.java:43) > at sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:334) > at java.nio.channels.Channels.writeFullyImpl(Channels.java:59) > at java.nio.channels.Channels.writeFully(Channels.java:81) > at java.nio.channels.Channels.access$000(Channels.java:47) > at java.nio.channels.Channels$1.write(Channels.java:155) > 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:218) > at org.apache.cassandra.streaming.FileStreamTask.stream(FileStreamTask.java:164) > at org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91) > at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) > ... 3 more > ERROR 16:53:03,951 Exception in thread Thread[Thread-11,5,main] > java.lang.RuntimeException: Last written key DecoratedKey(113424593524874987650593774422007331058, 3036303936343535) >= current key DecoratedKey(59229538317742990547810678738983628664, 3036313133373139) writing into /var/lib/cassandra/data/Keyspace1-Standard1-tmp-ia-95-Data.db > at org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:132) > at org.apache.cassandra.io.sstable.SSTableWriter.appendFromStream(SSTableWriter.java:208) > at org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:164) > at org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:107) > at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:220) > at org.apache.cassandra.net.IncomingTcpConnection.handleStream(IncomingTcpConnection.java:165) > at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:65) > {noformat} > I didn't do anything fancy here, just inserted about 6M keys at rf=2, then ran repair and got this. -- 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