Did the test again, empty database, with replication factor 3, Cassandra 
running in it's own jvm.
All data is now stored using a separate program that connects to the database 
using THRIFT.
At least this results in a lot less Dead/Up messages (I guess the GC had too 
much work handling the non-cassandra memory objects), but it is still there.

Also the exception 'invalid column name length 0' is there again. Below is a 
log of machine x.x.x.59 starting after 00:00 hour. One hour before 00:00 I 
stopped all storing, so that the machines had nothing else to do besides 
compacting and cleaning up and ... (still compacting and discarding obsolete 
commit logs).

Checked the log files on all machines, and no exception nor assert related to 
column names could be found.

2011-10-05 00:06:25.172 InetAddress /x.x.x.60 is now dead.
2011-10-05 00:06:25.179 InetAddress /x.x.x.60 is now UP
2011-10-05 00:46:47.091 Saved KsFullIdx-ForwardStringValues-KeyCache (94 items) 
in 19 ms
2011-10-05 00:46:47.334 Saved KsFullIdx-ReverseLongValues-KeyCache (98732 
items) in 117 ms
2011-10-05 00:46:47.797 Saved KsFullIdx-ReverseLabelValues-KeyCache (273425 
items) in 259 ms
2011-10-05 00:46:48.645 Saved KsFullIdx-ReverseStringValues-KeyCache (500000 
items) in 472 ms
2011-10-05 01:00:52.691 ColumnFamilyStore(table='system', 
columnFamily='HintsColumnFamily') liveRatio is 28.375375375375377 (just-counted 
was 28.375375375375377).  calculation took 4ms for 56 columns
2011-10-05 01:07:02.052 InetAddress /x.x.x.60 is now dead.
2011-10-05 01:07:02.058 InetAddress /x.x.x.60 is now UP
2011-10-05 01:07:02.060 InetAddress /x.x.x.61 is now dead.
2011-10-05 01:07:02.060 InetAddress /x.x.x.61 is now UP
2011-10-05 02:07:33.785 InetAddress /x.x.x.60 is now dead.
2011-10-05 02:07:33.791 InetAddress /x.x.x.60 is now UP
2011-10-05 02:41:12.528 Fatal exception in thread Thread[HintedHandoff:1,5,main]
java.io.IOError: 
org.apache.cassandra.db.ColumnSerializer$CorruptColumnException: invalid column 
name length 0
        at 
org.apache.cassandra.io.util.ColumnIterator.deserializeNext(ColumnSortedMap.java:265)
 ~[apache-cassandra-0.8.6.jar:0.8.6]
        at 
org.apache.cassandra.io.util.ColumnIterator.next(ColumnSortedMap.java:281) 
~[apache-cassandra-0.8.6.jar:0.8.6]
        at 
org.apache.cassandra.io.util.ColumnIterator.next(ColumnSortedMap.java:236) 
~[apache-cassandra-0.8.6.jar:0.8.6]
        at 
java.util.concurrent.ConcurrentSkipListMap.buildFromSorted(ConcurrentSkipListMap.java:1493)
 ~[na:1.6.0_24]
        at 
java.util.concurrent.ConcurrentSkipListMap.<init>(ConcurrentSkipListMap.java:1443)
 ~[na:1.6.0_24]
        at 
org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:445) 
~[apache-cassandra-0.8.6.jar:0.8.6]
        at 
org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:428) 
~[apache-cassandra-0.8.6.jar:0.8.6]
        at 
org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:418) 
~[apache-cassandra-0.8.6.jar:0.8.6]
        at 
org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:380) 
~[apache-cassandra-0.8.6.jar:0.8.6]
        at 
org.apache.cassandra.db.columniterator.IndexedSliceReader$IndexedBlockFetcher.getNextBlock(IndexedSliceReader.java:179)
 ~[apache-cassandra-0.8.6.jar:0.8.6]
        at 
org.apache.cassandra.db.columniterator.IndexedSliceReader.computeNext(IndexedSliceReader.java:121)
 ~[apache-cassandra-0.8.6.jar:0.8.6]
        at 
org.apache.cassandra.db.columniterator.IndexedSliceReader.computeNext(IndexedSliceReader.java:49)
 ~[apache-cassandra-0.8.6.jar:0.8.6]
        at 
com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:140)
 ~[guava-r08.jar:na]
        at 
com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:135) 
~[guava-r08.jar:na]
        at 
org.apache.cassandra.db.columniterator.SSTableSliceIterator.hasNext(SSTableSliceIterator.java:108)
 ~[apache-cassandra-0.8.6.jar:0.8.6]
        at 
org.apache.commons.collections.iterators.CollatingIterator.set(CollatingIterator.java:283)
 ~[commons-collections-3.2.1.jar:3.2.1]
        at 
org.apache.commons.collections.iterators.CollatingIterator.least(CollatingIterator.java:326)
 ~[commons-collections-3.2.1.jar:3.2.1]
        at 
org.apache.commons.collections.iterators.CollatingIterator.next(CollatingIterator.java:230)
 ~[commons-collections-3.2.1.jar:3.2.1]
        at 
org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingIterator.java:69)
 ~[apache-cassandra-0.8.6.jar:0.8.6]
        at 
com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:140)
 ~[guava-r08.jar:na]
        at 
com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:135) 
~[guava-r08.jar:na]
        at 
org.apache.cassandra.db.filter.SliceQueryFilter.collectReducedColumns(SliceQueryFilter.java:116)
 ~[apache-cassandra-0.8.6.jar:0.8.6]
        at 
org.apache.cassandra.db.filter.QueryFilter.collectCollatedColumns(QueryFilter.java:142)
 ~[apache-cassandra-0.8.6.jar:0.8.6]
        at 
org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1427)
 ~[apache-cassandra-0.8.6.jar:0.8.6]
        at 
org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1304)
 ~[apache-cassandra-0.8.6.jar:0.8.6]
        at 
org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1261)
 ~[apache-cassandra-0.8.6.jar:0.8.6]
        at 
org.apache.cassandra.db.HintedHandOffManager.sendRow(HintedHandOffManager.java:155)
 ~[apache-cassandra-0.8.6.jar:0.8.6]
        at 
org.apache.cassandra.db.HintedHandOffManager.deliverHintsToEndpoint(HintedHandOffManager.java:350)
 ~[apache-cassandra-0.8.6.jar:0.8.6]
        at 
org.apache.cassandra.db.HintedHandOffManager.access$100(HintedHandOffManager.java:89)
 ~[apache-cassandra-0.8.6.jar:0.8.6]
        at 
org.apache.cassandra.db.HintedHandOffManager$2.runMayThrow(HintedHandOffManager.java:397)
 ~[apache-cassandra-0.8.6.jar:0.8.6]
        at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30) 
~[apache-cassandra-0.8.6.jar:0.8.6]
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
 ~[na:1.6.0_24]
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) 
~[na:1.6.0_24]
        at java.lang.Thread.run(Thread.java:662) ~[na:1.6.0_24]
Caused by: org.apache.cassandra.db.ColumnSerializer$CorruptColumnException: 
invalid column name length 0
        at 
org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:89) 
~[apache-cassandra-0.8.6.jar:0.8.6]
        at 
org.apache.cassandra.io.util.ColumnIterator.deserializeNext(ColumnSortedMap.java:261)
 ~[apache-cassandra-0.8.6.jar:0.8.6]
        ... 33 common frames omitted
2011-10-05 02:42:04.961 Started hinted handoff for endpoint /x.x.x.60
2011-10-05 02:42:04.961 Finished hinted handoff of 0 rows to endpoint /x.x.x.60
2011-10-05 02:42:18.381 Started hinted handoff for endpoint /x.x.x.61
2011-10-05 03:08:04.882 InetAddress /x.x.x.60 is now dead.
2011-10-05 03:08:04.890 InetAddress /x.x.x.61 is now dead.

-----Original Message-----
From: Desimpel, Ignace [mailto:ignace.desim...@nuance.com] 
Sent: dinsdag 4 oktober 2011 10:48
To: user@cassandra.apache.org
Subject: RE: invalid column name length 0

I run the application with the JVM -ea option, so assertions are enabled.

I insert records using the StorageProxy.mutate function. The elements are 
created  as specified below. 
Below : The arForwardFuncValueBytes and arReverseFuncValueBytes are tested for 
null or length = 0 by my code. The oTokenColumnName bytebuffer is created each 
time, but is reused in the two QueryPaths. I assume this is allowed.

QueryPath oPathtoInsert = new QueryPath( sForwardColumnFamToAdd, 
ByteBuffer.wrap(arForwardFuncValueBytes), oTokenColumnName); 
oForwardRowMut.add(oPathtoInsert, oTokenPos, lUpdateTimeStamp);

oPathtoInsert = new QueryPath(sReverseColumnFamToAdd, 
ByteBuffer.wrap(arReverseFuncValueBytes), oTokenColumnName); 
oReverseRowMut.add(oPathtoInsert, oTokenPos, lUpdateTimeStamp);

Anyway, I will do a test inserting the same data, but via thrift and with 
Cassandra in a separate jvm.

Ignace

-----Original Message-----
From: Sylvain Lebresne [mailto:sylv...@datastax.com]
Sent: maandag 3 oktober 2011 18:02
To: user@cassandra.apache.org
Subject: Re: invalid column name length 0

On the 'invalid column name length 0' exception, since you're embedding the 
Cassandra server, it could be that you modify a column ByteBuffer that you feed 
to Cassandra (that's fairly easy to do with ByteBuffer by calling some relative 
get method of ByteBuffer). Or more generally that you feed a zero length 
ByteBuffer as a column name (maybe by using a relative put without a 
rewind/reset afterwards).

Which leads me to a question: do you run your server without assertions enabled 
? (I suspect you do).
If so I suggest you turn them on (to help you find the problem). It turns out 
that we detect zero length column name at write time in an assertion, while we 
detect them at read time using a good old 'if'. So if you do feed a zero length 
column name to Cassandra throught the StorageService interface, you'd only get 
the exception you get at read time.

Now I don't know how much those exceptions are related to the timeoutException 
you're seeing, but such error would typically produce timeout on reads whatever 
the rpc_timeout value is.

--
Sylvain

On Mon, Oct 3, 2011 at 4:58 PM, Desimpel, Ignace <ignace.desim...@nuance.com> 
wrote:
> I did an extra test, again starting from scratch but with replication factor 
> 1.
> I still get the dead/up messages and timeout exceptions, but the system keeps 
> running and storing. However I ran out of disk space, logically producing a 
> lot of other errors.
> Then I restarted the Cassandra servers, so they were able to cleanup and 
> restart without errors.
> Then I did some queries I normally do and got again exceptions like " invalid 
> column name length 0", but also other like " Corrupt (negative) value length 
> encountered".
> Exception : see below.
>
> With this test, I run Cassandra embedded, so a lot of processing ( and object 
> allocations ) are done within the same JVM. I will modify the code so that 
> 'my processing/allcations' are done outside and the Cassandra jvm only has to 
> store the records. But that's for tomorrow.
>
> Did anyone ran into this type of error? And what was the reason? Any help?
>
> 2011-10-03 11:49:21.035 Fatal exception in thread 
> Thread[ReadStage:623,5,main]
> java.io.IOError: java.io.IOException: Corrupt (negative) value length 
> encountered
>        at
> org.apache.cassandra.io.util.ColumnIterator.deserializeNext(ColumnSort
> edMap.java:265) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.io.util.ColumnIterator.next(ColumnSortedMap.java:
> 281) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.io.util.ColumnIterator.next(ColumnSortedMap.java:
> 236) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> java.util.concurrent.ConcurrentSkipListMap.buildFromSorted(ConcurrentS
> kipListMap.java:1493) ~[na:1.6.0_24]
>        at
> java.util.concurrent.ConcurrentSkipListMap.<init>(ConcurrentSkipListMa
> p.java:1443) ~[na:1.6.0_24]
>        at
> org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.
> java:445) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.
> java:428) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.
> java:418) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.
> java:380) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.db.columniterator.IndexedSliceReader$IndexedBlock
> Fetcher.getNextBlock(IndexedSliceReader.java:179)
> ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.db.columniterator.IndexedSliceReader.computeNext(
> IndexedSliceReader.java:121) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.db.columniterator.IndexedSliceReader.computeNext(
> IndexedSliceReader.java:49) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIt
> erator.java:140) ~[guava-r08.jar:na]
>        at
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.ja
> va:135) ~[guava-r08.jar:na]
>        at
> org.apache.cassandra.db.columniterator.SSTableSliceIterator.hasNext(SS
> TableSliceIterator.java:108) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.commons.collections.iterators.CollatingIterator.set(Collati
> ngIterator.java:283) ~[commons-collections-3.2.1.jar:3.2.1]
>        at
> org.apache.commons.collections.iterators.CollatingIterator.least(Colla
> tingIterator.java:326) ~[commons-collections-3.2.1.jar:3.2.1]
>        at
> org.apache.commons.collections.iterators.CollatingIterator.next(Collat
> ingIterator.java:230) ~[commons-collections-3.2.1.jar:3.2.1]
>        at
> org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingIterat
> or.java:69) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIt
> erator.java:140) ~[guava-r08.jar:na]
>        at
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.ja
> va:135) ~[guava-r08.jar:na]
>        at
> org.apache.cassandra.db.filter.SliceQueryFilter.collectReducedColumns(
> SliceQueryFilter.java:116) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.db.filter.QueryFilter.collectCollatedColumns(Quer
> yFilter.java:142) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFam
> ilyStore.java:1427) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamily
> Store.java:1304) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamily
> Store.java:1261) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at org.apache.cassandra.db.Table.getRow(Table.java:385)
> ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadComma
> nd.java:61) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.db.ReadVerbHandler.doVerb(ReadVerbHandler.java:69
> ) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.j
> ava:59) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecu
> tor.java:886) ~[na:1.6.0_24]
>        at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.
> java:908) ~[na:1.6.0_24]
>        at java.lang.Thread.run(Thread.java:662) ~[na:1.6.0_24] Caused
> by: java.io.IOException: Corrupt (negative) value length encountered
>        at
> org.apache.cassandra.utils.ByteBufferUtil.readWithLength(ByteBufferUti
> l.java:348) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.
> java:108) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.io.util.ColumnIterator.deserializeNext(ColumnSort
> edMap.java:261) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        ... 32 common frames omitted
> 2011-10-03 11:55:01.745 Fatal exception in thread 
> Thread[ReadStage:624,5,main]
> java.io.IOError: 
> org.apache.cassandra.db.ColumnSerializer$CorruptColumnException: 
> invalid column name length 0
>        at
> org.apache.cassandra.io.util.ColumnIterator.deserializeNext(ColumnSort
> edMap.java:265) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.io.util.ColumnIterator.next(ColumnSortedMap.java:
> 281) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.io.util.ColumnIterator.next(ColumnSortedMap.java:
> 236) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> java.util.concurrent.ConcurrentSkipListMap.buildFromSorted(ConcurrentS
> kipListMap.java:1493) ~[na:1.6.0_24]
>        at
> java.util.concurrent.ConcurrentSkipListMap.<init>(ConcurrentSkipListMa
> p.java:1443) ~[na:1.6.0_24]
>        at
> org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.
> java:445) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.
> java:428) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.
> java:418) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.
> java:380) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.db.columniterator.IndexedSliceReader$IndexedBlock
> Fetcher.getNextBlock(IndexedSliceReader.java:179)
> ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.db.columniterator.IndexedSliceReader.computeNext(
> IndexedSliceReader.java:121) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.db.columniterator.IndexedSliceReader.computeNext(
> IndexedSliceReader.java:49) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIt
> erator.java:140) ~[guava-r08.jar:na]
>        at
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.ja
> va:135) ~[guava-r08.jar:na]
>        at
> org.apache.cassandra.db.columniterator.SSTableSliceIterator.hasNext(SS
> TableSliceIterator.java:108) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.commons.collections.iterators.CollatingIterator.anyHasNext(
> CollatingIterator.java:364) ~[commons-collections-3.2.1.jar:3.2.1]
>        at
> org.apache.commons.collections.iterators.CollatingIterator.hasNext(Col
> latingIterator.java:217) ~[commons-collections-3.2.1.jar:3.2.1]
>        at
> org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingIterat
> or.java:55) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIt
> erator.java:140) ~[guava-r08.jar:na]
>        at
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.ja
> va:135) ~[guava-r08.jar:na]
>        at
> org.apache.cassandra.db.filter.SliceQueryFilter.collectReducedColumns(
> SliceQueryFilter.java:116) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.db.filter.QueryFilter.collectCollatedColumns(Quer
> yFilter.java:142) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFam
> ilyStore.java:1427) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamily
> Store.java:1304) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamily
> Store.java:1261) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at org.apache.cassandra.db.Table.getRow(Table.java:385)
> ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadComma
> nd.java:61) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.db.ReadVerbHandler.doVerb(ReadVerbHandler.java:69
> ) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.j
> ava:59) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecu
> tor.java:886) ~[na:1.6.0_24]
>        at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.
> java:908) ~[na:1.6.0_24]
>        at java.lang.Thread.run(Thread.java:662) ~[na:1.6.0_24] Caused
> by: org.apache.cassandra.db.ColumnSerializer$CorruptColumnException: 
> invalid column name length 0
>        at
> org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.
> java:89) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at
> org.apache.cassandra.io.util.ColumnIterator.deserializeNext(ColumnSort
> edMap.java:261) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        ... 31 common frames omitted
> 2011-10-03 14:18:53.436 Saved CodeStructure-Computers-KeyCache (1
> items) in 16 ms
> 2011-10-03 14:19:24.804 Saved KsFullIdx-ReverseLabelValues-KeyCache
> (123981 items) in 192 ms
>
> -----Original Message-----
> From: Desimpel, Ignace [mailto:ignace.desim...@nuance.com]
> Sent: dinsdag 27 september 2011 17:51
> To: user@cassandra.apache.org
> Subject: RE: invalid column name length 0
>
> No, brand new, started from scratch, no data at all.
> My setup get into trouble after a couple of hours since it then starts giving 
> the Dead/up messages.
> I also get messages like '11 MUTATION messages dropped in server lifetime'
> Later on I got this type of exception also.
>
> -----Original Message-----
> From: Jonathan Ellis [mailto:jbel...@gmail.com]
> Sent: dinsdag 27 september 2011 17:41
> To: user@cassandra.apache.org
> Subject: Re: invalid column name length 0
>
> Is this upgraded from an earlier version?
>
> On Tue, Sep 27, 2011 at 8:51 AM, Desimpel, Ignace 
> <ignace.desim...@nuance.com> wrote:
>> Version 0.8.6.
>>
>> After an extreme load to 4 (embedded) cassandra servers with 
>> replication factor 3 ( Ubuntu 10.4, dual six core, 64 bit, no swap, 1
>> 15000 rpm commitlog disk, 1 15000 rpm datafile disk, ) I get a fatal 
>> exception as listed below.
>>
>> No more messages were found after that.
>>
>>
>>
>> Probably other things are going wrong like the message "410 Could not 
>> complete hinted handoff to /xxx.yyy.zzz.60", or the Dead/Up messages 
>> (occurring long before this exception).
>>
>> But maybe this exception can point me in the right direction or even 
>> point out some bug in Cassandra.
>>
>>
>>
>> Thanks,
>>
>> Ignace
>>
>>
>>
>>
>>
>> 2011-09-27 06:59:46,383 Compacting large row
>> KsFullIdx/ForwardStringValues:3237343034 (178032211 bytes) 
>> incrementally
>>
>> 2011-09-27 07:00:09,738 GC for ParNew: 311 ms for 1 collections,
>> 8139389704 used; max is 33344716800
>>
>> 2011-09-27 07:00:12,818 Compacting large row
>> KsFullIdx/ForwardStringValues:31363437 (1281862723 bytes) 
>> incrementally
>>
>> 2011-09-27 07:02:16,025 Compacting large row
>> KsFullIdx/ForwardStringValues:31363438 (1623095072 bytes) 
>> incrementally
>>
>> 2011-09-27 07:04:38,332 GC for ParNew: 534 ms for 1 collections,
>> 7811259472 used; max is 33344716800
>>
>> 2011-09-27 07:04:52,803 Compacting large row
>> KsFullIdx/ForwardStringValues:3238313433 (1435774436 bytes) 
>> incrementally
>>
>> 2011-09-27 07:06:57,160 Compacted to
>> /media/datadrive1/capd.cassandra.capd/dbdatafile/KsFullIdx/ForwardStringValues-tmp-g-542-Data.db.
>> 43,244,902,670 to 42,780,624,408 (~98% of original) bytes for 1,260 keys.
>> Time: 4,321,960ms.
>>
>> 2011-09-27 08:01:42,090 Saved KsFullIdx-ForwardStringValues-KeyCache
>> (572
>> items) in 16 ms
>>
>> 2011-09-27 08:01:42,182 Saved KsFullIdx-ReverseStringValues-KeyCache
>> (25688
>> items) in 63 ms
>>
>> 2011-09-27 08:18:13,078 InetAddress /xxx.yyy.zzz.62 is now dead.
>>
>> 2011-09-27 08:18:16,467 InetAddress /xxx.yyy.zzz.62 is now UP
>>
>> 2011-09-27 08:48:56,410 Could not complete hinted handoff to
>> /xxx.yyy.zzz.60
>>
>> 2011-09-27 08:48:56,410 Enqueuing flush of
>> Memtable-HintsColumnFamily@2083796703(12097/196566 serialized/live 
>> bytes,
>> 254 ops)
>>
>> 2011-09-27 08:48:56,411 Writing
>> Memtable-HintsColumnFamily@2083796703(12097/196566 serialized/live 
>> bytes,
>> 254 ops)
>>
>> 2011-09-27 08:48:56,411 Nothing to compact in HintsColumnFamily; use 
>> forceUserDefinedCompaction if you wish to force compaction of single 
>> sstables (e.g. for tombstone collection)
>>
>> 2011-09-27 08:48:56,411 Finished hinted handoff of 254 rows to 
>> endpoint
>> /xxx.yyy.zzz.60
>>
>> 2011-09-27 08:48:56,490 Completed flushing 
>> /media/datadrive1/capd.cassandra.capd/dbdatafile/system/HintsColumnFa
>> m
>> ily-g-10-Data.db
>> (25079 bytes)
>>
>> 2011-09-27 08:49:42,858 Started hinted handoff for endpoint
>> /xxx.yyy.zzz.62
>>
>> 2011-09-27 12:01:42,100 Saved KsFullIdx-ForwardStringValues-KeyCache
>> (712
>> items) in 27 ms
>>
>> 2011-09-27 12:01:42,182 Saved KsFullIdx-ReverseStringValues-KeyCache
>> (30742
>> items) in 55 ms
>>
>> 2011-09-27 12:10:01,016 InetAddress /xxx.yyy.zzz.59 is now dead.
>>
>> 2011-09-27 12:10:02,272 InetAddress /xxx.yyy.zzz.59 is now UP
>>
>> 2011-09-27 12:17:34,596 Fatal exception in thread 
>> Thread[HintedHandoff:1,5,RMI Runtime]
>>
>> java.io.IOError:
>> org.apache.cassandra.db.ColumnSerializer$CorruptColumnException:
>> invalid column name length 0
>>
>>                 at
>> org.apache.cassandra.io.util.ColumnIterator.deserializeNext(ColumnSor
>> t
>> edMap.java:265)
>>
>>                 at
>> org.apache.cassandra.io.util.ColumnIterator.next(ColumnSortedMap.java:
>> 281)
>>
>>                 at
>> org.apache.cassandra.io.util.ColumnIterator.next(ColumnSortedMap.java:
>> 236)
>>
>>                 at
>> java.util.concurrent.ConcurrentSkipListMap.buildFromSorted(Concurrent
>> S
>> kipListMap.java:1493)
>>
>>                 at
>> java.util.concurrent.ConcurrentSkipListMap.<init>(ConcurrentSkipListM
>> a
>> p.java:1443)
>>
>>                 at
>> org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.
>> java:445)
>>
>>                 at
>> org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.
>> java:428)
>>
>>                 at
>> org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.
>> java:418)
>>
>>                 at
>> org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.
>> java:380)
>>
>>                 at
>> org.apache.cassandra.db.columniterator.IndexedSliceReader$IndexedBloc
>> k
>> Fetcher.getNextBlock(IndexedSliceReader.java:179)
>>
>>                 at
>> org.apache.cassandra.db.columniterator.IndexedSliceReader.computeNext
>> (
>> IndexedSliceReader.java:121)
>>
>>                 at
>> org.apache.cassandra.db.columniterator.IndexedSliceReader.computeNext
>> (
>> IndexedSliceReader.java:49)
>>
>>                 at
>> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractI
>> t
>> erator.java:140)
>>
>>                 at
>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.j
>> a
>> va:135)
>>
>>                 at
>> org.apache.cassandra.db.columniterator.SSTableSliceIterator.hasNext(S
>> S
>> TableSliceIterator.java:108)
>>
>>                 at
>> org.apache.commons.collections.iterators.CollatingIterator.set(Collat
>> i
>> ngIterator.java:283)
>>
>>                 at
>> org.apache.commons.collections.iterators.CollatingIterator.least(Coll
>> a
>> tingIterator.java:326)
>>
>>                 at
>> org.apache.commons.collections.iterators.CollatingIterator.next(Colla
>> t
>> ingIterator.java:230)
>>
>>                 at
>> org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingItera
>> t
>> or.java:69)
>>
>>                 at
>> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractI
>> t
>> erator.java:140)
>>
>>                 at
>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.j
>> a
>> va:135)
>>
>>                 at
>> org.apache.cassandra.db.filter.SliceQueryFilter.collectReducedColumns
>> (
>> SliceQueryFilter.java:116)
>>
>>                 at
>> org.apache.cassandra.db.filter.QueryFilter.collectCollatedColumns(Que
>> r
>> yFilter.java:142)
>>
>>                 at
>> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFa
>> m
>> ilyStore.java:1427)
>>
>>                 at
>> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamil
>> y
>> Store.java:1304)
>>
>>                 at
>> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamil
>> y
>> Store.java:1261)
>>
>>                 at
>> org.apache.cassandra.db.HintedHandOffManager.sendRow(HintedHandOffMan
>> a
>> ger.java:155)
>>
>>                 at
>> org.apache.cassandra.db.HintedHandOffManager.deliverHintsToEndpoint(H
>> i
>> ntedHandOffManager.java:350)
>>
>>                 at
>> org.apache.cassandra.db.HintedHandOffManager.access$100(HintedHandOff
>> M
>> anager.java:89)
>>
>>                 at
>> org.apache.cassandra.db.HintedHandOffManager$2.runMayThrow(HintedHand
>> O
>> ffManager.java:397)
>>
>>                 at
>> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:3
>> 0
>> )
>>
>>                 at
>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExec
>> u
>> tor.java:886)
>>
>>                 at
>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.
>> java:908)
>>
>>                 at java.lang.Thread.run(Thread.java:662)
>>
>> Caused by: org.apache.cassandra.db.ColumnSerializer$CorruptColumnException:
>> invalid column name length 0
>>
>>                 at
>> org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.
>> java:89)
>>
>>                 at
>> org.apache.cassandra.io.util.ColumnIterator.deserializeNext(ColumnSor
>> t
>> edMap.java:261)
>>
>>                 ... 33 more
>>
>> 2011-09-27 12:17:53,291 Started hinted handoff for endpoint
>> /xxx.yyy.zzz.59
>
>
>
> --
> Jonathan Ellis
> Project Chair, Apache Cassandra
> co-founder of DataStax, the source for professional Cassandra support 
> http://www.datastax.com
>

Reply via email to