Yes, please open a ticket for the assertion error.  (Once JIRA is back up...)

On Wed, Aug 25, 2010 at 10:46 PM, Aaron Morton <aa...@thelastpickle.com> wrote:
> yes, starting the others made things a bit happier.
> Any thoughts on the assertion error that cause the startup to fail ? I've
> seen it a couple of times.
> Seems to be from this line in CommitLogHeader.java
> 157:             assert clHeader.cfDirtiedAt.size() <= clHeader.cfCount;
> Thanks
> Aaron
> On 26 Aug, 2010,at 03:25 PM, Jonathan Ellis <jbel...@gmail.com> wrote:
>
> the one node you restarted thinks it's the only node in the cluster.
> starting the others will fix that.
>
> On Wed, Aug 25, 2010 at 10:10 PM, Aaron Morton <aa...@thelastpickle.com>
> wrote:
>> 0.7.0-bet1 4 node clustered, i'd managed to get it into some sort of awful
>> state (i think by accidently creating to many clients, it was also
>> complaining about running out of file handles). Anyway I killed it all and
>> restarted just one node, thought I would let it settle down then start the
>> others. On the first node I got this.
>> (Sorry I cannot be more specific, was not paying too much attention it it
>> all went bang)
>> I managed to get a couple of errors, one of which shutdown the server.
>> Just
>> checking before putting them into Jira, should I split them up?
>> ERROR [pool-1-thread-29] 2010-08-26 14:58:20,021 Cassandra.java (line
>> 2651)
>> Internal error processing get_slice
>> java.lang.IllegalStateException: replication factor (3) exceeds number of
>> endpoints (1)
>>         at
>>
>> org.apache.cassandra.locator.RackUnawareStrategy.calculateNaturalEndpoints(RackUnawareStrategy.java:57)
>>         at
>>
>> org.apache.cassandra.locator.AbstractReplicationStrategy.getNaturalEndpoints(AbstractReplicationStrategy.java:88)
>>         at
>>
>> org.apache.cassandra.service.StorageService.getNaturalEndpoints(StorageService.java:1289)
>>         at
>>
>> org.apache.cassandra.service.StorageService.getNaturalEndpoints(StorageService.java:1277)
>>         at
>>
>> org.apache.cassandra.service.StorageService.findSuitableEndpoint(StorageService.java:1323)
>>         at
>>
>> org.apache.cassandra.service.StorageProxy.strongRead(StorageProxy.java:402)
>>         at
>>
>> org.apache.cassandra.service.StorageProxy.readProtocol(StorageProxy.java:302)
>>         at
>>
>> org.apache.cassandra.thrift.CassandraServer.readColumnFamily(CassandraServer.java:125)
>>         at
>>
>> org.apache.cassandra.thrift.CassandraServer.getSlice(CassandraServer.java:231)
>>         at
>>
>> org.apache.cassandra.thriftCassandraServer.multigetSliceInternal(CassandraServer.java:309)
>>         at
>>
>> org.apache.cassandra.thrift.CassandraServer.get_slice(CassandraServer.java:270)
>>         at
>>
>> org.apache.cassandra.thrift.Cassandra$Processor$get_slice.process(Cassandra.java:2643)
>>         at
>>
>> org.apache.cassandra.thrift.Cassandra$Processor.process(Cassandra.java:2499)
>>         at
>>
>> org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:167)
>>         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:619)
>>
>> So I started the other 3, and two suffered the error below which caused
>> the
>> process to shutdown...
>> ERROR [main] 2010-08-26 14:59:22,315 AbstractCassandraDaemonjava (line
>> 107)
>> Exception encountered during startup.
>> java.lang.RuntimeException: java.util.concurrentExecutionException:
>> java.lang.RuntimeException: java.lang.RuntimeException:
>> java.util.concurrent.ExecutionException: java.lang.AssertionError
>>         at
>> org.apache.cassandra.utils.FBUtilitieswaitOnFutures(FBUtilitiesjava:549)
>>         at
>> org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:339)
>>         at
>> orgapache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:174)
>>         at
>>
>> org.apache.cassandra.thrift.CassandraDaemon.setup(CassandraDaemon.java:120)
>>         at
>>
>> org.apache.cassandra.service.AbstractCassandraDaemon.activate(AbstractCassandraDaemon.java:90)
>>         at
>> org.apache.cassandra.thrift.CassandraDaemon.main(CassandraDaemon.java:224)
>> Caused by: java.util.concurrent.ExecutionException:
>> java.lang.RuntimeException: java.lang.RuntimeException:
>> java.util.concurrent.ExecutionException: java.lang.AssertionError
>>         at
>> java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:222)
>>         at java.util.concurrent.FutureTask.get(FutureTask.java:83)
>>         at
>> org.apache.cassandra.utils.FBUtilities.waitOnFutures(FBUtilities.java:545)
>>         ... 5 more
>> Caused by: java.lang.RuntimeException: java.lang.RuntimeException:
>> java.util.concurrent.ExecutionException: java.lang.AssertionError
>>         at
>> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
>>         at
>> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
>>         at
>> java.util.concurrentFutureTask$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:619)
>> Caused by: java.lang.RuntimeException:
>> java.util.concurrent.ExecutionException: java.lang.AssertionError
>>         at
>>
>> org.apache.cassandra.db.commitlog.CommitLog.discardCompletedSegments(CommitLog.java:408)
>>         at
>>
>> org.apache.cassandra.dbColumnFamilyStore$2.runMayThrow(ColumnFamilyStore.java:445)
>>         at
>> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
>>         .. 6 more
>> Caused by: java.util.concurrent.ExecutionException:
>> java.lang.AssertionError
>>         at
>> java.util.concurrent.FutureTask$Sync.innerGet(FutureTaskjava:222)
>>         at java.util.concurrent.FutureTask.get(FutureTask.java:83)
>>         at
>>
>> org.apache.cassandra.db.commitlog.CommitLog.discardCompletedSegments(CommitLog.java:400)
>>         ... 8 more
>> Caused by: java.lang.AssertionError
>>         at
>>
>> org.apache.cassandra.db.commitlog.CommitLogHeader$CommitLogHeaderSerializer.serialize(CommitLogHeader.java:157)
>>         at
>>
>> org.apache.cassandra.db.commitlog.CommitLogHeader.writeCommitLogHeader(CommitLogHeader.java:124)
>>         at
>>
>> org.apache.cassandra.db.commitlog.CommitLogSegment.writeHeader(CommitLogSegment.java:70)
>>         at
>>
>> org.apache.cassandra.db.commitlog.CommitLog.discardCompletedSegmentsInternal(CommitLog.java:450)
>>         at
>> org.apache.cassandra.db.commitlog.CommitLog.access$300(CommitLog.java:75)
>>         at
>> org.apache.cassandra.db.commitlog.CommitLog$6.call(CommitLog.java:394)
>>         at
>> java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>>         at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>>         at
>>
>> org.apache.cassandra.db.commitlog.PeriodicCommitLogExecutorService$1.runMayThrow(PeriodicCommitLogExecutorService.java:52)
>>         at
>> orgapache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
>>         ... 1 more
>>
>> Aaron
>>
>
>
>
> --
> Jonathan Ellis
> Project Chair, Apache Cassandra
> co-founder of Riptano, the source for professional Cassandra support
> http://riptano.com
>



-- 
Jonathan Ellis
Project Chair, Apache Cassandra
co-founder of Riptano, the source for professional Cassandra support
http://riptano.com

Reply via email to