You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Ravi Prasad (JIRA)" <ji...@apache.org> on 2013/10/04 01:08:42 UTC

[jira] [Commented] (CASSANDRA-4733) Last written key >= current key exception when streaming

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

Ravi Prasad commented on CASSANDRA-4733:
----------------------------------------

We are seeing this in clusters running version 1.2.9 on freshly flushed sstable during compaction.  Below, cassie-facetrevs4-ic-2015-Data.db got flushed, and compaction picked it as a candidate. But it fails the key ordering check right away.  Running sstablekeys on cassie-facetrevs4-ic-2015-Data.db also fails with the same exception of 'keys out of order'.  Note, this is on a cluster which has always been running 1.2.9 (no migration/upgrades) and murmur3partitioning from the beginning.  

INFO [OptionalTasks:1] 2013-09-29 18:49:28,845 ColumnFamilyStore.java (line 629) Enqueuing flush of Memtable-facetrevs4@432766073(12420768/30408704 serialized/live bytes, 1201311 ops)
 INFO [FlushWriter:914] 2013-09-29 18:49:28,881 Memtable.java (line 461) Writing Memtable-facetrevs4@432766073(12420768/30408704 serialized/live bytes, 1201311 ops)
 INFO [FlushWriter:914] 2013-09-29 18:49:30,527 Memtable.java (line 495) Completed flushing /home/y/var/cassandra/data/cassie/facetrevs4/cassie-facetrevs4-ic-2015-Data.db (20227333 bytes) for commitlog position ReplayPosition(segmentId=1378452013797, position=65238)
 INFO [CompactionExecutor:14184] 2013-09-29 18:49:30,530 CompactionTask.java (line 105) Compacting [SSTableReader(path='/home/y/var/cassandra/data/cassie/facetrevs4/cassie-facetrevs4-ic-2014-Data.db'), SSTableReader(path='/home/y/var/cassandra/data/cassie/facetrevs4/cassie-facetrevs4-ic-2015-Data.db')]
ERROR [CompactionExecutor:14184] 2013-09-29 18:50:01,040 CassandraDaemon.java (line 192) Exception in thread Thread[CompactionExecutor:14184,1,main]
java.lang.RuntimeException: Last written key DecoratedKey(-3090218156741434252, 6465724944223a225961686f6f4c6f63616c73222c2244617461547970654944223a224c6f63616c73496e736967687473) >= current key DecoratedKey(-5673637012055785579, 94fdbfa0d7683714ac1e2693360b44627961686f6f2d6c697374696e672d7973622d6d73646173683a6d65746164617461) writing into /home/y/var/cassandra/data/cassie/facetrevs4/cassie-facetrevs4-tmp-ic-2016-Data.db
        at org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:133)
        at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:155)
        at org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:162)
        at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
        at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
        at org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:58)
        at org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:60)
        at org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:211)
        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
        at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
        at java.util.concurrent.FutureTask.run(FutureTask.java:166)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        at java.lang.Thread.run(Thread.java:722)

> 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
>             Fix For: 1.2.0 beta 2
>
>
> {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 was sent by Atlassian JIRA
(v6.1#6144)