[jira] [Comment Edited] (CASSANDRA-4321) stackoverflow building interval tree & possible sstable corruptions
[ https://issues.apache.org/jira/browse/CASSANDRA-4321?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13406254#comment-13406254 ] Anton Winter edited comment on CASSANDRA-4321 at 7/4/12 3:28 AM: - I have repeatedly run sstablescrub across all my nodes and the exceptions do not occur as frequently now, however, the integrity check still throws exceptions and compactionstats shows a large number of pending tasks but no progression afterwards. Should this ticket be reopened or a new one raised? {code} ERROR [CompactionExecutor:912] 2012-07-04 01:07:16,470 AbstractCassandraDaemon.java (line 134) Exception in thread Thread[CompactionExecutor:912,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} was (Author: awinter): I have repeatedly run sstablescrub across all my nodes and the exceptions do not occur as frequently now, however, the integrity check still throw exceptions. compactionstats shows a large number of pending tasks but no progression after this error. Should this ticket be reopened or a new one raised? {code} ERROR [CompactionExecutor:912] 2012-07-04 01:07:16,470 AbstractCassandraDaemon.java (line 134) Exception in thread Thread[CompactionExecutor:912,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} > stackoverflow building interval tree & possible sstable corruptions > --- > > Key: CASSANDRA-4321 > URL: https://issues.apache.org/jira/browse/CASSANDRA-4321 > Project: Cassandra > Issue Type: Bug > Components: Core >Affects Versions: 1.1.1 >Reporter: Anton Winter >Assignee: Sylvain Lebresne > Fix For: 1.1.2 > > Attachments: 0001-Fix-overlapping-computation-v7.txt, > 0002-Scrub-detects-and-repair-outOfOrder-rows-v7.txt, > 0003-Create-standalone-scrub-v7.txt, > 0004-Add-manifest-integrity-check-v7.txt, cleanup.txt, > ooyala-hastur-stacktrace.txt > > > After upgrading to 1.1.1 (from 1.1.0) I have started experiencing > StackOverflowError's resulting in compaction backlog and f
[jira] [Comment Edited] (CASSANDRA-4321) stackoverflow building interval tree & possible sstable corruptions
[ https://issues.apache.org/jira/browse/CASSANDRA-4321?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13396433#comment-13396433 ] Anton Winter edited comment on CASSANDRA-4321 at 6/19/12 2:01 AM: -- I can confirm I also experienced the "Unexpected empty index file" errors on some of the nodes that I have run sstablescrub on. Other nodes had this error when running sstablescrub: {code} Scrub of SSTableReader(path='/var/lib//data/cassandra/KS/CF/KS-CF-hd-259648-Data.db') complete: 1592 rows in new sstable and 0 empty (tombstoned) rows dropped EOF after 6 bytes out of 8 {code} Compactions stop with the "java.lang.RuntimeException: Last written key DecoratedKey" error on the nodes affected by either of the above 2 errors . Nodes that seem to have been repaired by the sstablescrub still continue to have "java.lang.RuntimeException: Last written key DecoratedKey" errors scattered through the logs but are still be compacting. Is there any further information we can supply to help debug? was (Author: awinter): I can confirm I also experienced the "Unexpected empty index file" errors on some of the nodes that I have run sstablescrub on. On some other nodes the sstablescrub command appears to complete successfully but compactions still stops at the "java.lang.RuntimeException: Last written key DecoratedKey" error. Is there any further information we can supply to help debug? > stackoverflow building interval tree & possible sstable corruptions > --- > > Key: CASSANDRA-4321 > URL: https://issues.apache.org/jira/browse/CASSANDRA-4321 > Project: Cassandra > Issue Type: Bug > Components: Core >Affects Versions: 1.1.1 >Reporter: Anton Winter >Assignee: Sylvain Lebresne > Fix For: 1.1.2 > > Attachments: > 0001-Change-Range-Bounds-in-LeveledManifest.overlapping-v3.txt, > 0002-Scrub-detects-and-repair-outOfOrder-rows-v3.txt, > 0003-Create-standalone-scrub-v3.txt, ooyala-hastur-stacktrace.txt > > > After upgrading to 1.1.1 (from 1.1.0) I have started experiencing > StackOverflowError's resulting in compaction backlog and failure to restart. > The ring currently consists of 6 DC's and 22 nodes using LCS & compression. > This issue was first noted on 2 nodes in one DC and then appears to have > spread to various other nodes in the other DC's. > When the first occurrence of this was found I restarted the instance but it > failed to start so I cleared its data and treated it as a replacement node > for the token it was previously responsible for. This node successfully > streamed all the relevant data back but failed again a number of hours later > with the same StackOverflowError and again was unable to restart. > The initial stack overflow error on a running instance looks like this: > ERROR [CompactionExecutor:314] 2012-06-07 09:59:43,017 > AbstractCassandraDaemon.java (line 134) Exception in thread > Thread[CompactionExecutor:314,1,main] > java.lang.StackOverflowError > at java.util.Arrays.mergeSort(Arrays.java:1157) > at java.util.Arrays.sort(Arrays.java:1092) > at java.util.Collections.sort(Collections.java:134) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.findMinMedianMax(IntervalNode.java:114) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:49) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > [snip - this repeats until stack overflow. Compactions stop from this point > onwards] > I restarted this failing instance with DEBUG logging enabled and it throws > the following exception part way through startup: > ERROR 11:37:51,046 Exception in thread Thread[OptionalTasks:1,5,main] > java.lang.StackOverflowError > at > org.slf4j.helpers.MessageFormatter.safeObjectAppend(MessageFormatter.java:307) > at > org.slf4j.helpers.MessageFormatter.deeplyAppendParameter(MessageFormatter.java:276) > at > org.slf4j.helpers.MessageFormatter.arrayFormat(MessageFormatter.java:230) > at > org.slf4j.helpers.MessageFormatter.format(MessageFormatter.java:124) > at > org.slf4j.impl.Log4jLoggerAdapter.debug(Log4jLoggerAdapter.java:228) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:45) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > [snip - this repeat
[jira] [Comment Edited] (CASSANDRA-4321) stackoverflow building interval tree & possible sstable corruptions
[ https://issues.apache.org/jira/browse/CASSANDRA-4321?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13396433#comment-13396433 ] Anton Winter edited comment on CASSANDRA-4321 at 6/19/12 2:02 AM: -- I can confirm I also experienced the "Unexpected empty index file" errors on some of the nodes that I have run sstablescrub on. Other nodes had this error when running sstablescrub: {code} Scrub of SSTableReader(path='/var/lib//data/cassandra/KS/CF/KS-CF-hd-259648-Data.db') complete: 1592 rows in new sstable and 0 empty (tombstoned) rows dropped EOF after 6 bytes out of 8 {code} Compactions stop with the "java.lang.RuntimeException: Last written key DecoratedKey" error on the nodes affected by either of the above 2 errors . Nodes that seem to have been repaired by the sstablescrub still continue to have "java.lang.RuntimeException: Last written key DecoratedKey" errors scattered through the logs but are still compacting. Is there any further information we can supply to help debug? was (Author: awinter): I can confirm I also experienced the "Unexpected empty index file" errors on some of the nodes that I have run sstablescrub on. Other nodes had this error when running sstablescrub: {code} Scrub of SSTableReader(path='/var/lib//data/cassandra/KS/CF/KS-CF-hd-259648-Data.db') complete: 1592 rows in new sstable and 0 empty (tombstoned) rows dropped EOF after 6 bytes out of 8 {code} Compactions stop with the "java.lang.RuntimeException: Last written key DecoratedKey" error on the nodes affected by either of the above 2 errors . Nodes that seem to have been repaired by the sstablescrub still continue to have "java.lang.RuntimeException: Last written key DecoratedKey" errors scattered through the logs but are still be compacting. Is there any further information we can supply to help debug? > stackoverflow building interval tree & possible sstable corruptions > --- > > Key: CASSANDRA-4321 > URL: https://issues.apache.org/jira/browse/CASSANDRA-4321 > Project: Cassandra > Issue Type: Bug > Components: Core >Affects Versions: 1.1.1 >Reporter: Anton Winter >Assignee: Sylvain Lebresne > Fix For: 1.1.2 > > Attachments: > 0001-Change-Range-Bounds-in-LeveledManifest.overlapping-v3.txt, > 0002-Scrub-detects-and-repair-outOfOrder-rows-v3.txt, > 0003-Create-standalone-scrub-v3.txt, ooyala-hastur-stacktrace.txt > > > After upgrading to 1.1.1 (from 1.1.0) I have started experiencing > StackOverflowError's resulting in compaction backlog and failure to restart. > The ring currently consists of 6 DC's and 22 nodes using LCS & compression. > This issue was first noted on 2 nodes in one DC and then appears to have > spread to various other nodes in the other DC's. > When the first occurrence of this was found I restarted the instance but it > failed to start so I cleared its data and treated it as a replacement node > for the token it was previously responsible for. This node successfully > streamed all the relevant data back but failed again a number of hours later > with the same StackOverflowError and again was unable to restart. > The initial stack overflow error on a running instance looks like this: > ERROR [CompactionExecutor:314] 2012-06-07 09:59:43,017 > AbstractCassandraDaemon.java (line 134) Exception in thread > Thread[CompactionExecutor:314,1,main] > java.lang.StackOverflowError > at java.util.Arrays.mergeSort(Arrays.java:1157) > at java.util.Arrays.sort(Arrays.java:1092) > at java.util.Collections.sort(Collections.java:134) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.findMinMedianMax(IntervalNode.java:114) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:49) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > [snip - this repeats until stack overflow. Compactions stop from this point > onwards] > I restarted this failing instance with DEBUG logging enabled and it throws > the following exception part way through startup: > ERROR 11:37:51,046 Exception in thread Thread[OptionalTasks:1,5,main] > java.lang.StackOverflowError > at > org.slf4j.helpers.MessageFormatter.safeObjectAppend(MessageFormatter.java:307) > at > org.slf4j.helpers.MessageFormatter.deeplyAppendParameter(MessageFormatter.java:276) > at > org.slf4j.helpers.MessageFormatter.arrayFormat(MessageFormatter.java:230) > at > org.slf4j.helpers.M
[jira] [Comment Edited] (CASSANDRA-4321) stackoverflow building interval tree & possible sstable corruptions
[ https://issues.apache.org/jira/browse/CASSANDRA-4321?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13396319#comment-13396319 ] Al Tobey edited comment on CASSANDRA-4321 at 6/18/12 11:26 PM: --- Offline scrub ran fine for me. I downgraded to 1.1.0 and ran a compaction and it looks fine. (edit) finished offline scrub on both affected nodes and they're back to normal. was (Author: a...@ooyala.com): Offline scrub ran fine for me. I downgraded to 1.1.0 and ran a compaction and it looks fine. > stackoverflow building interval tree & possible sstable corruptions > --- > > Key: CASSANDRA-4321 > URL: https://issues.apache.org/jira/browse/CASSANDRA-4321 > Project: Cassandra > Issue Type: Bug > Components: Core >Affects Versions: 1.1.1 >Reporter: Anton Winter >Assignee: Sylvain Lebresne > Fix For: 1.1.2 > > Attachments: > 0001-Change-Range-Bounds-in-LeveledManifest.overlapping-v3.txt, > 0002-Scrub-detects-and-repair-outOfOrder-rows-v3.txt, > 0003-Create-standalone-scrub-v3.txt, ooyala-hastur-stacktrace.txt > > > After upgrading to 1.1.1 (from 1.1.0) I have started experiencing > StackOverflowError's resulting in compaction backlog and failure to restart. > The ring currently consists of 6 DC's and 22 nodes using LCS & compression. > This issue was first noted on 2 nodes in one DC and then appears to have > spread to various other nodes in the other DC's. > When the first occurrence of this was found I restarted the instance but it > failed to start so I cleared its data and treated it as a replacement node > for the token it was previously responsible for. This node successfully > streamed all the relevant data back but failed again a number of hours later > with the same StackOverflowError and again was unable to restart. > The initial stack overflow error on a running instance looks like this: > ERROR [CompactionExecutor:314] 2012-06-07 09:59:43,017 > AbstractCassandraDaemon.java (line 134) Exception in thread > Thread[CompactionExecutor:314,1,main] > java.lang.StackOverflowError > at java.util.Arrays.mergeSort(Arrays.java:1157) > at java.util.Arrays.sort(Arrays.java:1092) > at java.util.Collections.sort(Collections.java:134) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.findMinMedianMax(IntervalNode.java:114) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:49) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > [snip - this repeats until stack overflow. Compactions stop from this point > onwards] > I restarted this failing instance with DEBUG logging enabled and it throws > the following exception part way through startup: > ERROR 11:37:51,046 Exception in thread Thread[OptionalTasks:1,5,main] > java.lang.StackOverflowError > at > org.slf4j.helpers.MessageFormatter.safeObjectAppend(MessageFormatter.java:307) > at > org.slf4j.helpers.MessageFormatter.deeplyAppendParameter(MessageFormatter.java:276) > at > org.slf4j.helpers.MessageFormatter.arrayFormat(MessageFormatter.java:230) > at > org.slf4j.helpers.MessageFormatter.format(MessageFormatter.java:124) > at > org.slf4j.impl.Log4jLoggerAdapter.debug(Log4jLoggerAdapter.java:228) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:45) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > [snip - this repeats until stack overflow] > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.In
[jira] [Comment Edited] (CASSANDRA-4321) stackoverflow building interval tree & possible sstable corruptions
[ https://issues.apache.org/jira/browse/CASSANDRA-4321?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13396246#comment-13396246 ] Omid Aladini edited comment on CASSANDRA-4321 at 6/18/12 9:52 PM: -- Thanks for the patch. Offline scrub is indeed very useful. Tried the v3 patches and the scrub didn't complete, possibly because of a different issue, with the following failed assertion: {code} Exception in thread "main" java.lang.AssertionError: Unexpected empty index file: RandomAccessReader(filePath='/var/lib/cassandra/abcd/data/SOMEKSP/CF3/SOMEKSP-CF3-tmp-hd-33827-Index.db', skipIOCache=true) at org.apache.cassandra.io.sstable.SSTable.estimateRowsFromIndex(SSTable.java:221) at org.apache.cassandra.io.sstable.SSTableReader.load(SSTableReader.java:376) at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:203) at org.apache.cassandra.io.sstable.SSTableReader.openNoValidation(SSTableReader.java:143) at org.apache.cassandra.tools.StandaloneScrubber.main(StandaloneScrubber.java:79) {code} which consequently, encountered corrupt SSTables during start-up: {code} 2012-06-18_20:36:19.89543 INFO 20:36:19,895 Opening /var/lib/cassandra/abcd/data/SOMEKSP/CF3/SOMEKSP-CF3-hd-24984 (1941993 bytes) 2012-06-18_20:36:19.90217 ERROR 20:36:19,900 Exception in thread Thread[SSTableBatchOpen:9,5,main] 2012-06-18_20:36:19.90222 java.lang.IllegalStateException: SSTable first key DecoratedKey(41255474878128469814942789647212295629, 31303132393937357c3337313730333536) > last key DecoratedKey(41219536226656199861610796307350537953, 31303234323538397c3331383436373338) 2012-06-18_20:36:19.90261 at org.apache.cassandra.io.sstable.SSTableReader.validate(SSTableReader.java:441) 2012-06-18_20:36:19.90275 at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:208) 2012-06-18_20:36:19.90291 at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:153) 2012-06-18_20:36:19.90309 at org.apache.cassandra.io.sstable.SSTableReader$1.run(SSTableReader.java:245) 2012-06-18_20:36:19.90324 at java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source) 2012-06-18_20:36:19.90389 at java.util.concurrent.FutureTask$Sync.innerRun(Unknown Source) 2012-06-18_20:36:19.90391 at java.util.concurrent.FutureTask.run(Unknown Source) 2012-06-18_20:36:19.90391 at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown Source) 2012-06-18_20:36:19.90392 at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source) 2012-06-18_20:36:19.90392 at java.lang.Thread.run(Unknown Source) {code} although didn't prevent Cassandra from starting up, but compaction failed subsequently: {code} 2012-06-18_20:51:41.79122 ERROR 20:51:41,790 Exception in thread Thread[CompactionExecutor:81,1,main] 2012-06-18_20:51:41.79131 java.lang.RuntimeException: Last written key DecoratedKey(12341204629749023303706929560940823070, 33363037353338) >= current key DecoratedKey(12167298275958419273792070792442127650, 31363431343537) writing into /var/lib/cassandra/abcd/data/SOMEKSP/CF3/SOMEKSP-CF3-tmp-hd-40992-Data.db 2012-06-18_20:51:41.79161 at org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:134) 2012-06-18_20:51:41.79169 at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:153) 2012-06-18_20:51:41.79180 at org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:159) 2012-06-18_20:51:41.79189 at org.apache.cassandra.db.compaction.LeveledCompactionTask.execute(LeveledCompactionTask.java:50) 2012-06-18_20:51:41.79199 at org.apache.cassandra.db.compaction.CompactionManager$1.runMayThrow(CompactionManager.java:150) 2012-06-18_20:51:41.79210 at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30) 2012-06-18_20:51:41.79218 at java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source) 2012-06-18_20:51:41.79227 at java.util.concurrent.FutureTask$Sync.innerRun(Unknown Source) 2012-06-18_20:51:41.79235 at java.util.concurrent.FutureTask.run(Unknown Source) 2012-06-18_20:51:41.79242 at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown Source) 2012-06-18_20:51:41.79250 at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source) 2012-06-18_20:51:41.79259 at java.lang.Thread.run(Unknown Source) {code} was (Author: omid): Thanks for the patch. Offline scrub is indeed very useful. Tried the v3 patches and the scrub didn't complete, possibly because of a different issue, with the following failed assertion: {code} Exception in thread "main" java.lang.AssertionError: Unexpected empty index file: RandomAccessReader(filePath='/var/lib/cassandra/abcd/data/SOMEKSP/C
[jira] [Comment Edited] (CASSANDRA-4321) stackoverflow building interval tree & possible sstable corruptions
[ https://issues.apache.org/jira/browse/CASSANDRA-4321?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13293412#comment-13293412 ] Anton Winter edited comment on CASSANDRA-4321 at 6/12/12 7:46 AM: -- If I use the v2 patch startup stops with the following: {code} INFO [main] 2012-06-12 14:23:33,899 ColumnFamilyStore.java (line 633) Enqueuing flush of Memtable-LocationInfo@1141455324(41/51 serialized/live bytes, 1 ops) INFO [FlushWriter:2] 2012-06-12 14:23:33,903 Memtable.java (line 266) Writing Memtable-LocationInfo@1141455324(41/51 serialized/live bytes, 1 ops) ERROR [FlushWriter:2] 2012-06-12 14:23:33,953 AbstractCassandraDaemon.java (line 134) Exception in thread Thread[FlushWriter:2,5,main]java.lang.RuntimeException: Last written key null >= current key DecoratedKey(61078635599166706937511052402724559481, 4c) writing into /var/lib//cassandra/system/LocationInfo/system-LocationInfo-tmp-hd-65597-Data.db {code} Given the above I (probably incorrectly) scrubbed the system keyspace which removed all sstables, leaving only the snapshots eg: {code} WARN [CompactionExecutor:5] 2012-06-12 14:29:41,672 CompactionManager.java (line 651) Row at 100 is unreadable; skipping to next WARN [CompactionExecutor:5] 2012-06-12 14:29:41,672 CompactionManager.java (line 602) Non-fatal error reading row (stacktrace follows) java.lang.RuntimeException: Last written key null >= current key DecoratedKey(135285944860343992175601105924967452217, 63716c) writing into /var/lib//data/cassandra/system/Versions/system-Versions-tmp-hd-37-Data.db {code} ..eventually resulting in {code} WARN [CompactionExecutor:5] 2012-06-12 14:29:41,674 CompactionManager.java (line 692) No valid rows found while scrubbing SSTableReader(path='/var/lib//data/cassandra/system/Versions/system-Versions-hd-35-Data.db'); it is marked for deletion now. If you want to attempt manual recovery, you can find a copy in the pre-scrub snapshot {code} A clean bootstrap also stops with similar errors: {code} java.lang.RuntimeException: Last written key null >= current key DecoratedKey(61078635599166706937511052402724559481, 4c) writing into /var/lib//data/cassandra/system/LocationInfo/system-LocationInfo-tmp-hd-1-Data.db {code} and {code} java.lang.RuntimeException: Last written key null >= current key DecoratedKey(93220794208128599841715671226150005828, 746872696674) writing into /var/lib//data/cassandra/system/Versions/system-Versions-tmp-hd-1-Data.db {code} was (Author: awinter): If I use the v2 patch startup stops with the following: {code} INFO [main] 2012-06-12 14:23:33,899 ColumnFamilyStore.java (line 633) Enqueuing flush of Memtable-LocationInfo@1141455324(41/51 serialized/live bytes, 1 ops) INFO [FlushWriter:2] 2012-06-12 14:23:33,903 Memtable.java (line 266) Writing Memtable-LocationInfo@1141455324(41/51 serialized/live bytes, 1 ops) ERROR [FlushWriter:2] 2012-06-12 14:23:33,953 AbstractCassandraDaemon.java (line 134) Exception in thread Thread[FlushWriter:2,5,main]java.lang.RuntimeException: Last written key null >= current key DecoratedKey(61078635599166706937511052402724559481, 4c) writing into /var/lib//cassandra/system/LocationInfo/system-LocationInfo-tmp-hd-65597-Data.db {code} Given the above I scrubbed the system keyspace which removed all sstables, leaving only the snapshots eg: {code} WARN [CompactionExecutor:5] 2012-06-12 14:29:41,672 CompactionManager.java (line 651) Row at 100 is unreadable; skipping to next WARN [CompactionExecutor:5] 2012-06-12 14:29:41,672 CompactionManager.java (line 602) Non-fatal error reading row (stacktrace follows) java.lang.RuntimeException: Last written key null >= current key DecoratedKey(135285944860343992175601105924967452217, 63716c) writing into /var/lib//data/cassandra/system/Versions/system-Versions-tmp-hd-37-Data.db {code} ..eventually resulting in {code} WARN [CompactionExecutor:5] 2012-06-12 14:29:41,674 CompactionManager.java (line 692) No valid rows found while scrubbing SSTableReader(path='/var/lib//data/cassandra/system/Versions/system-Versions-hd-35-Data.db'); it is marked for deletion now. If you want to attempt manual recovery, you can find a copy in the pre-scrub snapshot {code} A clean bootstrap also stops with similar errors: {code} java.lang.RuntimeException: Last written key null >= current key DecoratedKey(61078635599166706937511052402724559481, 4c) writing into /var/lib//data/cassandra/system/LocationInfo/system-LocationInfo-tmp-hd-1-Data.db {code} and {code} java.lang.RuntimeException: Last written key null >= current key DecoratedKey(93220794208128599841715671226150005828, 746872696674) writing into /var/lib//data/cassandra/system/Versions/system-Versions-tmp-hd-1-Data.db {code} > stackoverflow building interval tree & possible sstable corrup
[jira] [Comment Edited] (CASSANDRA-4321) stackoverflow building interval tree & possible sstable corruptions
[ https://issues.apache.org/jira/browse/CASSANDRA-4321?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13292773#comment-13292773 ] Jonathan Ellis edited comment on CASSANDRA-4321 at 6/11/12 12:28 PM: - Omid, just to verify -- are you also using LeveledCompactionStrategy? was (Author: jbellis): Omid and Javier, are you also using LeveledCompactionStrategy? > stackoverflow building interval tree & possible sstable corruptions > --- > > Key: CASSANDRA-4321 > URL: https://issues.apache.org/jira/browse/CASSANDRA-4321 > Project: Cassandra > Issue Type: Bug > Components: Core >Affects Versions: 1.1.1 >Reporter: Anton Winter > > After upgrading to 1.1.1 (from 1.1.0) I have started experiencing > StackOverflowError's resulting in compaction backlog and failure to restart. > The ring currently consists of 6 DC's and 22 nodes using LCS & compression. > This issue was first noted on 2 nodes in one DC and then appears to have > spread to various other nodes in the other DC's. > When the first occurrence of this was found I restarted the instance but it > failed to start so I cleared its data and treated it as a replacement node > for the token it was previously responsible for. This node successfully > streamed all the relevant data back but failed again a number of hours later > with the same StackOverflowError and again was unable to restart. > The initial stack overflow error on a running instance looks like this: > ERROR [CompactionExecutor:314] 2012-06-07 09:59:43,017 > AbstractCassandraDaemon.java (line 134) Exception in thread > Thread[CompactionExecutor:314,1,main] > java.lang.StackOverflowError > at java.util.Arrays.mergeSort(Arrays.java:1157) > at java.util.Arrays.sort(Arrays.java:1092) > at java.util.Collections.sort(Collections.java:134) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.findMinMedianMax(IntervalNode.java:114) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:49) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > [snip - this repeats until stack overflow. Compactions stop from this point > onwards] > I restarted this failing instance with DEBUG logging enabled and it throws > the following exception part way through startup: > ERROR 11:37:51,046 Exception in thread Thread[OptionalTasks:1,5,main] > java.lang.StackOverflowError > at > org.slf4j.helpers.MessageFormatter.safeObjectAppend(MessageFormatter.java:307) > at > org.slf4j.helpers.MessageFormatter.deeplyAppendParameter(MessageFormatter.java:276) > at > org.slf4j.helpers.MessageFormatter.arrayFormat(MessageFormatter.java:230) > at > org.slf4j.helpers.MessageFormatter.format(MessageFormatter.java:124) > at > org.slf4j.impl.Log4jLoggerAdapter.debug(Log4jLoggerAdapter.java:228) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:45) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > [snip - this repeats until stack overflow] > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalTree.(IntervalTree.java:39) > at > org.apache.cassandra.db.DataTracker.buildIntervalTree(DataTracker.java:560) > at > org.apache.cassandra.db.DataTracker$View.replace(DataTracker.java:617) > at org.apache.cass
[jira] [Comment Edited] (CASSANDRA-4321) stackoverflow building interval tree & possible sstable corruptions
[ https://issues.apache.org/jira/browse/CASSANDRA-4321?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13291958#comment-13291958 ] Omid Aladini edited comment on CASSANDRA-4321 at 6/8/12 9:45 PM: - We're seeing the same issue after upgrading from 1.0.9 to 1.1.1 on only a single node in a 16 node cluster. Wiping the data off and bootstrapping again didn't help. Compaction looks to be not progressing (according to compactionstats) and I can reproduce this on every "nodetool flush". was (Author: omid): We're see the same issue after upgrading from 1.0.9 to 1.1.1 on only a single node in a 16 node cluster. Wiping the data off and bootstrapping again didn't help. Compaction looks to be not progressing (according to compactionstats) and I can reproduce this on every "nodetool flush". > stackoverflow building interval tree & possible sstable corruptions > --- > > Key: CASSANDRA-4321 > URL: https://issues.apache.org/jira/browse/CASSANDRA-4321 > Project: Cassandra > Issue Type: Bug > Components: Core >Affects Versions: 1.1.1 >Reporter: Anton Winter > > After upgrading to 1.1.1 (from 1.1.0) I have started experiencing > StackOverflowError's resulting in compaction backlog and failure to restart. > The ring currently consists of 6 DC's and 22 nodes using LCS & compression. > This issue was first noted on 2 nodes in one DC and then appears to have > spread to various other nodes in the other DC's. > When the first occurrence of this was found I restarted the instance but it > failed to start so I cleared its data and treated it as a replacement node > for the token it was previously responsible for. This node successfully > streamed all the relevant data back but failed again a number of hours later > with the same StackOverflowError and again was unable to restart. > The initial stack overflow error on a running instance looks like this: > ERROR [CompactionExecutor:314] 2012-06-07 09:59:43,017 > AbstractCassandraDaemon.java (line 134) Exception in thread > Thread[CompactionExecutor:314,1,main] > java.lang.StackOverflowError > at java.util.Arrays.mergeSort(Arrays.java:1157) > at java.util.Arrays.sort(Arrays.java:1092) > at java.util.Collections.sort(Collections.java:134) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.findMinMedianMax(IntervalNode.java:114) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:49) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > [snip - this repeats until stack overflow. Compactions stop from this point > onwards] > I restarted this failing instance with DEBUG logging enabled and it throws > the following exception part way through startup: > ERROR 11:37:51,046 Exception in thread Thread[OptionalTasks:1,5,main] > java.lang.StackOverflowError > at > org.slf4j.helpers.MessageFormatter.safeObjectAppend(MessageFormatter.java:307) > at > org.slf4j.helpers.MessageFormatter.deeplyAppendParameter(MessageFormatter.java:276) > at > org.slf4j.helpers.MessageFormatter.arrayFormat(MessageFormatter.java:230) > at > org.slf4j.helpers.MessageFormatter.format(MessageFormatter.java:124) > at > org.slf4j.impl.Log4jLoggerAdapter.debug(Log4jLoggerAdapter.java:228) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:45) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > [snip - this repeats until stack overflow] > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64) > at > org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62) > at > org.apache.cassandra.utils.Interva