[jira] [Commented] (CASSANDRA-4411) Assertion with LCS compaction

2012-08-09 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4411?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13431857#comment-13431857
 ] 

Jonathan Ellis commented on CASSANDRA-4411:
---

done in 115f380a86912e5918f534db2ec2935253909fad

 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
Assignee: Sylvain Lebresne
 Fix For: 1.1.3

 Attachments: 0001-Add-debugging-info-for-LCS.txt, 4411-followup.txt, 
 4411.txt, assertion-w-more-debugging-info-omid.log, 
 assertion.moreinfo.system.log, system.log


 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




[jira] [Commented] (CASSANDRA-4411) Assertion with LCS compaction

2012-08-09 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4411?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13431964#comment-13431964
 ] 

Jonathan Ellis commented on CASSANDRA-4411:
---

committed, thanks!

 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
Assignee: Sylvain Lebresne
 Fix For: 1.1.3

 Attachments: 0001-Add-debugging-info-for-LCS.txt, 
 0001-Fix-off-by-one-for-out-of-order-and-overlapping-ssta.patch, 
 4411-followup.txt, 4411.txt, assertion-w-more-debugging-info-omid.log, 
 assertion.moreinfo.system.log, system.log


 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




[jira] [Commented] (CASSANDRA-4411) Assertion with LCS compaction

2012-08-08 Thread Omid Aladini (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4411?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13431275#comment-13431275
 ] 

Omid Aladini commented on CASSANDRA-4411:
-

I can confirm that the problem is still there. I offline-scrubbed using 1.1.3 
(sstables were generated by 1.1.0) , but the scrubber did not report any 
out-of-order sstables, but sent some sstables back to L0. On compaction though, 
I get the exception:

{quote}
2012-08-08_18:15:41.85260 java.lang.RuntimeException: Last written key 
DecoratedKey(135076574692378869287086649376333921820, SOME_KEY_1) = current 
key DecoratedKey(135076574692378869287086649376333921820, SOME_KEY_1) writing 
into /var/lib/cassandra/abcd/data/KSP/CF1/KSP-CF1-tmp-he-178793-Data.db
2012-08-08_18:15:41.85303   at 
org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:134)
2012-08-08_18:15:41.85314   at 
org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:153)
2012-08-08_18:15:41.85326   at 
org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:159)
2012-08-08_18:15:41.85338   at 
org.apache.cassandra.db.compaction.LeveledCompactionTask.execute(LeveledCompactionTask.java:50)
2012-08-08_18:15:41.85351   at 
org.apache.cassandra.db.compaction.CompactionManager$6.runMayThrow(CompactionManager.java:288)
2012-08-08_18:15:41.85364   at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
2012-08-08_18:15:41.85375   at 
java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)
2012-08-08_18:15:41.85385   at 
java.util.concurrent.FutureTask$Sync.innerRun(Unknown Source)
2012-08-08_18:15:41.85395   at java.util.concurrent.FutureTask.run(Unknown 
Source)
2012-08-08_18:15:41.85403   at 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown Source)
2012-08-08_18:15:41.85414   at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
2012-08-08_18:15:41.85424   at java.lang.Thread.run(Unknown Source)
{quote}



 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
Assignee: Sylvain Lebresne
 Fix For: 1.1.3

 Attachments: 0001-Add-debugging-info-for-LCS.txt, 4411-followup.txt, 
 4411.txt, assertion-w-more-debugging-info-omid.log, 
 assertion.moreinfo.system.log, system.log


 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: 

[jira] [Commented] (CASSANDRA-4411) Assertion with LCS compaction

2012-08-08 Thread Omid Aladini (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4411?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13431295#comment-13431295
 ] 

Omid Aladini commented on CASSANDRA-4411:
-

Not sure, but from the fact that the two keys are identical, shouldn't the  
be = in:

https://github.com/apache/cassandra/blob/cassandra-1.1/src/java/org/apache/cassandra/db/compaction/Scrubber.java#L178

?

 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
Assignee: Sylvain Lebresne
 Fix For: 1.1.3

 Attachments: 0001-Add-debugging-info-for-LCS.txt, 4411-followup.txt, 
 4411.txt, assertion-w-more-debugging-info-omid.log, 
 assertion.moreinfo.system.log, system.log


 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




[jira] [Commented] (CASSANDRA-4411) Assertion with LCS compaction

2012-08-08 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4411?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13431431#comment-13431431
 ] 

Jonathan Ellis commented on CASSANDRA-4411:
---

I think you're right.  I'll push a fix for that shortly.

 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
Assignee: Sylvain Lebresne
 Fix For: 1.1.3

 Attachments: 0001-Add-debugging-info-for-LCS.txt, 4411-followup.txt, 
 4411.txt, assertion-w-more-debugging-info-omid.log, 
 assertion.moreinfo.system.log, system.log


 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




[jira] [Commented] (CASSANDRA-4411) Assertion with LCS compaction

2012-08-06 Thread Omid Aladini (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4411?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13429257#comment-13429257
 ] 

Omid Aladini commented on CASSANDRA-4411:
-

@Mina were the sstables created after CASSANDRA-4321 patch? Otherwise 
offline-scrub with --manifest-check is unlikely to solve the problem (or at 
least I don't understand how) since there would still be out-of-order sstables 
existing.

 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
Assignee: Sylvain Lebresne
 Fix For: 1.1.3

 Attachments: 0001-Add-debugging-info-for-LCS.txt, 4411-followup.txt, 
 4411.txt, assertion-w-more-debugging-info-omid.log, 
 assertion.moreinfo.system.log, system.log


 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




[jira] [Commented] (CASSANDRA-4411) Assertion with LCS compaction

2012-07-26 Thread Mina Naguib (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4411?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13423108#comment-13423108
 ] 

Mina Naguib commented on CASSANDRA-4411:


Quick follow-up

All the problematic nodes have been offline scrubbed (successfully using 
--manifest-check to speed things up).  There are no more compaction errors / 
pending compactions.

Like Anton, I'm a bit weary and keeping an eye on things - but so far so good.

On a tangent, it occurred to me that the amount of time it takes to run ( 
sstablescrub --manifest-check ) is mostly reading the sstables - the check 
itself and demoting the bad sstables to L0 appears very cheap - would it be a 
good idea to perform that check automatically on cassandra startup (after the 
sstables have been read) ?  It *may* be a quick fix for 1.1.3 to help people 
out who have been bitten by this but don't know it yet.



 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
Assignee: Sylvain Lebresne
 Fix For: 1.1.3

 Attachments: 0001-Add-debugging-info-for-LCS.txt, 4411-followup.txt, 
 4411.txt, assertion-w-more-debugging-info-omid.log, 
 assertion.moreinfo.system.log, system.log


 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




[jira] [Commented] (CASSANDRA-4411) Assertion with LCS compaction

2012-07-24 Thread Sylvain Lebresne (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4411?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13421233#comment-13421233
 ] 

Sylvain Lebresne commented on CASSANDRA-4411:
-

@Mina Did you run the offline scrub introduced with CASSANDRA-4321. Otherwise, 
it won't fix the problem. So you need to 1) shut down the node (this is 
important before running the offline scrub) and 2) run ./bin/sstablescrub. That 
last step should print some lines indicating having corrected some problems 
(otherwise, something is wrong with the scrubbing).

If after that you still get an exception, it might be helpful if you could run 
with 0001-Add-debugging-info-for-LCS.txt applied.

 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
Assignee: Sylvain Lebresne
 Fix For: 1.1.3

 Attachments: 0001-Add-debugging-info-for-LCS.txt, 4411-followup.txt, 
 4411.txt, assertion-w-more-debugging-info-omid.log, 
 assertion.moreinfo.system.log, system.log


 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




[jira] [Commented] (CASSANDRA-4411) Assertion with LCS compaction

2012-07-24 Thread Mina Naguib (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4411?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13421350#comment-13421350
 ] 

Mina Naguib commented on CASSANDRA-4411:


I ran the scrub in online mode.

I just took down a node and am now running it in offline mode.  Will report 
back.

BTW, the default sstablescrub does not respect the memory limits set in 
cassandra.in.sh, so it failed for me with:
{code}
Exception in thread main java.lang.OutOfMemoryError: Java heap space
at sun.security.provider.DigestBase.engineDigest(DigestBase.java:146)
at 
java.security.MessageDigest$Delegate.engineDigest(MessageDigest.java:546)
at java.security.MessageDigest.digest(MessageDigest.java:323)
at org.apache.cassandra.utils.FBUtilities.hash(FBUtilities.java:229)
at 
org.apache.cassandra.utils.FBUtilities.hashToBigInteger(FBUtilities.java:213)
at 
org.apache.cassandra.dht.RandomPartitioner.getToken(RandomPartitioner.java:154)
at 
org.apache.cassandra.dht.RandomPartitioner.decorateKey(RandomPartitioner.java:47)
at 
org.apache.cassandra.cache.AutoSavingCache.readSaved(AutoSavingCache.java:118)
at 
org.apache.cassandra.db.ColumnFamilyStore.init(ColumnFamilyStore.java:230)
at 
org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:341)
at 
org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:313)
at org.apache.cassandra.db.Table.initCf(Table.java:371)
at org.apache.cassandra.db.Table.init(Table.java:304)
at org.apache.cassandra.db.Table.open(Table.java:119)
at org.apache.cassandra.db.Table.openWithoutSSTables(Table.java:102)
at 
org.apache.cassandra.tools.StandaloneScrubber.main(StandaloneScrubber.java:65)
{code}

I edited it to update the hardocded limit of 256MB to a more reasonable value 
(the same as my cassandra.in.sh) to allow it to run without crashing.



 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
Assignee: Sylvain Lebresne
 Fix For: 1.1.3

 Attachments: 0001-Add-debugging-info-for-LCS.txt, 4411-followup.txt, 
 4411.txt, assertion-w-more-debugging-info-omid.log, 
 assertion.moreinfo.system.log, system.log


 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] [Commented] (CASSANDRA-4411) Assertion with LCS compaction

2012-07-24 Thread Mina Naguib (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4411?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13421487#comment-13421487
 ] 

Mina Naguib commented on CASSANDRA-4411:


Things appear better after an offline scrub.  While the scrubbing itself was 
uneventful, at the very end it did Checking leveled manifest, found 14 
sstables in level 3 and level 4 that were problematic and moved them back to 
level 0.

I started the node back up and all (+/- 10) compactions ran successfully.

I'll keep an eye on it and if it stays well I'll do the same to the other 
nodes.  Perhaps I'll try my luck with sstablescrub --manifest-check to see if I 
can keep the downtime to a minimum.

 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
Assignee: Sylvain Lebresne
 Fix For: 1.1.3

 Attachments: 0001-Add-debugging-info-for-LCS.txt, 4411-followup.txt, 
 4411.txt, assertion-w-more-debugging-info-omid.log, 
 assertion.moreinfo.system.log, system.log


 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




[jira] [Commented] (CASSANDRA-4411) Assertion with LCS compaction

2012-07-23 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4411?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13420838#comment-13420838
 ] 

Jonathan Ellis commented on CASSANDRA-4411:
---

The former.

 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
Assignee: Sylvain Lebresne
 Fix For: 1.1.3

 Attachments: 0001-Add-debugging-info-for-LCS.txt, 4411-followup.txt, 
 4411.txt, assertion-w-more-debugging-info-omid.log, 
 assertion.moreinfo.system.log, system.log


 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




[jira] [Commented] (CASSANDRA-4411) Assertion with LCS compaction

2012-07-23 Thread Mina Naguib (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4411?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13421049#comment-13421049
 ] 

Mina Naguib commented on CASSANDRA-4411:



Unfortunately the problem did not go away after scrubbing.

I scrubbed 2 of the problematic nodes.  Immediately after the scrub (5 hours) 
finished, a compaction was attempted and again failed:

{code}
ERROR [CompactionExecutor:47] 2012-07-23 19:48:52,500 
AbstractCassandraDaemon.java (line 134) Exception in thread 
Thread[CompactionExecutor:47,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:441)
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)
{code}

I've verified (using ls and the timestamps of *-Data.db sstables) that there 
are no old sstables and all of the sstables in the CF are the ones generated 
during the 5 hours of scrubbing.

I've also stopped and restarted one of these nodes, and again shortly after 
restart the compaction failed with a different stack trace:

{code}
java.lang.RuntimeException: Last written key 
DecoratedKey(225595347341523546110318866012608496, 
64313635626665302d333764372d313165302d393933622d303032366239333763386531) = 
current key DecoratedKey(221078382620949716286900834756484795, 
37303538643361662d616362662d343030312d313565382d633662303030303030336131) 
writing into /var/lib/cassandra/data/MYKS/MYCF/MYKS-MYCF-tmp-hd-520277-Data.db
at 
org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:134)
at 
org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:153)
at 
org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:159)
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:441)
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)
{code}

Is the fix for this ticket contained in some other code beyond the 2 posted 
patches (4411.txt, 4411-followup.txt) ?  That's what I'm running with on top of 
1.1.2.


 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
Assignee: Sylvain Lebresne
 Fix For: 1.1.3

 Attachments: 0001-Add-debugging-info-for-LCS.txt, 4411-followup.txt, 
 4411.txt, assertion-w-more-debugging-info-omid.log, 
 assertion.moreinfo.system.log, system.log


 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 
 

[jira] [Commented] (CASSANDRA-4411) Assertion with LCS compaction

2012-07-23 Thread Anton Winter (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4411?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13421061#comment-13421061
 ] 

Anton Winter commented on CASSANDRA-4411:
-

I can also confirm that after multiple offline sstablescrubs across all nodes 
that I still had several nodes (but not all) spread across multiple DC's still 
exhibiting this problem as described above by Mina.  

In an attempt to work around the problem I shut down the affected instances, 
deleted all data and re-bootstrapped them as if they were dead nodes.  Since 
doing so I haven't had the problem return however it is still early days.

 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
Assignee: Sylvain Lebresne
 Fix For: 1.1.3

 Attachments: 0001-Add-debugging-info-for-LCS.txt, 4411-followup.txt, 
 4411.txt, assertion-w-more-debugging-info-omid.log, 
 assertion.moreinfo.system.log, system.log


 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




[jira] [Commented] (CASSANDRA-4411) Assertion with LCS compaction

2012-07-18 Thread Omid Aladini (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4411?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13417024#comment-13417024
 ] 

Omid Aladini commented on CASSANDRA-4411:
-

Awesome. Thanks for the patch. Tested it and it works.

Sylvain, regarding your earlier comment on CASSANDRA-4321:

{quote} This is not really a new bug, but I believe that prior to 
CASSANDRA-4142, this had less consequences. {quote}

Does it mean LC-compacted SSTables created by 1.1.0 or earlier are as well 
affected and need to be scrubbed?

 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
Assignee: Sylvain Lebresne
 Fix For: 1.1.3

 Attachments: 0001-Add-debugging-info-for-LCS.txt, 4411-followup.txt, 
 4411.txt, assertion-w-more-debugging-info-omid.log, 
 assertion.moreinfo.system.log, system.log


 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




[jira] [Commented] (CASSANDRA-4411) Assertion with LCS compaction

2012-07-18 Thread Sylvain Lebresne (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4411?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13417234#comment-13417234
 ] 

Sylvain Lebresne commented on CASSANDRA-4411:
-

bq. Does it mean LC-compacted SSTables created by 1.1.0 or earlier are as well 
affected and need to be scrubbed?

Potentially, yes, unfortunately.

 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
Assignee: Sylvain Lebresne
 Fix For: 1.1.3

 Attachments: 0001-Add-debugging-info-for-LCS.txt, 4411-followup.txt, 
 4411.txt, assertion-w-more-debugging-info-omid.log, 
 assertion.moreinfo.system.log, system.log


 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




[jira] [Commented] (CASSANDRA-4411) Assertion with LCS compaction

2012-07-16 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4411?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13415060#comment-13415060
 ] 

Jonathan Ellis commented on CASSANDRA-4411:
---

+1

 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

 Attachments: 0001-Add-debugging-info-for-LCS.txt, 4411.txt, 
 assertion-w-more-debugging-info-omid.log, assertion.moreinfo.system.log, 
 system.log


 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




[jira] [Commented] (CASSANDRA-4411) Assertion with LCS compaction

2012-07-16 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4411?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13415214#comment-13415214
 ] 

Jonathan Ellis commented on CASSANDRA-4411:
---

+1

 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
Assignee: Sylvain Lebresne
 Fix For: 1.1.3

 Attachments: 0001-Add-debugging-info-for-LCS.txt, 4411-followup.txt, 
 4411.txt, assertion-w-more-debugging-info-omid.log, 
 assertion.moreinfo.system.log, system.log


 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




[jira] [Commented] (CASSANDRA-4411) Assertion with LCS compaction

2012-07-14 Thread Rudolf VanderLeeden (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4411?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13414429#comment-13414429
 ] 

Rudolf VanderLeeden commented on CASSANDRA-4411:


The file assertion.moreinfo.system.log has been attached. 

The assertion ERROR could be reproduced by doing the following:
(1) Bulk load 200 SSTables from a snapshot into a new 3-node cluster
(2) Run nodetool compact and repair
(3) Add ~500 SSTables from the same snapshot
(4) Run nodetool repair

RESULT: 
- we see immediately a StackOverflowError (from the repair command)
- after 2 minutes compaction starts
- after 11 minutes there is the AssertionError


 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

 Attachments: 0001-Add-debugging-info-for-LCS.txt, 
 assertion.moreinfo.system.log, system.log


 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




[jira] [Commented] (CASSANDRA-4411) Assertion with LCS compaction

2012-07-12 Thread Rudolf VanderLeeden (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4411?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13412683#comment-13412683
 ] 

Rudolf VanderLeeden commented on CASSANDRA-4411:


I attached the file assertion.system.log with DEBUG lines around the assertion 
ERROR.
This is just an excerpt from system.log which is much larger.

 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

 Attachments: assertion.system.log, system.log


 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




[jira] [Commented] (CASSANDRA-4411) Assertion with LCS compaction

2012-07-11 Thread Rudolf VanderLeeden (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4411?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=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 
 

[jira] [Commented] (CASSANDRA-4411) Assertion with LCS compaction

2012-07-10 Thread Christian Schnidrig (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-4411?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13410084#comment-13410084
 ] 

Christian Schnidrig commented on CASSANDRA-4411:


I got the same AssertionError on a 1.1.2 version cluster which I did not 
upgrade from an earlier version.

 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

 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