This is a problem…

[default@system] list NodeIdInfo ;
Using default limit of 100
...
-------------------
RowKey: 43757272656e744c6f63616c
=> (column=01efa5d0-e133-11e1-0000-51be601cd0ff, value=0a1020d2, timestamp=1344414498989)
=> (column=92109b80-ea0a-11e1-0000-51be601cd0af, value=0a1020d2, timestamp=1345386691897)

There is an assertion that we should only have one column for the "CurrentLocal" row. The exception is occurring in a static class member initialisation and so is getting lost. 

I'm not sure how two ended up there.

Deleting the NodeIdInfo CF SSTables should fix it. 

I created https://issues.apache.org/jira/browse/CASSANDRA-4626 can you please add more information there if you can and/or watch the ticket incase there are other questions.

Thanks


-----------------
Aaron Morton
Freelance Developer
@aaronmorton

On 5/09/2012, at 10:18 PM, Thomas van Neerijnen <tom@bossastudios.com> wrote:

forgot to answer your first question. I see this:
INFO 14:31:31,896 No saved local node id, using newly generated: 92109b80-ea0a-11e1-0000-51be601cd0af


On Wed, Sep 5, 2012 at 8:41 AM, Thomas van Neerijnen <tom@bossastudios.com> wrote:
Thanks for the help Aaron.
I've checked NodeIdInfo and LocationInfo as below.
What am I looking at? I'm guessing the first row in NodeIdInfo represents the ring with the node ids, but the second row perhaps dead nodes with old schemas? That's a total guess, I'd be very interested to know what it and the LocationInfo are.
If there's anything else you'd like me to check let me know, otherwise I'll attempt your workaround later today.

[default@system] list NodeIdInfo ;
Using default limit of 100
-------------------
RowKey: 4c6f63616c
=> (column=b10552c0-ea0f-11e0-0000-cb1f02ccbcff, value=0a1020d2, timestamp=1317241393645)
=> (column=e64fc8f0-595b-11e1-0000-51be601cd0d7, value=0a1020d2, timestamp=1329478703871)
=> (column=732d4690-a596-11e1-0000-51be601cd09f, value=0a1020d2, timestamp=1337860139385)
=> (column=bffd9d40-aa45-11e1-0000-51be601cd0fe, value=0a1020d2, timestamp=1338375234836)
=> (column=01efa5d0-e133-11e1-0000-51be601cd0ff, value=0a1020d2, timestamp=1344414498989)
=> (column=92109b80-ea0a-11e1-0000-51be601cd0af, value=0a1020d2, timestamp=1345386691897)
-------------------
RowKey: 43757272656e744c6f63616c
=> (column=01efa5d0-e133-11e1-0000-51be601cd0ff, value=0a1020d2, timestamp=1344414498989)
=> (column=92109b80-ea0a-11e1-0000-51be601cd0af, value=0a1020d2, timestamp=1345386691897)

2 Rows Returned.
Elapsed time: 128 msec(s).
[default@system] list LocationInfo ;
Using default limit of 100
-------------------
RowKey: 52696e67
=> (column=00, value=0a1080d2, timestamp=1346666104900)
=> (column=04a7128b6c83505dcd618720f92028f4, value=0a1020b7, timestamp=1332360971660)
=> (column=09249249249249249249249249249249, value=0a1080cd, timestamp=1341136002862)
=> (column=12492492492492492492492492492492, value=0a1020d3, timestamp=1341135999465)
=> (column=15555555555555000000000000000000, value=0a1060d3, timestamp=1346666104671)
=> (column=15555555555555555555555555555555, value=0a1020d3, timestamp=1344530188382)
=> (column=1b6db6db6db6db6db6db6db6db6db6db, value=0a1020b1, timestamp=1341135997643)
=> (column=1c71c71c71c71bffffffffffffffffff, value=0a1080d2, timestamp=1317241889689)
=> (column=24924924924924924924924924924924, value=0a1060d3, timestamp=1341135996555)
=> (column=2aaaaaaaaaaaa9ffffffffffffffffff, value=0a1020d3, timestamp=1317241534292)
=> (column=2aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa, value=0a1060d3, timestamp=1344530187539)
=> (column=38e38e38e38e37ffffffffffffffffff, value=0a1060d3, timestamp=1317241257569)
=> (column=38e38e38e38e38e38e38e38e38e38e38, value=0a1060d3, timestamp=1343136501647)
=> (column=393170e0207a17d8519f0c1bfe325d51, value=0a1020d3, timestamp=1345381375120)
=> (column=3fffffffffffffffffffffffffffffff, value=0a1080d3, timestamp=1346666104939)
=> (column=471c71c71c71c71c71c71c71c71c71c6, value=0a1080d3, timestamp=1343133153701)
=> (column=471c71c71c71c7ffffffffffffffffff, value=0a1080d3, timestamp=1317241786636)
=> (column=49249249249249249249249249249249, value=0a1080d3, timestamp=1341136002693)
=> (column=52492492492492492492492492492492, value=0a106010, timestamp=1341136002626)
=> (column=55555555555553ffffffffffffffffff, value=0a1020d4, timestamp=1328473688357)
=> (column=55555555555555555555555555555554, value=0a1060d4, timestamp=1346666104910)
=> (column=5b6db6db6db6db6db6db6db6db6db6da, value=0a1060d4, timestamp=1332389784945)
=> (column=5b6db6db6db6db6db6db6db6db6db6db, value=0a1060d4, timestamp=1341136001027)
=> (column=638e38e38e38e38e38e38e38e38e38e2, value=0a1060d4, timestamp=1343125208462)
=> (column=638e38e38e38e3ffffffffffffffffff, value=0a1060d4, timestamp=1317241257577)
=> (column=6aaaaaaaaaaaac000000000000000000, value=0a1020d3, timestamp=1346666104789)
-------------------
RowKey: 4c
=> (column=436c75737465724e616d65, value=4d6f6e737465724d696e642050726f6420436c7573746572, timestamp=1317241251097000)
=> (column=47656e65726174696f6e, value=50447e78, timestamp=1346666104152000)
=> (column=50617274696f6e6572, value=6f72672e6170616368652e63617373616e6472612e6468742e52616e646f6d506172746974696f6e6572, timestamp=1317241251097000)
=> (column=546f6b656e, value=2aaaaaaaaaaaaa000000000000000000, timestamp=1346666104214)
-------------------
RowKey: 436f6f6b696573
=> (column=48696e7473207075726765642061732070617274206f6620757067726164696e672066726f6d20302e362e7820746f20302e37, value=6f68207965732c20697420746865792077657265207075726765642e, timestamp=1317241251249)
=> (column=5072652d312e302068696e747320707572676564, value=6f68207965732c2074686579207765726520707572676564, timestamp=1326274339337)
-------------------
RowKey: 426f6f747374726170
=> (column=42, value=01, timestamp=1346666104213)

4 Rows Returned.
Elapsed time: 34 msec(s).


On Wed, Sep 5, 2012 at 2:42 AM, aaron morton <aaron@thelastpickle.com> wrote:
Hmmm, this looks like an error in ctor for NodeId$LocalNodeIdHistory. Are there any other ERROR log messages?

Do you see either of these two messages in the log:
"No saved local node id, using newly generated: {}"
or
"Saved local node id: {}"


Can you use cassandra-cli / cqlsh to print the contents of the NodeIdInfo and LocationInfo CF's from system ? 

I *think* a work around may be to:

* stop the node
* remove  LocationInfo and NodeInfo cfs. 
* restart

Note this will read the token from the yaml file again, so make sure it's right.

 cheers

-----------------
Aaron Morton
Freelance Developer
@aaronmorton

On 4/09/2012, at 9:51 PM, Thomas van Neerijnen <tom@bossastudios.com> wrote:

Hi

I have a single node in a 6 node Cassandra 1.0.11 cluster that seems to have a single column family in a weird state.

Repairs, upgradesstables, anything that touches this CF crashes.
I've drained the node, removed every file for this CF from said node, removed the commit log, started it up and as soon as data is written to this CF on this node I'm in the same situation again. Anyone have any suggestions for how to fix this?
I'm tempted to remove the node and re-add it but I was hoping for something a little less disruptive.

$ nodetool -h localhost upgradesstables Player PlayerCounters
Error occured while upgrading the sstables for keyspace Player
java.util.concurrent.ExecutionException: java.lang.NoClassDefFoundError: Could not initialize class org.apache.cassandra.utils.NodeId$LocalIds
        at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:222)
        at java.util.concurrent.FutureTask.get(FutureTask.java:83)
        at org.apache.cassandra.db.compaction.CompactionManager.performAllSSTableOperation(CompactionManager.java:219)
        at org.apache.cassandra.db.compaction.CompactionManager.performSSTableRewrite(CompactionManager.java:235)
        at org.apache.cassandra.db.ColumnFamilyStore.sstablesRewrite(ColumnFamilyStore.java:999)
        at org.apache.cassandra.service.StorageService.upgradeSSTables(StorageService.java:1652)
        at sun.reflect.GeneratedMethodAccessor59.invoke(Unknown Source)
        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
        at java.lang.reflect.Method.invoke(Method.java:597)
        at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:93)
        at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:27)
        at com.sun.jmx.mbeanserver.MBeanIntrospector.invokeM(MBeanIntrospector.java:208)
        at com.sun.jmx.mbeanserver.PerInterface.invoke(PerInterface.java:120)
        at com.sun.jmx.mbeanserver.MBeanSupport.invoke(MBeanSupport.java:262)
        at com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.invoke(DefaultMBeanServerInterceptor.java:836)
        at com.sun.jmx.mbeanserver.JmxMBeanServer.invoke(JmxMBeanServer.java:761)
        at javax.management.remote.rmi.RMIConnectionImpl.doOperation(RMIConnectionImpl.java:1427)
        at javax.management.remote.rmi.RMIConnectionImpl.access$200(RMIConnectionImpl.java:72)
        at javax.management.remote.rmi.RMIConnectionImpl$PrivilegedOperation.run(RMIConnectionImpl.java:1265)
        at javax.management.remote.rmi.RMIConnectionImpl.doPrivilegedOperation(RMIConnectionImpl.java:1360)
        at javax.management.remote.rmi.RMIConnectionImpl.invoke(RMIConnectionImpl.java:788)
        at sun.reflect.GeneratedMethodAccessor58.invoke(Unknown Source)
        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
        at java.lang.reflect.Method.invoke(Method.java:597)
        at sun.rmi.server.UnicastServerRef.dispatch(UnicastServerRef.java:303)
        at sun.rmi.transport.Transport$1.run(Transport.java:159)
        at java.security.AccessController.doPrivileged(Native Method)
        at sun.rmi.transport.Transport.serviceCall(Transport.java:155)
        at sun.rmi.transport.tcp.TCPTransport.handleMessages(TCPTransport.java:535)
        at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(TCPTransport.java:790)
        at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run(TCPTransport.java:649)
        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)
Caused by: java.lang.NoClassDefFoundError: Could not initialize class org.apache.cassandra.utils.NodeId$LocalIds
        at org.apache.cassandra.utils.NodeId.localIds(NodeId.java:49)
        at org.apache.cassandra.utils.NodeId.getOldLocalNodeIds(NodeId.java:79)
        at org.apache.cassandra.db.CounterColumn.computeOldShardMerger(CounterColumn.java:251)
        at org.apache.cassandra.db.CounterColumn.mergeAndRemoveOldShards(CounterColumn.java:297)
        at org.apache.cassandra.db.CounterColumn.mergeAndRemoveOldShards(CounterColumn.java:271)
        at org.apache.cassandra.db.compaction.PrecompactedRow.removeDeletedAndOldShards(PrecompactedRow.java:81)
        at org.apache.cassandra.db.compaction.PrecompactedRow.<init>(PrecompactedRow.java:97)
        at org.apache.cassandra.db.compaction.CompactionController.getCompactedRow(CompactionController.java:137)
        at org.apache.cassandra.db.compaction.CompactionIterable$Reducer.getReduced(CompactionIterable.java:97)
        at org.apache.cassandra.db.compaction.CompactionIterable$Reducer.getReduced(CompactionIterable.java:82)
        at org.apache.cassandra.utils.MergeIterator$OneToOne.computeNext(MergeIterator.java:207)
        at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:140)
        at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:135)
        at com.google.common.collect.Iterators$7.computeNext(Iterators.java:614)
        at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:140)
        at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:135)
        at org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:141)
        at org.apache.cassandra.db.compaction.CompactionManager$4.perform(CompactionManager.java:247)
        at org.apache.cassandra.db.compaction.CompactionManager$2.call(CompactionManager.java:198)
        at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
        at java.util.concurrent.FutureTask.run(FutureTask.java:138)
        ... 3 more
$ tail /var/log/cassandra/system.log
INFO [CompactionExecutor:852] 2012-09-04 09:45:37,021 CompactionTask.java (line 112) Compacting [SSTableReader(path='/var/lib/cassandra/data/Player/PlayerCounters-hd-1-Data.db')]
ERROR [CompactionExecutor:852] 2012-09-04 09:45:37,030 AbstractCassandraDaemon.java (line 139) Fatal exception in thread Thread[CompactionExecutor:852,1,RMI Runtime]
java.lang.NoClassDefFoundError: Could not initialize class org.apache.cassandra.utils.NodeId$LocalIds
        at org.apache.cassandra.utils.NodeId.localIds(NodeId.java:49)
        at org.apache.cassandra.utils.NodeId.getOldLocalNodeIds(NodeId.java:79)
        at org.apache.cassandra.db.CounterColumn.computeOldShardMerger(CounterColumn.java:251)
        at org.apache.cassandra.db.CounterColumn.mergeAndRemoveOldShards(CounterColumn.java:297)
        at org.apache.cassandra.db.CounterColumn.mergeAndRemoveOldShards(CounterColumn.java:271)
        at org.apache.cassandra.db.compaction.PrecompactedRow.removeDeletedAndOldShards(PrecompactedRow.java:81)
        at org.apache.cassandra.db.compaction.PrecompactedRow.<init>(PrecompactedRow.java:97)
        at org.apache.cassandra.db.compaction.CompactionController.getCompactedRow(CompactionController.java:137)
        at org.apache.cassandra.db.compaction.CompactionIterable$Reducer.getReduced(CompactionIterable.java:97)
        at org.apache.cassandra.db.compaction.CompactionIterable$Reducer.getReduced(CompactionIterable.java:82)
        at org.apache.cassandra.utils.MergeIterator$OneToOne.computeNext(MergeIterator.java:207)
        at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:140)
        at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:135)
        at com.google.common.collect.Iterators$7.computeNext(Iterators.java:614)
        at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:140)
        at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:135)
        at org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:141)
        at org.apache.cassandra.db.compaction.CompactionManager$4.perform(CompactionManager.java:247)
        at org.apache.cassandra.db.compaction.CompactionManager$2.call(CompactionManager.java:198)
        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)