[
https://issues.apache.org/jira/browse/CASSANDRA-4411?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13411423#comment-13411423
]
Rudolf VanderLeeden commented on CASSANDRA-4411:
------------------------------------------------
I could reproduce the problem on a 3-node testcluster with 1.1.2 and LCS.
Replication factor is 3 and number of total keys is 24m.
I added SSTables from a previous backup to node1.
Then running on node1: nodetool repair -pr
Result:
INFO [CompactionExecutor:7] 2012-07-11 10:06:57,632 CompactionTask.java (line
109) Compacting
[SSTableReader(path='/mnt/cassandra/data/highscores/highscore/highscores-highscore-hd-4937-Data.db')]
INFO [CompactionExecutor:7] 2012-07-11 10:06:58,601 CompactionTask.java (line
221) Compacted to
[/mnt/cassandra/data/highscores/highscore/highscores-highscore-hd-5591-Data.db,].
5,252,617 to 5,252,617 (~100% of original) bytes for 51,419 keys at
5.174882MB/s. Time: 968ms.
INFO [CompactionExecutor:6] 2012-07-11 10:06:58,602 CompactionTask.java (line
109) Compacting
[SSTableReader(path='/mnt/cassandra/data/highscores/highscore/highscores-highscore-hd-5590-Data.db'),
SSTableReader(path='/mnt/cassandra/data/highscores/highscore/highscores-highscore-hd-5571-Data.db')]
ERROR [CompactionExecutor:6] 2012-07-11 10:06:59,655
AbstractCassandraDaemon.java (line 134) Exception in thread
Thread[CompactionExecutor:6,1,main]
java.lang.AssertionError
at
org.apache.cassandra.db.compaction.LeveledManifest.promote(LeveledManifest.java:214)
at
org.apache.cassandra.db.compaction.LeveledCompactionStrategy.handleNotification(LeveledCompactionStrategy.java:158)
at
org.apache.cassandra.db.DataTracker.notifySSTablesChanged(DataTracker.java:531)
at
org.apache.cassandra.db.DataTracker.replaceCompactedSSTables(DataTracker.java:254)
at
org.apache.cassandra.db.ColumnFamilyStore.replaceCompactedSSTables(ColumnFamilyStore.java:978)
at
org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:200)
at
org.apache.cassandra.db.compaction.LeveledCompactionTask.execute(LeveledCompactionTask.java:50)
at
org.apache.cassandra.db.compaction.CompactionManager$1.runMayThrow(CompactionManager.java:150)
at
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
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:1110)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
at java.lang.Thread.run(Thread.java:636)
The next repair command throws the following assertion:
ERROR [ValidationExecutor:2] 2012-07-11 10:31:28,020
AbstractCassandraDaemon.java (line 134) Exception in thread
Thread[ValidationExecutor:2,1,main]
java.lang.AssertionError: row
DecoratedKey(162957119114255422766928006879345246467,
c9e91cfb77634f32b9399dd4ad6b784e93dec9d0b11f431dad58a35e9f623de9) received out
of order wrt DecoratedKey(165755005851296361665897424577644629314,
ac63200da3fb452ca0b57a648b90c8a427a3d45b2d2146e089c6d04b959bb207)
at
org.apache.cassandra.service.AntiEntropyService$Validator.add(AntiEntropyService.java:349)
at
org.apache.cassandra.db.compaction.CompactionManager.doValidationCompaction(CompactionManager.java:712)
...etc...
Let me know if you need more from the log.
Thanks, -Rudolf.
> Assertion with LCS compaction
> -----------------------------
>
> Key: CASSANDRA-4411
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4411
> Project: Cassandra
> Issue Type: Bug
> Components: Core
> Affects Versions: 1.1.2
> Reporter: Anton Winter
> Fix For: 1.1.3
>
>
> As instructed in CASSANDRA-4321 I have raised this issue as a continuation of
> that issue as it appears the problem still exists.
> I have repeatedly run sstablescrub across all my nodes after the 1.1.2
> upgrade until sstablescrub shows no errors. The exceptions described in
> CASSANDRA-4321 do not occur as frequently now but the integrity check still
> throws exceptions on a number of nodes. Once those exceptions occur
> compactionstats shows a large number of pending tasks with no progression
> afterwards.
> {code}
> ERROR [CompactionExecutor:150] 2012-07-05 04:26:15,570
> AbstractCassandraDaemon.java (line 134) Exception in thread
> Thread[CompactionExecutor:150,1,main]
> java.lang.AssertionError
> at
> org.apache.cassandra.db.compaction.LeveledManifest.promote(LeveledManifest.java:214)
> at
> org.apache.cassandra.db.compaction.LeveledCompactionStrategy.handleNotification(LeveledCompactionStrategy.java:158)
> at
> org.apache.cassandra.db.DataTracker.notifySSTablesChanged(DataTracker.java:531)
> at
> org.apache.cassandra.db.DataTracker.replaceCompactedSSTables(DataTracker.java:254)
> at
> org.apache.cassandra.db.ColumnFamilyStore.replaceCompactedSSTables(ColumnFamilyStore.java:978)
> at
> org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:200)
> at
> org.apache.cassandra.db.compaction.LeveledCompactionTask.execute(LeveledCompactionTask.java:50)
> at
> org.apache.cassandra.db.compaction.CompactionManager$1.runMayThrow(CompactionManager.java:150)
> at
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> 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:1110)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
> at java.lang.Thread.run(Thread.java:636)
> {code}
--
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