[
https://issues.apache.org/jira/browse/CASSANDRA-4321?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13293762#comment-13293762
]
Omid Aladini commented on CASSANDRA-4321:
-----------------------------------------
Tried the patch but the server still doesn't start. The StackOverFlow that gets
thrown causes an already loaded column family to be loaded again:
Load CF1:
{code}
reading saved cache /var/lib/cassandra/abcd/saved_caches/SOMEKSP-CF1-KeyCache
2012-06-12_16:18:04.12387 INFO 16:18:04,123 Opening
/var/lib/cassandra/abcd/data/SOMEKSP/CF1/SOMEKSP-CF1-hd-2248
...
{code}
Load CF3 which has the corrupted sstables
{code}
2012-06-12_15:31:20.56185 INFO 15:31:20,561 Opening
/var/lib/cassandra/abcd/data/SOMEKSP/CF3/SOMEKSP-CF3-hd-7924 (2372910 bytes)
2012-06-12_15:31:20.81897 ERROR 15:31:20,811 Exception in thread
Thread[OptionalTasks:1,5,main]
2012-06-12_15:31:20.81901 java.lang.StackOverflowError
2012-06-12_15:31:20.81901 at
org.apache.cassandra.db.DecoratedKey.compareTo(DecoratedKey.java:90)
2012-06-12_15:31:20.81906 at
org.apache.cassandra.db.DecoratedKey.compareTo(DecoratedKey.java:38)
2012-06-12_15:31:20.81918 at java.util.Arrays.mergeSort(Unknown Source)
2012-06-12_15:31:20.81927 at java.util.Arrays.mergeSort(Unknown Source)
2012-06-12_15:31:20.81934 at java.util.Arrays.mergeSort(Unknown Source)
2012-06-12_15:31:20.81940 at java.util.Arrays.mergeSort(Unknown Source)
2012-06-12_15:31:20.81946 at java.util.Arrays.mergeSort(Unknown Source)
2012-06-12_15:31:20.81954 at java.util.Arrays.sort(Unknown Source)
2012-06-12_15:31:20.81960 at java.util.Collections.sort(Unknown Source)
2012-06-12_15:31:20.81980 at
org.apache.cassandra.utils.IntervalTree.IntervalNode.findMinMedianMax(IntervalNode.java:114)
2012-06-12_15:31:20.81981 at
org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:49)
2012-06-12_15:31:20.81990 at
org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
// stacktrace goes on
2012-06-12_15:31:20.88633 at
org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:64)
2012-06-12_15:31:20.88643 at
org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:64)
2012-06-12_15:31:20.88654 at
org.apache.cassandra.utils.IntervalTree.IntervalTree.<init>(IntervalTree.java:39)
2012-06-12_15:31:20.88664 at
org.apache.cassandra.db.DataTracker.buildIntervalTree(DataTracker.java:560)
2012-06-12_15:31:20.88673 at
org.apache.cassandra.db.DataTracker$View.replace(DataTracker.java:617)
2012-06-12_15:31:20.88683 at
org.apache.cassandra.db.DataTracker.replace(DataTracker.java:320)
2012-06-12_15:31:20.88692 at
org.apache.cassandra.db.DataTracker.addInitialSSTables(DataTracker.java:259)
2012-06-12_15:31:20.88702 at
org.apache.cassandra.db.ColumnFamilyStore.<init>(ColumnFamilyStore.java:234)
2012-06-12_15:31:20.88712 at
org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:331)
2012-06-12_15:31:20.88723 at
org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:309)
2012-06-12_15:31:20.88734 at
org.apache.cassandra.db.Table.initCf(Table.java:367)
2012-06-12_15:31:20.88742 at
org.apache.cassandra.db.Table.<init>(Table.java:299)
2012-06-12_15:31:20.88750 at
org.apache.cassandra.db.Table.open(Table.java:114)
2012-06-12_15:31:20.88758 at
org.apache.cassandra.db.Table.open(Table.java:97)
2012-06-12_15:31:20.88766 at
org.apache.cassandra.db.Table$2.apply(Table.java:574)
2012-06-12_15:31:20.88773 at
org.apache.cassandra.db.Table$2.apply(Table.java:571)
2012-06-12_15:31:20.88782 at
com.google.common.collect.Iterators$8.next(Iterators.java:751)
2012-06-12_15:31:20.88790 at
org.apache.cassandra.db.ColumnFamilyStore.all(ColumnFamilyStore.java:1625)
2012-06-12_15:31:20.88800 at
org.apache.cassandra.db.MeteredFlusher.countFlushingBytes(MeteredFlusher.java:118)
2012-06-12_15:31:20.88810 at
org.apache.cassandra.db.MeteredFlusher.run(MeteredFlusher.java:45)
2012-06-12_15:31:20.88818 at
org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run(DebuggableScheduledThreadPoolExecutor.java:79)
2012-06-12_15:31:20.88833 at
java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)
2012-06-12_15:31:20.88842 at
java.util.concurrent.FutureTask$Sync.innerRunAndReset(Unknown Source)
2012-06-12_15:31:20.88851 at
java.util.concurrent.FutureTask.runAndReset(Unknown Source)
2012-06-12_15:31:20.88860 at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$101(Unknown
Source)
2012-06-12_15:31:20.88870 at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.runPeriodic(Unknown
Source)
2012-06-12_15:31:20.88882 at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(Unknown
Source)
2012-06-12_15:31:20.88892 at
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown Source)
2012-06-12_15:31:20.88901 at
java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
2012-06-12_15:31:20.88910 at java.lang.Thread.run(Unknown Source)
{code}
Then it tries to load CF1 again:
{code}
2012-06-12_15:33:52.92593 INFO 15:33:52,925 Opening
/var/lib/cassandra/abcd/data/SOMEKSP/CF1/SOMEKSP-CF1-hd-3578 (2528503 bytes)
2012-06-12_15:33:52.92792 INFO 15:33:52,927 Opening
/var/lib/cassandra/abcd/data/SOMEKSP/CF1/SOMEKSP-CF1-hd-2613 (3374796 bytes)
{code}
Therefore the server fails with the following exception:
{code}
2012-06-12_15:33:53.17913 ERROR 15:33:53,178 Exception encountered during
startup
2012-06-12_15:33:53.17919 java.lang.RuntimeException:
javax.management.InstanceAlreadyExistsException:
org.apache.cassandra.db:type=ColumnFamilies,keyspace=SOMEKSP,columnfamily=CF1
2012-06-12_15:33:53.17934 at
org.apache.cassandra.db.ColumnFamilyStore.<init>(ColumnFamilyStore.java:257)
2012-06-12_15:33:53.17940 at
org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:331)
2012-06-12_15:33:53.17948 at
org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:309)
2012-06-12_15:33:53.17956 at
org.apache.cassandra.db.Table.initCf(Table.java:367)
2012-06-12_15:33:53.17962 at
org.apache.cassandra.db.Table.<init>(Table.java:299)
2012-06-12_15:33:53.17967 at
org.apache.cassandra.db.Table.open(Table.java:114)
2012-06-12_15:33:53.17972 at
org.apache.cassandra.db.Table.open(Table.java:97)
2012-06-12_15:33:53.17979 at
org.apache.cassandra.service.AbstractCassandraDaemon.setup(AbstractCassandraDaemon.java:204)
2012-06-12_15:33:53.17987 at
org.apache.cassandra.service.AbstractCassandraDaemon.activate(AbstractCassandraDaemon.java:353)
2012-06-12_15:33:53.17996 at
org.apache.cassandra.thrift.CassandraDaemon.main(CassandraDaemon.java:106)
2012-06-12_15:33:53.18002 Caused by:
javax.management.InstanceAlreadyExistsException:
org.apache.cassandra.db:type=ColumnFamilies,keyspace=SOMEKSP,columnfamily=CF1
2012-06-12_15:33:53.18013 at
com.sun.jmx.mbeanserver.Repository.addMBean(Unknown Source)
2012-06-12_15:33:53.18019 at
com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.internal_addObject(Unknown
Source)
2012-06-12_15:33:53.18028 at
com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.registerDynamicMBean(Unknown
Source)
2012-06-12_15:33:53.18038 at
com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.registerObject(Unknown
Source)
2012-06-12_15:33:53.18045 at
com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.registerMBean(Unknown
Source)
2012-06-12_15:33:53.18053 at
com.sun.jmx.mbeanserver.JmxMBeanServer.registerMBean(Unknown Source)
2012-06-12_15:33:53.18060 at
org.apache.cassandra.db.ColumnFamilyStore.<init>(ColumnFamilyStore.java:253)
2012-06-12_15:33:53.18067 ... 9 more
2012-06-12_15:33:53.18069 java.lang.RuntimeException:
javax.management.InstanceAlreadyExistsException:
org.apache.cassandra.db:type=ColumnFamilies,keyspace=SOMEKSP,columnfamily=CF1
2012-06-12_15:33:53.18083 at
org.apache.cassandra.db.ColumnFamilyStore.<init>(ColumnFamilyStore.java:257)
2012-06-12_15:33:53.18092 at
org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:331)
2012-06-12_15:33:53.18100 at
org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:309)
2012-06-12_15:33:53.18109 at
org.apache.cassandra.db.Table.initCf(Table.java:367)
2012-06-12_15:33:53.18114 at
org.apache.cassandra.db.Table.<init>(Table.java:299)
2012-06-12_15:33:53.18119 at
org.apache.cassandra.db.Table.open(Table.java:114)
2012-06-12_15:33:53.18124 at
org.apache.cassandra.db.Table.open(Table.java:97)
2012-06-12_15:33:53.18129 at
org.apache.cassandra.service.AbstractCassandraDaemon.setup(AbstractCassandraDaemon.java:204)
2012-06-12_15:33:53.18139 at
org.apache.cassandra.service.AbstractCassandraDaemon.activate(AbstractCassandraDaemon.java:353)
2012-06-12_15:33:53.18148 at
org.apache.cassandra.thrift.CassandraDaemon.main(CassandraDaemon.java:106)
2012-06-12_15:33:53.18155 Caused by:
javax.management.InstanceAlreadyExistsException:
org.apache.cassandra.db:type=ColumnFamilies,keyspace=SOMEKSP,columnfamily=CF1
2012-06-12_15:33:53.18167 at
com.sun.jmx.mbeanserver.Repository.addMBean(Unknown Source)
2012-06-12_15:33:53.18173 at
com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.internal_addObject(Unknown
Source)
2012-06-12_15:33:53.18181 at
com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.registerDynamicMBean(Unknown
Source)
2012-06-12_15:33:53.18191 at
com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.registerObject(Unknown
Source)
2012-06-12_15:33:53.18198 at
com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.registerMBean(Unknown
Source)
2012-06-12_15:33:53.18206 at
com.sun.jmx.mbeanserver.JmxMBeanServer.registerMBean(Unknown Source)
2012-06-12_15:33:53.18212 at
org.apache.cassandra.db.ColumnFamilyStore.<init>(ColumnFamilyStore.java:253)
2012-06-12_15:33:53.18219 ... 9 more
2012-06-12_15:33:53.18221 Exception encountered during startup:
javax.management.InstanceAlreadyExistsException:
org.apache.cassandra.db:type=ColumnFamilies,keyspace=SOMEKSP,columnfamily=CF1
{code}
Enabling assertions though causes the corrupted sstables to be ignored:
{code}
2012-06-12_16:25:43.32075 INFO 16:25:43,320 Opening
/var/lib/cassandra/abcd/data/SOMEKSP/CF3/SOMEKSP-CF3-hc-6965 (2105724 bytes)
2012-06-12_16:25:43.32562 ERROR 16:25:43,325 Exception in thread
Thread[SSTableBatchOpen:1,5,main]
2012-06-12_16:25:43.32577 java.lang.AssertionError: SSTable first key
DecoratedKey(41255474878128469814942789647212295629,
31303132393937357c3337313730333536) > last key
DecoratedKey(41219536226656199861610796307350537953,
31303234323538397c3331383436373338)
2012-06-12_16:25:43.32614 at
org.apache.cassandra.io.sstable.SSTableReader.load(SSTableReader.java:412)
2012-06-12_16:25:43.32626 at
org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:187)
2012-06-12_16:25:43.32638 at
org.apache.cassandra.io.sstable.SSTableReader$1.run(SSTableReader.java:225)
2012-06-12_16:25:43.32651 at
java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)
2012-06-12_16:25:43.32662 at
java.util.concurrent.FutureTask$Sync.innerRun(Unknown Source)
2012-06-12_16:25:43.32672 at java.util.concurrent.FutureTask.run(Unknown
Source)
2012-06-12_16:25:43.32681 at
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown Source)
2012-06-12_16:25:43.32692 at
java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
2012-06-12_16:25:43.32703 at java.lang.Thread.run(Unknown Source)
{code}
which leads to cassandra booting up. I wonder if scrub will pick up the ignored
sstables.
Shouldn't the assertion above (SSTable first key > last key) turn into an
exception and get handled properly?
> 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-v2.txt,
> 0001-Change-Range-Bounds-in-LeveledManifest.overlapping-v3.txt,
> 0001-Change-Range-Bounds-in-LeveledManifest.overlapping.txt,
> 0002-Scrub-detects-and-repair-outOfOrder-rows.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.<init>(IntervalNode.java:49)
> at
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
> at
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
> at
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(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.<init>(IntervalNode.java:45)
> at
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
> at
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
> [snip - this repeats until stack overflow]
> at
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
> at
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:64)
> at
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
> at
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
> at
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:64)
> at
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
> at
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
> at
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:64)
> at
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
> at
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
> at
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
> at
> org.apache.cassandra.utils.IntervalTree.IntervalTree.<init>(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.cassandra.db.DataTracker.replace(DataTracker.java:320)
> at
> org.apache.cassandra.db.DataTracker.addInitialSSTables(DataTracker.java:259)
> at
> org.apache.cassandra.db.ColumnFamilyStore.<init>(ColumnFamilyStore.java:234)
> at
> org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:331)
> at
> org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:309)
> at org.apache.cassandra.db.Table.initCf(Table.java:367)
> at org.apache.cassandra.db.Table.<init>(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(ColumnFamilyStore.java:1625)
> at
> org.apache.cassandra.db.MeteredFlusher.countFlushingBytes(MeteredFlusher.java:118)
> at org.apache.cassandra.db.MeteredFlusher.run(MeteredFlusher.java:45)
> at
> org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run(DebuggableScheduledThreadPoolExecutor.java:79)
> at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
> at
> java.util.concurrent.FutureTask$Sync.innerRunAndReset(FutureTask.java:351)
> at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:178)
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:165)
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:267)
> 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)
> 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(104860264640932324846851821824650966808,
> 4fcc88eb0218216164673394),
> DecoratedKey(93975306025956344620001177071135439009,
> 4fc8fb042c98458c7a58bc3b)),
> Interval(DecoratedKey(104860264640932324846851821824650966808,
> 4fcc88eb0218216164673394),
> DecoratedKey(93975306025956344620001177071135439009,
> 4fc8fb042c98458c7a58bc3b)),
> Interval(DecoratedKey(104860264640932324846851821824650966808,
> 4fcc88eb0218216164673394),
> DecoratedKey(93975306025956344620001177071135439009,
> 4fc8fb042c98458c7a58bc3b)),
> Interval(DecoratedKey(104860264640932324846851821824650966808,
> 4fcc88eb0218216164673394),
> DecoratedKey(93975306025956344620001177071135439009,
> 4fc8fb042c98458c7a58bc3b)),
> Interval(DecoratedKey(104860264640932324846851821824650966808,
> 4fcc88eb0218216164673394),
> DecoratedKey(93975306025956344620001177071135439009,
> 4fc8fb042c98458c7a58bc3b)),
> Interval(DecoratedKey(104860264640932324846851821824650966808,
> 4fcc88eb0218216164673394),
> DecoratedKey(93975306025956344620001177071135439009,
> 4fc8fb042c98458c7a58bc3b))]
> 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(NativeMethodAccessorImpl.java:57)
> at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.lang.reflect.Method.invoke(Method.java:616)
> at
> org.apache.commons.daemon.support.DaemonLoader.load(DaemonLoader.java:160)
> Caused by: 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.<init>(IntervalNode.java:45)
> at
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
> at
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
> [snip - this repeats until stack overflow]
> at
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
> at
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
> at
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
> at
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:64)
> at
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
> at
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
> at
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:64)
> at
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
> at
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
> at
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:64)
> at
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
> at
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
> at
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
> at
> org.apache.cassandra.utils.IntervalTree.IntervalTree.<init>(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.cassandra.db.DataTracker.replace(DataTracker.java:320)
> at
> org.apache.cassandra.db.DataTracker.addInitialSSTables(DataTracker.java:259)
> at
> org.apache.cassandra.db.ColumnFamilyStore.<init>(ColumnFamilyStore.java:234)
> at
> org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:331)
> at
> org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:309)
> at org.apache.cassandra.db.Table.initCf(Table.java:367)
> at org.apache.cassandra.db.Table.<init>(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(AbstractCassandraDaemon.java:204)
> at
> org.apache.cassandra.service.AbstractCassandraDaemon.init(AbstractCassandraDaemon.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]java.lang.AssertionError: SSTable first key
> DecoratedKey(100294972947100949193477090306072672386,
> 4fcf051ef5067d7f17d9fc35) > last key
> DecoratedKey(90250429663386465697464050082134975058, 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.java: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(81958437188197992567937826278457419048,
> 4fa1aebad23f81e4321d344d) >= current key
> DecoratedKey(64546479828744423263742604083767363606,
> 4fcafc0f19f6a8092d4d4f94) writing into
> /var/lib/XX/data/cassandra/ks1/cf1/ks1-cf1-tmp-hd-657317-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: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)
> Just like the initial errors compactions appear to stop occurring after this
> point.
> Given the above this looks like sstables are getting corrupted. By
> restarting nodes I am able to identify several hundred sstables exhibiting
> the same 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
> validating sstables for correctness? Given that it has spread to various
> servers 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 administrators:
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira