Return-Path: X-Original-To: apmail-cassandra-commits-archive@www.apache.org Delivered-To: apmail-cassandra-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 4A4099B4D for ; Wed, 20 Jun 2012 11:46:45 +0000 (UTC) Received: (qmail 12787 invoked by uid 500); 20 Jun 2012 11:46:45 -0000 Delivered-To: apmail-cassandra-commits-archive@cassandra.apache.org Received: (qmail 12627 invoked by uid 500); 20 Jun 2012 11:46:44 -0000 Mailing-List: contact commits-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@cassandra.apache.org Delivered-To: mailing list commits@cassandra.apache.org Received: (qmail 12539 invoked by uid 99); 20 Jun 2012 11:46:43 -0000 Received: from issues-vm.apache.org (HELO issues-vm) (140.211.11.160) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 20 Jun 2012 11:46:43 +0000 Received: from isssues-vm.apache.org (localhost [127.0.0.1]) by issues-vm (Postfix) with ESMTP id 8AEE61427F2 for ; Wed, 20 Jun 2012 11:46:43 +0000 (UTC) Date: Wed, 20 Jun 2012 11:46:43 +0000 (UTC) From: "Omid Aladini (JIRA)" To: commits@cassandra.apache.org Message-ID: <971464125.33572.1340192803573.JavaMail.jiratomcat@issues-vm> In-Reply-To: <1233198175.51084.1339120943319.JavaMail.jiratomcat@issues-vm> Subject: [jira] [Commented] (CASSANDRA-4321) stackoverflow building interval tree & possible sstable corruptions MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: quoted-printable X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/CASSANDRA-4321?page=3Dcom.atlas= sian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=3D= 13397437#comment-13397437 ]=20 Omid Aladini commented on CASSANDRA-4321: ----------------------------------------- The exceptions happens not quickly afterwards but after some rounds of comp= action on the CF. I had re-bootstrapped so there are tons of ~(1500) pendin= g compaction tasks. If I restart the node and compact the problem happens a= gain and I can reproduce it. I'll send you an email about the data. =20 > 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.overlapp= ing-v3.txt, 0002-Scrub-detects-and-repair-outOfOrder-rows-v3.txt, 0003-Crea= te-standalone-scrub-v3.txt, 0003-Create-standalone-scrub-v4.txt, ooyala-has= tur-stacktrace.txt > > > After upgrading to 1.1.1 (from 1.1.0) I have started experiencing StackOv= erflowError's resulting in compaction backlog and failure to restart.=20 > The ring currently consists of 6 DC's and 22 nodes using LCS & compressio= n. This issue was first noted on 2 nodes in one DC and then appears to hav= e spread to various other nodes in the other DC's. =20 > 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 no= de for the token it was previously responsible for. This node successfully= streamed all the relevant data back but failed again a number of hours lat= er with the same StackOverflowError and again was unable to restart.=20 > The initial stack overflow error on a running instance looks like this: > ERROR [CompactionExecutor:314] 2012-06-07 09:59:43,017 AbstractCassandraD= aemon.java (line 134) Exception in thread Thread[CompactionExecutor:314,1,m= ain] > 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.findMinMe= dianMax(IntervalNode.java:114) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:49) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:62) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:62) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:62) > [snip - this repeats until stack overflow. Compactions stop from this po= int onwards] > I restarted this failing instance with DEBUG logging enabled and it throw= s 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(MessageFor= matter.java:307) > at org.slf4j.helpers.MessageFormatter.deeplyAppendParameter(Messa= geFormatter.java:276) > at org.slf4j.helpers.MessageFormatter.arrayFormat(MessageFormatte= r.java:230) > at org.slf4j.helpers.MessageFormatter.format(MessageFormatter.jav= a:124) > at org.slf4j.impl.Log4jLoggerAdapter.debug(Log4jLoggerAdapter.jav= a:228) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:45) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:62) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:62) > [snip - this repeats until stack overflow] > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:62) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:64) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:62) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:62) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:64) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:62) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:62) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:64) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:62) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:62) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:62) > at org.apache.cassandra.utils.IntervalTree.IntervalTree.(In= tervalTree.java:39) > at org.apache.cassandra.db.DataTracker.buildIntervalTree(DataTrac= ker.java:560) > at org.apache.cassandra.db.DataTracker$View.replace(DataTracker.j= ava:617) > at org.apache.cassandra.db.DataTracker.replace(DataTracker.java:3= 20) > at org.apache.cassandra.db.DataTracker.addInitialSSTables(DataTra= cker.java:259) > at org.apache.cassandra.db.ColumnFamilyStore.(ColumnFamilyS= tore.java:234) > at org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilySt= ore(ColumnFamilyStore.java:331) > at org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilySt= ore(ColumnFamilyStore.java:309) > at org.apache.cassandra.db.Table.initCf(Table.java:367) > at org.apache.cassandra.db.Table.(Table.java:299) > at org.apache.cassandra.db.Table.open(Table.java:114) > at org.apache.cassandra.db.Table.open(Table.java:97) > at org.apache.cassandra.db.Table$2.apply(Table.java:574) > at org.apache.cassandra.db.Table$2.apply(Table.java:571) > at com.google.common.collect.Iterators$8.next(Iterators.java:751) > at org.apache.cassandra.db.ColumnFamilyStore.all(ColumnFamilyStor= e.java:1625) > at org.apache.cassandra.db.MeteredFlusher.countFlushingBytes(Mete= redFlusher.java:118) > at org.apache.cassandra.db.MeteredFlusher.run(MeteredFlusher.java= :45) > at org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolE= xecutor$UncomplainingRunnable.run(DebuggableScheduledThreadPoolExecutor.jav= a:79) > at java.util.concurrent.Executors$RunnableAdapter.call(Executors.= java:471) > at java.util.concurrent.FutureTask$Sync.innerRunAndReset(FutureTa= sk.java:351) > at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:17= 8) > at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutu= reTask.access$201(ScheduledThreadPoolExecutor.java:165) > at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutu= reTask.run(ScheduledThreadPoolExecutor.java:267) > at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolEx= ecutor.java:1110) > at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolE= xecutor.java:603) > at java.lang.Thread.run(Thread.java:636) > DEBUG 11:37:51,052 Initializing ksU.cfS > And then finally fails with the following: > DEBUG 11:49:03,752 Creating IntervalNode from [Interval(DecoratedKey(1048= 60264640932324846851821824650966808, 4fcc88eb0218216164673394), DecoratedKe= y(93975306025956344620001177071135439009, 4fc8fb042c98458c7a58bc3b)), Inter= val(DecoratedKey(104860264640932324846851821824650966808, 4fcc88eb021821616= 4673394), DecoratedKey(93975306025956344620001177071135439009, 4fc8fb042c98= 458c7a58bc3b)), Interval(DecoratedKey(1048602646409323248468518218246509668= 08, 4fcc88eb0218216164673394), DecoratedKey(9397530602595634462000117707113= 5439009, 4fc8fb042c98458c7a58bc3b)), Interval(DecoratedKey(1048602646409323= 24846851821824650966808, 4fcc88eb0218216164673394), DecoratedKey(9397530602= 5956344620001177071135439009, 4fc8fb042c98458c7a58bc3b)), Interval(Decorate= dKey(104860264640932324846851821824650966808, 4fcc88eb0218216164673394), De= coratedKey(93975306025956344620001177071135439009, 4fc8fb042c98458c7a58bc3b= )), Interval(DecoratedKey(104860264640932324846851821824650966808, 4fcc88eb= 0218216164673394), DecoratedKey(93975306025956344620001177071135439009, 4fc= 8fb042c98458c7a58bc3b))] > java.lang.reflect.InvocationTargetException > DEBUG 11:49:03,753 Configured datacenter replicas are dc1:2, dc2:2, dc3:2= , dc4:2, dc5:0, dc6:2, dc7:0, dc8:0, dc9:2 > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccess= orImpl.java:57) > at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMeth= odAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:616) > at org.apache.commons.daemon.support.DaemonLoader.load(DaemonLoad= er.java:160) > Caused by: java.lang.StackOverflowError > at org.slf4j.helpers.MessageFormatter.safeObjectAppend(MessageFor= matter.java:307) > at org.slf4j.helpers.MessageFormatter.deeplyAppendParameter(Messa= geFormatter.java:276) > at org.slf4j.helpers.MessageFormatter.arrayFormat(MessageFormatte= r.java:230) > at org.slf4j.helpers.MessageFormatter.format(MessageFormatter.jav= a:124) > at org.slf4j.impl.Log4jLoggerAdapter.debug(Log4jLoggerAdapter.jav= a:228) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:45) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:62) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:62) > [snip - this repeats until stack overflow] > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:62) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:62) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:62) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:64) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:62) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:62) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:64) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:62) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:62) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:64) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:62) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:62) > at org.apache.cassandra.utils.IntervalTree.IntervalNode.(In= tervalNode.java:62) > at org.apache.cassandra.utils.IntervalTree.IntervalTree.(In= tervalTree.java:39) > at org.apache.cassandra.db.DataTracker.buildIntervalTree(DataTrac= ker.java:560) > at org.apache.cassandra.db.DataTracker$View.replace(DataTracker.j= ava:617) > at org.apache.cassandra.db.DataTracker.replace(DataTracker.java:3= 20) > at org.apache.cassandra.db.DataTracker.addInitialSSTables(DataTra= cker.java:259) > at org.apache.cassandra.db.ColumnFamilyStore.(ColumnFamilyS= tore.java:234) > at org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilySt= ore(ColumnFamilyStore.java:331) > at org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilySt= ore(ColumnFamilyStore.java:309) > at org.apache.cassandra.db.Table.initCf(Table.java:367) > at org.apache.cassandra.db.Table.(Table.java:299) > at org.apache.cassandra.db.Table.open(Table.java:114) > at org.apache.cassandra.db.Table.open(Table.java:97) > at org.apache.cassandra.service.AbstractCassandraDaemon.setup(Abs= tractCassandraDaemon.java:204) > at org.apache.cassandra.service.AbstractCassandraDaemon.init(Abst= ractCassandraDaemon.java:254) > ... 5 more > Cannot load daemon > Service exit with a return value of 3 > Running with assertions enabled allows me to start the instance but when = doing so I get errors such as: > ERROR 01:22:22,753 Exception in thread Thread[SSTableBatchOpen:2,5,main]j= ava.lang.AssertionError: SSTable first key DecoratedKey(1002949729471009491= 93477090306072672386, 4fcf051ef5067d7f17d9fc35) > last key DecoratedKey(902= 50429663386465697464050082134975058, 4fce996e3c1eed8c4b17dd66) > at org.apache.cassandra.io.sstable.SSTableReader.load(SSTableReader.java:= 412) > at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:= 187) > at org.apache.cassandra.io.sstable.SSTableReader$1.run(SSTableReader.java= :225) > 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.j= ava:1110) > at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.= java:603) > at java.lang.Thread.run(Thread.java:636) > and: > ERROR 01:27:58,946 Exception in thread Thread[CompactionExecutor:9,1,main= ] > java.lang.AssertionError: Last written key DecoratedKey(81958437188197992= 567937826278457419048, 4fa1aebad23f81e4321d344d) >=3D current key Decorated= Key(64546479828744423263742604083767363606, 4fcafc0f19f6a8092d4d4f94) writi= ng into /var/lib/XX/data/cassandra/ks1/cf1/ks1-cf1-tmp-hd-657317-Data.db > at org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SST= ableWriter.java:134) > at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWr= iter.java:153) > at org.apache.cassandra.db.compaction.CompactionTask.execute(Comp= actionTask.java:159) > at org.apache.cassandra.db.compaction.LeveledCompactionTask.execu= te(LeveledCompactionTask.java:50) > at org.apache.cassandra.db.compaction.CompactionManager$1.runMayT= hrow(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(ThreadPoolEx= ecutor.java:1110) > at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolE= xecutor.java:603) > at java.lang.Thread.run(Thread.java:636) > Just like the initial errors compactions appear to stop occurring after t= his point. =20 > Given the above this looks like sstables are getting corrupted. By resta= rting nodes I am able to identify several hundred sstables exhibiting the s= ame problem and this appears to be growing. > I have tried scrubbing those affected nodes but the problem continues to = occur. If this is due to sstable corruptions is there another way of valid= ating sstables for correctness? Given that it has spread to various server= s in other DC's it looks like this is directly related to the 1.1.1 upgrade= recently performed on the ring. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrato= rs: https://issues.apache.org/jira/secure/ContactAdministrators!default.jsp= a For more information on JIRA, see: http://www.atlassian.com/software/jira