incubator-cassandra-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Piavlo <lolitus...@gmail.com>
Subject Urgent - IllegalArgumentException during compaction and memtable flush
Date Thu, 14 Jun 2012 06:26:03 GMT

  Hi,

I have a pretty urgent issue with 1.0.9 cluster

in opscenter i saw a compation that had a progress of 0% for a long 
time, looking at the cassandra log on the relevant node I see REPEATED 
messages of IllegalArgumentException in CompactionExecutor

  INFO [CompactionExecutor:3335] 2012-06-14 05:14:02,088 
CompactionTask.java (line 115) Compacting 
[SSTableReader(path='/var/lib/cassandra/data/PRODUCTION/UserCompletions-hc-838-Data.db'),

SSTableReader(path='/var/lib/cassandra/data/PRODUCTION/UserCompletions-hc-1038-Data.db'),

SSTableReader(path='/var/lib/cassandra/data/PRODUCTION/UserCompletions-hc-846-Data.db'), 
SSTableReader(path='/var/lib/cassandra/data/PRODUCTION/UserCompletions-hc-949-Data.db'), 
SSTableReader(path='/var/lib/cassandra/data/PRODUCTION/UserCompletions-hc-895-Data.db'), 
SSTableReader(path='/var/lib/cassandra/data/PRODUCTION/UserCompletions-hc-883-Data.db')]
ERROR [CompactionExecutor:3335] 2012-06-14 05:16:43,224 
AbstractCassandraDaemon.java (line 139) Fatal exception in thread 
Thread[CompactionExecutor:3335,1,main]
java.lang.IllegalArgumentException
         at java.nio.Buffer.limit(Buffer.java:249)
         at 
org.apache.cassandra.db.marshal.AbstractCompositeType.getBytes(AbstractCompositeType.java:57)
         at 
org.apache.cassandra.db.marshal.AbstractCompositeType.getWithShortLength(AbstractCompositeType.java:66)
         at 
org.apache.cassandra.db.marshal.AbstractCompositeType.compare(AbstractCompositeType.java:82)
         at 
org.apache.cassandra.db.marshal.AbstractCompositeType.compare(AbstractCompositeType.java:37)
         at 
org.apache.cassandra.db.ArrayBackedSortedColumns.compare(ArrayBackedSortedColumns.java:95)
         at 
org.apache.cassandra.db.ArrayBackedSortedColumns.addAll(ArrayBackedSortedColumns.java:214)
         at 
org.apache.cassandra.db.AbstractColumnContainer.addAll(AbstractColumnContainer.java:118)
         at 
org.apache.cassandra.db.compaction.PrecompactedRow.merge(PrecompactedRow.java:124)
         at 
org.apache.cassandra.db.compaction.PrecompactedRow.<init>(PrecompactedRow.java:97)
         at 
org.apache.cassandra.db.compaction.CompactionController.getCompactedRow(CompactionController.java:137)
         at 
org.apache.cassandra.db.compaction.CompactionIterable$Reducer.getReduced(CompactionIterable.java:97)
         at 
org.apache.cassandra.db.compaction.CompactionIterable$Reducer.getReduced(CompactionIterable.java:82)
         at 
org.apache.cassandra.utils.MergeIterator$ManyToOne.consume(MergeIterator.java:118)
         at 
org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:101)
         at 
com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:140)
         at 
com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:135)
         at 
com.google.common.collect.Iterators$7.computeNext(Iterators.java:614)
         at 
com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:140)
         at 
com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:135)
         at 
org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:175)
         at 
org.apache.cassandra.db.compaction.CompactionManager$1.call(CompactionManager.java:135)
         at 
org.apache.cassandra.db.compaction.CompactionManager$1.call(CompactionManager.java:115)
         at 
java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
         at java.util.concurrent.FutureTask.run(FutureTask.java:138)
         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)

I started looking for similar messages on other nodes saw a SINGLE 
IllegalArgumentException on ValidationExecutor on the same node and 2 
other nodes (this is a 6 node cluster)
which happened at almost the same time , in all nodes while flushing 
same UserCompletions CF memtable.
This happened about 12hours before the IllegalArgumentException in 
CompactionExecutor.

  INFO [ValidationExecutor:119] 2012-06-13 18:49:24,305 
ColumnFamilyStore.java (line 705) Enqueuing flush of 
Memtable-UserCompletions@1279460811(19014066/66201229 serialized/live 
bytes, 79838 ops)
  INFO [FlushWriter:2001] 2012-06-13 18:49:24,326 Memtable.java (line 
246) Writing Memtable-UserCompletions@1279460811(19014066/66201229 
serialized/live bytes, 79838 ops)
  INFO [FlushWriter:2001] 2012-06-13 18:49:24,848 Memtable.java (line 
283) Completed flushing 
/var/lib/cassandra/data/PRODUCTION/UserCompletions-hc-548-Data.db 
(3177074 bytes)
ERROR [ValidationExecutor:119] 2012-06-13 18:55:50,387 
AbstractCassandraDaemon.java (line 139) Fatal exception in thread 
Thread[ValidationExecutor:119,1,main]
java.lang.IllegalArgumentException
         at java.nio.Buffer.limit(Buffer.java:249)
         at 
org.apache.cassandra.db.marshal.AbstractCompositeType.getBytes(AbstractCompositeType.java:57)
         at 
org.apache.cassandra.db.marshal.AbstractCompositeType.getWithShortLength(AbstractCompositeType.java:66)
         at 
org.apache.cassandra.db.marshal.AbstractCompositeType.compare(AbstractCompositeType.java:82)
         at 
org.apache.cassandra.db.marshal.AbstractCompositeType.compare(AbstractCompositeType.java:37)
         at 
org.apache.cassandra.db.ArrayBackedSortedColumns.compare(ArrayBackedSortedColumns.java:95)
         at 
org.apache.cassandra.db.ArrayBackedSortedColumns.addAll(ArrayBackedSortedColumns.java:214)
         at 
org.apache.cassandra.db.AbstractColumnContainer.addAll(AbstractColumnContainer.java:118)
         at 
org.apache.cassandra.db.compaction.PrecompactedRow.merge(PrecompactedRow.java:124)
         at 
org.apache.cassandra.db.compaction.PrecompactedRow.<init>(PrecompactedRow.java:97)
         at 
org.apache.cassandra.db.compaction.CompactionController.getCompactedRow(CompactionController.java:137)
         at 
org.apache.cassandra.db.compaction.CompactionIterable$Reducer.getReduced(CompactionIterable.java:97)
         at 
org.apache.cassandra.db.compaction.CompactionIterable$Reducer.getReduced(CompactionIterable.java:82)
         at 
org.apache.cassandra.utils.MergeIterator$ManyToOne.consume(MergeIterator.java:118)
         at 
org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:101)
         at 
com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:140)
         at 
com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:135)
         at 
com.google.common.collect.Iterators$7.computeNext(Iterators.java:614)
         at 
com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:140)
         at 
com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:135)
         at 
org.apache.cassandra.db.compaction.CompactionManager.doValidationCompaction(CompactionManager.java:849)
         at 
org.apache.cassandra.db.compaction.CompactionManager.access$600(CompactionManager.java:64)
         at 
org.apache.cassandra.db.compaction.CompactionManager$8.call(CompactionManager.java:440)
         at 
java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
         at java.util.concurrent.FutureTask.run(FutureTask.java:138)
         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)


Why does it happen and what I can do resolve it?

Currently opscenter does not show this compaction as active any more and 
no more CompactionExecutor  excpetions in the log , but it looks like 
cassandra simply gave up on compacting since the relevant sstables still 
exist.

And even bigger problem now is that running repairs on other CFs against 
different nodes does not have any effect, for example running
/usr/bin/nodetool -h dsc2b.internal -pr repair PRODUCTION UserDirectVendors
does not trigger any repair activity and nothing in the logs to indicate 
a start of repair. And I have ~24hours left to repair some CFs before 
the gc period ends :(

Besides that the cluster is working fine.

Thanks
Alex

Mime
View raw message