You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@cassandra.apache.org by Aaron Morton <aa...@thelastpickle.com> on 2010/08/26 05:10:09 UTC

0.7.0.bet1 errors during start up

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.cassandraservice.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.thrift.CassandraServer.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 AbstractCassandraDaemon.java (line 107) Exception encountered during startup.
java.lang.RuntimeException: java.util.concurrent.ExecutionException: java.lang.RuntimeException: java.lang.RuntimeException: java.util.concurrent.ExecutionException: java.lang.AssertionError
        at org.apache.cassandra.utils.FBUtilities.waitOnFutures(FBUtilities.java:549)
        at org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:339)
        at org.apache.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.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: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.db.ColumnFamilyStore$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(FutureTask.java:222)
        at java.util.concurrent.FutureTask.get(FutureTask.java:83)
        at org.apachecassandra.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 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
        ... 1 more


Aaron


Re: 0.7.0.bet1 errors during start up

Posted by Jonathan Ellis <jb...@gmail.com>.
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 <jb...@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

Re: 0.7.0.bet1 errors during start up

Posted by Aaron Morton <aa...@thelastpickle.com>.
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 <jb...@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
> orgapache.cassandra.service.StorageProxy.readProtocol(StorageProxy.java:302)
>         at
> org.apache.cassandra.thrift.CassandraServer.readColumnFamily(CassandraServer.java:125)
>         at
> org.apache.cassandra.thrift.CassandraServer.getSlice(CassandraServerjava:231)
>         at
> org.apache.cassandra.thrift.CassandraServer.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.Threadrun(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.utilconcurrent.ExecutionException:
> java.lang.RuntimeException: java.lang.RuntimeException:
> java.util.concurrent.ExecutionException: java.lang.AssertionError
>         at
> org.apache.cassandra.utils.FBUtilitieswaitOnFutures(FBUtilities.java:549)
>         at
> org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:339)
>         at
> org.apache.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.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.Threadrun(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(FutureTask.java: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.dbcommitlog.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
> org.apache.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://riptanocom

Re: 0.7.0.bet1 errors during start up

Posted by Jonathan Ellis <jb...@gmail.com>.
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.thrift.CassandraServer.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.concurrent.ExecutionException:
> java.lang.RuntimeException: java.lang.RuntimeException:
> java.util.concurrent.ExecutionException: java.lang.AssertionError
>         at
> org.apache.cassandra.utils.FBUtilitieswaitOnFutures(FBUtilities.java:549)
>         at
> org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:339)
>         at
> org.apache.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.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: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(FutureTask.java: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
> org.apache.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