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

Brandon Kearby commented on CASSANDRA-6285:
-------------------------------------------

I'm getting it as well on 2.0.4. I'm testing a new cluster. I don't get the 
error with one node, but when I add two or more I get the same error.
{code}
ERROR [CompactionExecutor:6] 2014-01-15 17:13:13,395 CassandraDaemon.java (line 
187) Exception in thread Thread[CompactionExecutor:6,1,main]
java.lang.RuntimeException: Last written key DecoratedKey(-1983406872803353678, 
545749545445523a333535383030333439353835353830303334) >= current key 
DecoratedKey(-7683510718755081698, 
545749545445523a333639333235363931383339333238323537) writing into 
/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-tmp-jb-121-Data.db
        at 
org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:142)
        at 
org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:165)
        at 
org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:160)
        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:60)
        at 
org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
        at 
org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:197)
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
        at java.util.concurrent.FutureTask.run(FutureTask.java:262)
        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:744)
{code}

Here's the schema I'm testing with:
{code}
create column family signal
  with column_type = 'Standard'
  and comparator = 'UTF8Type'
  and default_validation_class = 'BytesType'
  and key_validation_class = 'UTF8Type'
  and read_repair_chance = 0.1
  and dclocal_read_repair_chance = 0.0
  and gc_grace = 432000
  and min_compaction_threshold = 4
  and max_compaction_threshold = 32
  and replicate_on_write = true
  and compaction_strategy = 
'org.apache.cassandra.db.compaction.LeveledCompactionStrategy'
  and caching = 'ALL'
  and compaction_strategy_options = {'sstable_size_in_mb' : '160'}
  and comment = 'A store of information about each individual signal.'
  and column_metadata = [
    {column_name : 'type',
    validation_class : UTF8Type},
    {column_name : 'foo_id',
    validation_class : LongType},
    validation_class : UTF8Type}]
  and compression_options = {'sstable_compression' : 
'org.apache.cassandra.io.compress.LZ4Compressor'};
{code}

> LCS compaction failing with Exception
> -------------------------------------
>
>                 Key: CASSANDRA-6285
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-6285
>             Project: Cassandra
>          Issue Type: Bug
>         Environment: 4 nodes, shortly updated from 1.2.11 to 2.0.2
>            Reporter: David Sauer
>
> After altering everything to LCS the table OpsCenter.rollups60 amd one other 
> none OpsCenter-Table got stuck with everything hanging around in L0.
> The compaction started and ran until the logs showed this:
> ERROR [CompactionExecutor:111] 2013-11-01 19:14:53,865 CassandraDaemon.java 
> (line 187) Exception in thread Thread[CompactionExecutor:111,1,RMI Runtime]
> java.lang.RuntimeException: Last written key 
> DecoratedKey(1326283851463420237, 
> 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574426c6f6f6d46696c746572537061636555736564)
>  >= current key DecoratedKey(954210699457429663, 
> 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574546f74616c4469736b5370616365557365640b0f)
>  writing into 
> /var/lib/cassandra/data/OpsCenter/rollups60/OpsCenter-rollups60-tmp-jb-58656-Data.db
>       at 
> org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:141)
>       at 
> org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:164)
>       at 
> org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:160)
>       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:60)
>       at 
> org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
>       at 
> org.apache.cassandra.db.compaction.CompactionManager$6.runMayThrow(CompactionManager.java:296)
>       at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>       at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
>       at java.util.concurrent.FutureTask.run(FutureTask.java:262)
>       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:724)
> Moving back to STC worked to keep the compactions running.
> Especialy my own Table i would like to move to LCS.
> After a major compaction with STC the move to LCS fails with the same 
> Exception.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)

Reply via email to