You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@cassandra.apache.org by "Desimpel, Ignace" <Ig...@nuance.com> on 2011/09/27 15:51:36 UTC

invalid column name length 0

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/ForwardString
Values-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/HintsColumnFamil
y-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(ColumnSorted
Map.java:265)

                at
org.apache.cassandra.io.util.ColumnIterator.next(ColumnSortedMap.java:28
1)

                at
org.apache.cassandra.io.util.ColumnIterator.next(ColumnSortedMap.java:23
6)

                at
java.util.concurrent.ConcurrentSkipListMap.buildFromSorted(ConcurrentSki
pListMap.java:1493)

                at
java.util.concurrent.ConcurrentSkipListMap.<init>(ConcurrentSkipListMap.
java:1443)

                at
org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.ja
va:445)

                at
org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.ja
va:428)

                at
org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.ja
va:418)

                at
org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.ja
va:380)

                at
org.apache.cassandra.db.columniterator.IndexedSliceReader$IndexedBlockFe
tcher.getNextBlock(IndexedSliceReader.java:179)

                at
org.apache.cassandra.db.columniterator.IndexedSliceReader.computeNext(In
dexedSliceReader.java:121)

                at
org.apache.cassandra.db.columniterator.IndexedSliceReader.computeNext(In
dexedSliceReader.java:49)

                at
com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIter
ator.java:140)

                at
com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java
:135)

                at
org.apache.cassandra.db.columniterator.SSTableSliceIterator.hasNext(SSTa
bleSliceIterator.java:108)

                at
org.apache.commons.collections.iterators.CollatingIterator.set(Collating
Iterator.java:283)

                at
org.apache.commons.collections.iterators.CollatingIterator.least(Collati
ngIterator.java:326)

                at
org.apache.commons.collections.iterators.CollatingIterator.next(Collatin
gIterator.java:230)

                at
org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingIterator
.java:69)

                at
com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIter
ator.java:140)

                at
com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java
:135)

                at
org.apache.cassandra.db.filter.SliceQueryFilter.collectReducedColumns(Sl
iceQueryFilter.java:116)

                at
org.apache.cassandra.db.filter.QueryFilter.collectCollatedColumns(QueryF
ilter.java:142)

                at
org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamil
yStore.java:1427)

                at
org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilySt
ore.java:1304)

                at
org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilySt
ore.java:1261)

                at
org.apache.cassandra.db.HintedHandOffManager.sendRow(HintedHandOffManage
r.java:155)

                at
org.apache.cassandra.db.HintedHandOffManager.deliverHintsToEndpoint(Hint
edHandOffManager.java:350)

                at
org.apache.cassandra.db.HintedHandOffManager.access$100(HintedHandOffMan
ager.java:89)

                at
org.apache.cassandra.db.HintedHandOffManager$2.runMayThrow(HintedHandOff
Manager.java:397)

                at
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)

                at
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecuto
r.java:886)

                at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.ja
va: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.ja
va:89)

                at
org.apache.cassandra.io.util.ColumnIterator.deserializeNext(ColumnSorted
Map.java:261)

                ... 33 more

2011-09-27 12:17:53,291 Started hinted handoff for endpoint
/xxx.yyy.zzz.59


RE: invalid column name length 0

Posted by "Desimpel, Ignace" <Ig...@nuance.com>.
TTLs : no
Deletion : yes ; but I think I can avoid this and thus running the same test without deletion, just to eliminate possibilities.

-----Original Message-----
From: Sylvain Lebresne [mailto:sylvain@datastax.com] 
Sent: woensdag 5 oktober 2011 15:34
To: user@cassandra.apache.org
Subject: Re: invalid column name length 0

Ok. Quick other question then. Did you issue deletion and/or used TTLs for that test ?

Also, it's probably worth creating a ticket on https://issues.apache.org/jira/browse/CASSANDRA
if you don't mind.

--
Sylvain

On Wed, Oct 5, 2011 at 2:42 PM, Desimpel, Ignace <Ig...@nuance.com> wrote:
> 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(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.HintedHandOffManager.sendRow(HintedHandOffMana
> ger.java:155) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at 
> org.apache.cassandra.db.HintedHandOffManager.deliverHintsToEndpoint(Hi
> ntedHandOffManager.java:350) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at 
> org.apache.cassandra.db.HintedHandOffManager.access$100(HintedHandOffM
> anager.java:89) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at 
> org.apache.cassandra.db.HintedHandOffManager$2.runMayThrow(HintedHandO
> ffManager.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(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]
>        ... 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.Desimpel@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:sylvain@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 <Ig...@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(ColumnSor
>> t
>> 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(Concurrent
>> S
>> kipListMap.java:1493) ~[na:1.6.0_24]
>>        at
>> java.util.concurrent.ConcurrentSkipListMap.<init>(ConcurrentSkipListM
>> a
>> 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$IndexedBloc
>> k
>> 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(AbstractI
>> t
>> erator.java:140) ~[guava-r08.jar:na]
>>        at
>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.j
>> a
>> va:135) ~[guava-r08.jar:na]
>>        at
>> org.apache.cassandra.db.columniterator.SSTableSliceIterator.hasNext(S
>> S
>> TableSliceIterator.java:108) ~[apache-cassandra-0.8.6.jar:0.8.6]
>>        at
>> org.apache.commons.collections.iterators.CollatingIterator.set(Collat
>> i
>> ngIterator.java:283) ~[commons-collections-3.2.1.jar:3.2.1]
>>        at
>> org.apache.commons.collections.iterators.CollatingIterator.least(Coll
>> a
>> tingIterator.java:326) ~[commons-collections-3.2.1.jar:3.2.1]
>>        at
>> org.apache.commons.collections.iterators.CollatingIterator.next(Colla
>> t
>> ingIterator.java:230) ~[commons-collections-3.2.1.jar:3.2.1]
>>        at
>> org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingItera
>> t
>> or.java:69) ~[apache-cassandra-0.8.6.jar:0.8.6]
>>        at
>> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractI
>> t
>> erator.java:140) ~[guava-r08.jar:na]
>>        at
>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.j
>> a
>> 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(Que
>> r
>> yFilter.java:142) ~[apache-cassandra-0.8.6.jar:0.8.6]
>>        at
>> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFa
>> m
>> ilyStore.java:1427) ~[apache-cassandra-0.8.6.jar:0.8.6]
>>        at
>> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamil
>> y
>> Store.java:1304) ~[apache-cassandra-0.8.6.jar:0.8.6]
>>        at
>> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamil
>> y
>> 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(SliceFromReadComm
>> a
>> nd.java:61) ~[apache-cassandra-0.8.6.jar:0.8.6]
>>        at
>> org.apache.cassandra.db.ReadVerbHandler.doVerb(ReadVerbHandler.java:6
>> 9
>> ) ~[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(ThreadPoolExec
>> u
>> 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(ByteBufferUt
>> i
>> 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(ColumnSor
>> t
>> 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(ColumnSor
>> t
>> 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(Concurrent
>> S
>> kipListMap.java:1493) ~[na:1.6.0_24]
>>        at
>> java.util.concurrent.ConcurrentSkipListMap.<init>(ConcurrentSkipListM
>> a
>> 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$IndexedBloc
>> k
>> 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(AbstractI
>> t
>> erator.java:140) ~[guava-r08.jar:na]
>>        at
>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.j
>> a
>> va:135) ~[guava-r08.jar:na]
>>        at
>> org.apache.cassandra.db.columniterator.SSTableSliceIterator.hasNext(S
>> S
>> 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(Co
>> l
>> latingIterator.java:217) ~[commons-collections-3.2.1.jar:3.2.1]
>>        at
>> org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingItera
>> t
>> or.java:55) ~[apache-cassandra-0.8.6.jar:0.8.6]
>>        at
>> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractI
>> t
>> erator.java:140) ~[guava-r08.jar:na]
>>        at
>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.j
>> a
>> 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(Que
>> r
>> yFilter.java:142) ~[apache-cassandra-0.8.6.jar:0.8.6]
>>        at
>> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFa
>> m
>> ilyStore.java:1427) ~[apache-cassandra-0.8.6.jar:0.8.6]
>>        at
>> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamil
>> y
>> Store.java:1304) ~[apache-cassandra-0.8.6.jar:0.8.6]
>>        at
>> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamil
>> y
>> 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(SliceFromReadComm
>> a
>> nd.java:61) ~[apache-cassandra-0.8.6.jar:0.8.6]
>>        at
>> org.apache.cassandra.db.ReadVerbHandler.doVerb(ReadVerbHandler.java:6
>> 9
>> ) ~[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(ThreadPoolExec
>> u
>> 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(ColumnSor
>> t
>> 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.Desimpel@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:jbellis@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 <Ig...@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/HintsColumnF
>>> a
>>> 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(ColumnSo
>>> r
>>> 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(Concurren
>>> t
>>> S
>>> kipListMap.java:1493)
>>>
>>>                 at
>>> java.util.concurrent.ConcurrentSkipListMap.<init>(ConcurrentSkipList
>>> M
>>> 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$IndexedBlo
>>> c
>>> k
>>> Fetcher.getNextBlock(IndexedSliceReader.java:179)
>>>
>>>                 at
>>> org.apache.cassandra.db.columniterator.IndexedSliceReader.computeNex
>>> t
>>> (
>>> IndexedSliceReader.java:121)
>>>
>>>                 at
>>> org.apache.cassandra.db.columniterator.IndexedSliceReader.computeNex
>>> t
>>> (
>>> IndexedSliceReader.java:49)
>>>
>>>                 at
>>> com.google.common.collect.AbstractIterator.tryToComputeNext(Abstract
>>> I
>>> 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(Colla
>>> t
>>> i
>>> ngIterator.java:283)
>>>
>>>                 at
>>> org.apache.commons.collections.iterators.CollatingIterator.least(Col
>>> l
>>> a
>>> tingIterator.java:326)
>>>
>>>                 at
>>> org.apache.commons.collections.iterators.CollatingIterator.next(Coll
>>> a
>>> t
>>> ingIterator.java:230)
>>>
>>>                 at
>>> org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingIter
>>> a
>>> t
>>> or.java:69)
>>>
>>>                 at
>>> com.google.common.collect.AbstractIterator.tryToComputeNext(Abstract
>>> I
>>> t
>>> erator.java:140)
>>>
>>>                 at
>>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.
>>> j
>>> a
>>> va:135)
>>>
>>>                 at
>>> org.apache.cassandra.db.filter.SliceQueryFilter.collectReducedColumn
>>> s
>>> (
>>> SliceQueryFilter.java:116)
>>>
>>>                 at
>>> org.apache.cassandra.db.filter.QueryFilter.collectCollatedColumns(Qu
>>> e
>>> r
>>> yFilter.java:142)
>>>
>>>                 at
>>> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnF
>>> a
>>> m
>>> ilyStore.java:1427)
>>>
>>>                 at
>>> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFami
>>> l
>>> y
>>> Store.java:1304)
>>>
>>>                 at
>>> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFami
>>> l
>>> y
>>> Store.java:1261)
>>>
>>>                 at
>>> org.apache.cassandra.db.HintedHandOffManager.sendRow(HintedHandOffMa
>>> n
>>> a
>>> ger.java:155)
>>>
>>>                 at
>>> org.apache.cassandra.db.HintedHandOffManager.deliverHintsToEndpoint(
>>> H
>>> i
>>> ntedHandOffManager.java:350)
>>>
>>>                 at
>>> org.apache.cassandra.db.HintedHandOffManager.access$100(HintedHandOf
>>> f
>>> M
>>> anager.java:89)
>>>
>>>                 at
>>> org.apache.cassandra.db.HintedHandOffManager$2.runMayThrow(HintedHan
>>> d
>>> O
>>> ffManager.java:397)
>>>
>>>                 at
>>> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:
>>> 3
>>> 0
>>> )
>>>
>>>                 at
>>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExe
>>> c
>>> 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(ColumnSo
>>> r
>>> 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
>>
>

Re: invalid column name length 0

Posted by Sylvain Lebresne <sy...@datastax.com>.
Ok. Quick other question then. Did you issue deletion and/or used TTLs
for that test ?

Also, it's probably worth creating a ticket on
https://issues.apache.org/jira/browse/CASSANDRA
if you don't mind.

--
Sylvain

On Wed, Oct 5, 2011 at 2:42 PM, Desimpel, Ignace
<Ig...@nuance.com> wrote:
> 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.Desimpel@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:sylvain@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 <Ig...@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.Desimpel@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:jbellis@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 <Ig...@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
>>
>

RE: invalid column name length 0

Posted by "Desimpel, Ignace" <Ig...@nuance.com>.
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.Desimpel@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:sylvain@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 <Ig...@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.Desimpel@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:jbellis@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 <Ig...@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
>

RE: invalid column name length 0

Posted by "Desimpel, Ignace" <Ig...@nuance.com>.
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:sylvain@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 <Ig...@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.Desimpel@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:jbellis@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 <Ig...@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
>

Re: invalid column name length 0

Posted by Sylvain Lebresne <sy...@datastax.com>.
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
<Ig...@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(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.Table.getRow(Table.java:385) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.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.java:59) ~[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: java.io.IOException: Corrupt (negative) value length encountered
>        at org.apache.cassandra.utils.ByteBufferUtil.readWithLength(ByteBufferUtil.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(ColumnSortedMap.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(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.anyHasNext(CollatingIterator.java:364) ~[commons-collections-3.2.1.jar:3.2.1]
>        at org.apache.commons.collections.iterators.CollatingIterator.hasNext(CollatingIterator.java:217) ~[commons-collections-3.2.1.jar:3.2.1]
>        at org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingIterator.java:55) ~[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.Table.getRow(Table.java:385) ~[apache-cassandra-0.8.6.jar:0.8.6]
>        at org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.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.java:59) ~[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]
>        ... 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.Desimpel@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:jbellis@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 <Ig...@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/HintsColumnFam
>> 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(ColumnSort
>> 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(ConcurrentS
>> kipListMap.java:1493)
>>
>>                 at
>> java.util.concurrent.ConcurrentSkipListMap.<init>(ConcurrentSkipListMa
>> 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$IndexedBlock
>> 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(AbstractIt
>> erator.java:140)
>>
>>                 at
>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.ja
>> va:135)
>>
>>                 at
>> org.apache.cassandra.db.columniterator.SSTableSliceIterator.hasNext(SS
>> TableSliceIterator.java:108)
>>
>>                 at
>> org.apache.commons.collections.iterators.CollatingIterator.set(Collati
>> ngIterator.java:283)
>>
>>                 at
>> org.apache.commons.collections.iterators.CollatingIterator.least(Colla
>> tingIterator.java:326)
>>
>>                 at
>> org.apache.commons.collections.iterators.CollatingIterator.next(Collat
>> ingIterator.java:230)
>>
>>                 at
>> org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingIterat
>> or.java:69)
>>
>>                 at
>> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIt
>> erator.java:140)
>>
>>                 at
>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.ja
>> va:135)
>>
>>                 at
>> org.apache.cassandra.db.filter.SliceQueryFilter.collectReducedColumns(
>> SliceQueryFilter.java:116)
>>
>>                 at
>> org.apache.cassandra.db.filter.QueryFilter.collectCollatedColumns(Quer
>> yFilter.java:142)
>>
>>                 at
>> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFam
>> ilyStore.java:1427)
>>
>>                 at
>> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamily
>> Store.java:1304)
>>
>>                 at
>> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamily
>> Store.java:1261)
>>
>>                 at
>> org.apache.cassandra.db.HintedHandOffManager.sendRow(HintedHandOffMana
>> ger.java:155)
>>
>>                 at
>> org.apache.cassandra.db.HintedHandOffManager.deliverHintsToEndpoint(Hi
>> ntedHandOffManager.java:350)
>>
>>                 at
>> org.apache.cassandra.db.HintedHandOffManager.access$100(HintedHandOffM
>> anager.java:89)
>>
>>                 at
>> org.apache.cassandra.db.HintedHandOffManager$2.runMayThrow(HintedHandO
>> ffManager.java:397)
>>
>>                 at
>> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30
>> )
>>
>>                 at
>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecu
>> 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(ColumnSort
>> 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
>

RE: invalid column name length 0

Posted by "Desimpel, Ignace" <Ig...@nuance.com>.
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(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.Table.getRow(Table.java:385) ~[apache-cassandra-0.8.6.jar:0.8.6]
	at org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.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.java:59) ~[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: java.io.IOException: Corrupt (negative) value length encountered
	at org.apache.cassandra.utils.ByteBufferUtil.readWithLength(ByteBufferUtil.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(ColumnSortedMap.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(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.anyHasNext(CollatingIterator.java:364) ~[commons-collections-3.2.1.jar:3.2.1]
	at org.apache.commons.collections.iterators.CollatingIterator.hasNext(CollatingIterator.java:217) ~[commons-collections-3.2.1.jar:3.2.1]
	at org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingIterator.java:55) ~[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.Table.getRow(Table.java:385) ~[apache-cassandra-0.8.6.jar:0.8.6]
	at org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.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.java:59) ~[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]
	... 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.Desimpel@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:jbellis@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 <Ig...@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/HintsColumnFam
> 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(ColumnSort
> 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(ConcurrentS
> kipListMap.java:1493)
>
>                 at
> java.util.concurrent.ConcurrentSkipListMap.<init>(ConcurrentSkipListMa
> 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$IndexedBlock
> 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(AbstractIt
> erator.java:140)
>
>                 at
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.ja
> va:135)
>
>                 at
> org.apache.cassandra.db.columniterator.SSTableSliceIterator.hasNext(SS
> TableSliceIterator.java:108)
>
>                 at
> org.apache.commons.collections.iterators.CollatingIterator.set(Collati
> ngIterator.java:283)
>
>                 at
> org.apache.commons.collections.iterators.CollatingIterator.least(Colla
> tingIterator.java:326)
>
>                 at
> org.apache.commons.collections.iterators.CollatingIterator.next(Collat
> ingIterator.java:230)
>
>                 at
> org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingIterat
> or.java:69)
>
>                 at
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIt
> erator.java:140)
>
>                 at
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.ja
> va:135)
>
>                 at
> org.apache.cassandra.db.filter.SliceQueryFilter.collectReducedColumns(
> SliceQueryFilter.java:116)
>
>                 at
> org.apache.cassandra.db.filter.QueryFilter.collectCollatedColumns(Quer
> yFilter.java:142)
>
>                 at
> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFam
> ilyStore.java:1427)
>
>                 at
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamily
> Store.java:1304)
>
>                 at
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamily
> Store.java:1261)
>
>                 at
> org.apache.cassandra.db.HintedHandOffManager.sendRow(HintedHandOffMana
> ger.java:155)
>
>                 at
> org.apache.cassandra.db.HintedHandOffManager.deliverHintsToEndpoint(Hi
> ntedHandOffManager.java:350)
>
>                 at
> org.apache.cassandra.db.HintedHandOffManager.access$100(HintedHandOffM
> anager.java:89)
>
>                 at
> org.apache.cassandra.db.HintedHandOffManager$2.runMayThrow(HintedHandO
> ffManager.java:397)
>
>                 at
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30
> )
>
>                 at
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecu
> 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(ColumnSort
> 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

Re: invalid column name length 0

Posted by aaron morton <aa...@thelastpickle.com>.
TimedOutErrors happen when less than CL nodes respond within rpc_timeout to the coordinator. I would look at the nodetool tpstats to see if the read or write thread pool of flooded, you will probably see  a high pending count. 

One way to overwhelm a node is to use a high number of record in a batch_mutation or multiget_slice. 

(Xmn sets the size of the young generation, not the heap size was this a typo ?)

Can you dial your bench marking down to something that does not get timeouts, then increase it and watch the TPStats to see if/when cassandra starts backing up ? 

Hope that helps. 


-----------------
Aaron Morton
Freelance Cassandra Developer
@aaronmorton
http://www.thelastpickle.com

On 30/09/2011, at 10:13 PM, Desimpel, Ignace wrote:

> Wanted to see if I still would similar exceptions if I lowered the load by reducing the replication factor.
> I saw that in the log files a lot of 'GC parNew' messages and started a complete new run (empty database) with smaller JVM -Xmn1000M and with replication factor 1 (before it was 3).
> Currently running, 10 % user cpu, 3% iowait, 'iostat await' 60 on commitlog device, 'iostat await 50' on datafiles device, 'iostat svctm 3', 'vmstat si 0', 'vmstat so 0'
> I still get the DEAD/UP messages, less 'GC parNew' messages (some parNew take more than 1 sec), but the system recovers and continues.
> But the weird thing is that sometimes I still get java.util.concurrent.TimeoutException exceptions , as well on batch inserts (per 100 rows) and on single row inserts (with just one 64 char row key and one 4 char column name and 64 char column value). Rpc_timeout is 36000 !
> 
> I looked up what the other 3 machines were doing at that time. The 'received record...' and 'Document stored ...' are msgs of our application.
> Below a stack trace of exceptions occurring at the 'same time', just 3 seconds time difference, on machine 1. 
> Machine 1 : 08:23:41 : the timeout exception stack trace as listed at end.
> Machine 2:  Nothing special
> =========
> ....
> 2011-09-30 08:23:20.888 Completed flushing /media/datadrive1/capd.cassandra.capd/dbdatafile/KsFullIdx/ForwardShortValues-g-149-Data.db (131582322 bytes)
> 2011-09-30 08:23:20.889 Writing Memtable-ForwardLabelValues@260608040(47867232/1983693805 serialized/live bytes, 1150959 ops)
> 2011-09-30 08:23:23.002 Document stored with count records : 55336 Part DB (ms) : 3666
> ...
> 2011-09-30 08:23:33.467 Creating new commitlog segment ../../../../data/capd.cassandra.capd/dbcommitlog/CommitLog-1317363813467.log
> 2011-09-30 08:23:33.915 Document stored with count records : 9046 Part DB (ms) : 653
> ....
> Machine 3: Is compacting. Also got a timeout exception about 36 seconds before the exception of machine 1
> =========
> 2011-09-30 08:22:34.458 Compacted to /media/datadrive1/capd.cassandra.capd/dbdatafile/KsFullIdx/ReverseStringValues-tmp-g-622-Data.db.  966,991,510 to 965,780,805 (~99% of original) bytes for 74,345 keys.  Time: 208,416ms.
> 2011-09-30 08:22:37.474 Compacting large row KsFullIdx/ForwardStringValues:3237393737 (723544055 bytes) incrementally
> 2011-09-30 08:22:37.751 Document stored with count records : 13149 Part DB (ms) : 1613
> 2011-09-30 08:22:37.751 Document stored with count records : 13149 Time elapsed (ms) : 1748
> 2011-09-30 08:23:14.149 java.util.concurrent.TimeoutException
> 2011-09-30 08:23:14.149 	at org.apache.cassandra.service.AbstractWriteResponseHandler.get(AbstractWriteResponseHandler.java:69)
> 2011-09-30 08:23:14.149 	at org.apache.cassandra.service.StorageProxy.mutate(StorageProxy.java:169)
> 2011-09-30 08:23:14.150 	at be.landc.services.search.server.db.baseserver.indexsearch.store.cassandra.CassandraStore.storeAnnotationsProxy(CassandraStore.java:1290)
> 2011-09-30 08:23:33.548 Received records to store with total count : 23277
> 2011-09-30 08:23:34.705 Received records to store with total count : 9816
> 2011-09-30 08:23:35.152 Received records to store with total count : 8646
> 2011-09-30 08:23:35.323 Received records to store with total count : 49888
> 2011-09-30 08:23:35.475 Document stored with count records : 23277 Part DB (ms) : 1926
> ....
> 2011-09-30 08:23:39.627 Received records to store with total count : 8443
> 2011-09-30 08:23:40.081 Document stored with count records : 26862 Part DB (ms) : 3192
> 2011-09-30 08:23:40.081 Document stored with count records : 26862 Time elapsed (ms) : 3363
> ...
> Machine 4: Nothing special, but also got the timeout exception for a single row insert at 08:22:30
> =========
> 2011-09-30 08:22:12.569 Compacted to /media/datadrive1/capd.cassandra.capd/dbdatafile/KsFullIdx/ForwardIntegerValues-tmp-g-150-Data.db.  586,039,224 to 586,036,960 (~99% of original) bytes for 36 keys.  Time: 152,823ms.
> 2011-09-30 08:22:30.713 TimedOutException()
> 2011-09-30 08:22:30.714 	at org.apache.cassandra.thrift.Cassandra$insert_result.read(Cassandra.java:15214)
> 2011-09-30 08:22:30.714 	at org.apache.cassandra.thrift.Cassandra$Client.recv_insert(Cassandra.java:858)
> 2011-09-30 08:22:30.715 	at org.apache.cassandra.thrift.Cassandra$Client.insert(Cassandra.java:830)
> ....
> 2011-09-30 08:23:18.902 Document stored with count records : 9186 Time elapsed (ms) : 2705
> 2011-09-30 08:23:19.240 Received records to store with total count : 8931
> 2011-09-30 08:23:19.898 flushing high-traffic column family ColumnFamilyStore(table='KsFullIdx', columnFamily='ForwardStringValues')
> 2011-09-30 08:23:19.898 Enqueuing flush of Memtable-ForwardStringValues@1365818075(53315582/2559687119 serialized/live bytes, 1158851 ops)
> 2011-09-30 08:23:19.899 Writing Memtable-ForwardStringValues@1365818075(53315582/2559687119 serialized/live bytes, 1158851 ops)
> 2011-09-30 08:23:21.725 Document stored with count records : 8931 Part DB (ms) : 2484
> ....
> 2011-09-30 08:23:27.158 Document stored with count records : 12923 Time elapsed (ms) : 1126
> 2011-09-30 08:23:28.319 Completed flushing /media/datadrive1/capd.cassandra.capd/dbdatafile/KsFullIdx/ForwardStringValues-g-517-Data.db (136428318 bytes)
> 2011-09-30 08:23:30.719 Received records to store with total count : 41543
> ...
> 2011-09-30 08:23:37.734 Received records to store with total count : 13653
> 2011-09-30 08:23:40.508 Document stored with count records : 13653 Part DB (ms) : 2774
> ...
> 
> 
> Machine 1 : 08:23:41 : the timeout exception as listed below
> =========
> First one is a StorageProxy.mutate of 100 rows. 
> Second one is a single row insert using thrift for a simple and small row/column/value. 
> Third is again the same batch of 100 rows. 
> Fourth is another single row insert (small row/column/value).
> 
> 2011-09-30 08:22:59.967 Received records to store with total count : 10224
> 2011-09-30 08:23:00.231 Document stored with count records : 22179 Part DB (ms) : 1120
> 2011-09-30 08:23:00.232 Document stored with count records : 22179 Time elapsed (ms) : 1362
> 2011-09-30 08:23:00.521 Document stored with count records : 10224 Part DB (ms) : 553
> 2011-09-30 08:23:00.521 Document stored with count records : 10224 Time elapsed (ms) : 604
> 2011-09-30 08:23:02.291 Received records to store with total count : 8169
> 2011-09-30 08:23:02.825 Document stored with count records : 8169 Part DB (ms) : 533
> 2011-09-30 08:23:02.825 Document stored with count records : 8169 Time elapsed (ms) : 604
> 2011-09-30 08:23:03.762 Received records to store with total count : 15874
> 2011-09-30 08:23:04.861 Document stored with count records : 15874 Part DB (ms) : 1098
> 2011-09-30 08:23:04.861 Document stored with count records : 15874 Time elapsed (ms) : 1257
> 2011-09-30 08:23:04.888 Received records to store with total count : 20536
> 2011-09-30 08:23:06.082 Received records to store with total count : 18046
> 2011-09-30 08:23:06.244 Received records to store with total count : 67283
> 2011-09-30 08:23:06.408 Received records to store with total count : 41107
> 2011-09-30 08:23:38.592 Compacted to /media/datadrive1/capd.cassandra.capd/dbdatafile/KsFullIdx/ReverseStringValues-tmp-g-622-Data.db.  958,447,377 to 958,447,377 (~100% of original) bytes for 74,860 keys.  Time: 148,318ms.
> 2011-09-30 08:23:41.553 java.util.concurrent.TimeoutException
> 2011-09-30 08:23:41.554 	at org.apache.cassandra.service.AbstractWriteResponseHandler.get(AbstractWriteResponseHandler.java:69)
> 2011-09-30 08:23:41.554 	at org.apache.cassandra.service.StorageProxy.mutate(StorageProxy.java:169)
> 2011-09-30 08:23:41.554 	at be.landc.services.search.server.db.baseserver.indexsearch.store.cassandra.CassandraStore.storeAnnotationsProxy(CassandraStore.java:1280)
> 2011-09-30 08:23:41.554 	at be.landc.services.search.server.db.baseserver.indexsearch.store.cassandra.CassandraStore.storeAnnotations(CassandraStore.java:1018)
> 2011-09-30 08:23:41.555 	at be.landc.services.search.server.db.baseserver.AnnotationStoreDb.storeAnnotations(AnnotationStoreDb.java:181)
> 2011-09-30 08:23:41.555 	at be.landc.services.search.server.db.baseserver.AnnotationStoreDb.processDocument(AnnotationStoreDb.java:333)
> 2011-09-30 08:23:41.555 	at be.landc.services.search.server.db.baseserver.AnnotationStoreDb.processDocument(AnnotationStoreDb.java:290)
> 2011-09-30 08:23:41.555 	at be.landc.services.search.server.base.SearchServerDelegatingBase.processDocument(SearchServerDelegatingBase.java:93)
> 2011-09-30 08:23:41.556 	at sun.reflect.GeneratedMethodAccessor133.invoke(Unknown Source)
> 2011-09-30 08:23:41.556 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
> 2011-09-30 08:23:41.556 	at java.lang.reflect.Method.invoke(Method.java:597)
> 2011-09-30 08:23:41.556 	at be.landc.framework.service.server.shared.Server.execute(Server.java:134)
> 2011-09-30 08:23:41.557 	at be.landc.framework.service.server.rmi.RMIServer.execute(RMIServer.java:80)
> 2011-09-30 08:23:41.557 	at sun.reflect.GeneratedMethodAccessor108.invoke(Unknown Source)
> 2011-09-30 08:23:41.557 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
> 2011-09-30 08:23:41.557 	at java.lang.reflect.Method.invoke(Method.java:597)
> 2011-09-30 08:23:41.558 	at sun.rmi.server.UnicastServerRef.dispatch(UnicastServerRef.java:305)
> 2011-09-30 08:23:41.558 	at sun.rmi.transport.Transport$1.run(Transport.java:159)
> 2011-09-30 08:23:41.558 	at java.security.AccessController.doPrivileged(Native Method)
> 2011-09-30 08:23:41.558 	at sun.rmi.transport.Transport.serviceCall(Transport.java:155)
> 2011-09-30 08:23:41.559 	at sun.rmi.transport.tcp.TCPTransport.handleMessages(TCPTransport.java:535)
> 2011-09-30 08:23:41.559 	at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(TCPTransport.java:790)
> 2011-09-30 08:23:41.559 	at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run(TCPTransport.java:649)
> 2011-09-30 08:23:41.559 	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> 2011-09-30 08:23:41.560 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> 2011-09-30 08:23:41.560 	at java.lang.Thread.run(Thread.java:662)
> 2011-09-30 08:23:42.285 TimedOutException()
> 2011-09-30 08:23:42.285 	at org.apache.cassandra.thrift.Cassandra$insert_result.read(Cassandra.java:15214)
> 2011-09-30 08:23:42.286 	at org.apache.cassandra.thrift.Cassandra$Client.recv_insert(Cassandra.java:858)
> 2011-09-30 08:23:42.286 	at org.apache.cassandra.thrift.Cassandra$Client.insert(Cassandra.java:830)
> 2011-09-30 08:23:42.286 	at be.landc.services.search.server.db.baseserver.indexsearch.store.cassandra.CassandraStore.storeAnnotationsProxy(CassandraStore.java:1236)
> 2011-09-30 08:23:42.286 	at be.landc.services.search.server.db.baseserver.indexsearch.store.cassandra.CassandraStore.storeAnnotations(CassandraStore.java:1018)
> 2011-09-30 08:23:42.286 	at be.landc.services.search.server.db.baseserver.AnnotationStoreDb.storeAnnotations(AnnotationStoreDb.java:181)
> 2011-09-30 08:23:42.286 	at be.landc.services.search.server.db.baseserver.AnnotationStoreDb.processDocument(AnnotationStoreDb.java:333)
> 2011-09-30 08:23:42.287 	at be.landc.services.search.server.db.baseserver.AnnotationStoreDb.processDocument(AnnotationStoreDb.java:290)
> 2011-09-30 08:23:42.287 	at be.landc.services.search.server.base.SearchServerDelegatingBase.processDocument(SearchServerDelegatingBase.java:93)
> 2011-09-30 08:23:42.287 	at sun.reflect.GeneratedMethodAccessor133.invoke(Unknown Source)
> 2011-09-30 08:23:42.287 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
> 2011-09-30 08:23:42.287 	at java.lang.reflect.Method.invoke(Method.java:597)
> 2011-09-30 08:23:42.287 	at be.landc.framework.service.server.shared.Server.execute(Server.java:134)
> 2011-09-30 08:23:42.288 	at be.landc.framework.service.server.rmi.RMIServer.execute(RMIServer.java:80)
> 2011-09-30 08:23:42.288 	at sun.reflect.GeneratedMethodAccessor108.invoke(Unknown Source)
> 2011-09-30 08:23:42.288 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
> 2011-09-30 08:23:42.288 	at java.lang.reflect.Method.invoke(Method.java:597)
> 2011-09-30 08:23:42.288 	at sun.rmi.server.UnicastServerRef.dispatch(UnicastServerRef.java:305)
> 2011-09-30 08:23:42.288 	at sun.rmi.transport.Transport$1.run(Transport.java:159)
> 2011-09-30 08:23:42.289 	at java.security.AccessController.doPrivileged(Native Method)
> 2011-09-30 08:23:42.289 	at sun.rmi.transport.Transport.serviceCall(Transport.java:155)
> 2011-09-30 08:23:42.289 	at sun.rmi.transport.tcp.TCPTransport.handleMessages(TCPTransport.java:535)
> 2011-09-30 08:23:42.289 	at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(TCPTransport.java:790)
> 2011-09-30 08:23:42.289 	at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run(TCPTransport.java:649)
> 2011-09-30 08:23:42.290 	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> 2011-09-30 08:23:42.290 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> 2011-09-30 08:23:42.290 	at java.lang.Thread.run(Thread.java:662)
> 2011-09-30 08:23:43.305 java.util.concurrent.TimeoutException
> 2011-09-30 08:23:43.305 	at org.apache.cassandra.service.AbstractWriteResponseHandler.get(AbstractWriteResponseHandler.java:69)
> 2011-09-30 08:23:43.306 	at org.apache.cassandra.service.StorageProxy.mutate(StorageProxy.java:169)
> 2011-09-30 08:23:43.306 	at be.landc.services.search.server.db.baseserver.indexsearch.store.cassandra.CassandraStore.storeAnnotationsProxy(CassandraStore.java:1280)
> 2011-09-30 08:23:43.306 	at be.landc.services.search.server.db.baseserver.indexsearch.store.cassandra.CassandraStore.storeAnnotations(CassandraStore.java:1018)
> 2011-09-30 08:23:43.306 	at be.landc.services.search.server.db.baseserver.AnnotationStoreDb.storeAnnotations(AnnotationStoreDb.java:181)
> 2011-09-30 08:23:43.306 	at be.landc.services.search.server.db.baseserver.AnnotationStoreDb.processDocument(AnnotationStoreDb.java:333)
> 2011-09-30 08:23:43.307 	at be.landc.services.search.server.db.baseserver.AnnotationStoreDb.processDocument(AnnotationStoreDb.java:290)
> 2011-09-30 08:23:43.307 	at be.landc.services.search.server.base.SearchServerDelegatingBase.processDocument(SearchServerDelegatingBase.java:93)
> 2011-09-30 08:23:43.307 	at sun.reflect.GeneratedMethodAccessor133.invoke(Unknown Source)
> 2011-09-30 08:23:43.307 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
> 2011-09-30 08:23:43.308 	at java.lang.reflect.Method.invoke(Method.java:597)
> 2011-09-30 08:23:43.308 	at be.landc.framework.service.server.shared.Server.execute(Server.java:134)
> 2011-09-30 08:23:43.308 	at be.landc.framework.service.server.rmi.RMIServer.execute(RMIServer.java:80)
> 2011-09-30 08:23:43.308 	at sun.reflect.GeneratedMethodAccessor108.invoke(Unknown Source)
> 2011-09-30 08:23:43.309 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
> 2011-09-30 08:23:43.309 	at java.lang.reflect.Method.invoke(Method.java:597)
> 2011-09-30 08:23:43.309 	at sun.rmi.server.UnicastServerRef.dispatch(UnicastServerRef.java:305)
> 2011-09-30 08:23:43.309 	at sun.rmi.transport.Transport$1.run(Transport.java:159)
> 2011-09-30 08:23:43.309 	at java.security.AccessController.doPrivileged(Native Method)
> 2011-09-30 08:23:43.310 	at sun.rmi.transport.Transport.serviceCall(Transport.java:155)
> 2011-09-30 08:23:43.310 	at sun.rmi.transport.tcp.TCPTransport.handleMessages(TCPTransport.java:535)
> 2011-09-30 08:23:43.310 	at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(TCPTransport.java:790)
> 2011-09-30 08:23:43.310 	at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run(TCPTransport.java:649)
> 2011-09-30 08:23:43.310 	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> 2011-09-30 08:23:43.311 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> 2011-09-30 08:23:43.311 	at java.lang.Thread.run(Thread.java:662)
> 2011-09-30 08:23:44.385 TimedOutException()
> 2011-09-30 08:23:44.385 	at org.apache.cassandra.thrift.Cassandra$insert_result.read(Cassandra.java:15214)
> 2011-09-30 08:23:44.386 	at org.apache.cassandra.thrift.Cassandra$Client.recv_insert(Cassandra.java:858)
> 2011-09-30 08:23:44.386 	at org.apache.cassandra.thrift.Cassandra$Client.insert(Cassandra.java:830)
> 2011-09-30 08:23:44.386 	at be.landc.services.search.server.db.baseserver.indexsearch.store.cassandra.CassandraStore.storeAnnotationsProxy(CassandraStore.java:1242)
> 2011-09-30 08:23:44.386 	at be.landc.services.search.server.db.baseserver.indexsearch.store.cassandra.CassandraStore.storeAnnotations(CassandraStore.java:1018)
> 2011-09-30 08:23:44.387 	at be.landc.services.search.server.db.baseserver.AnnotationStoreDb.storeAnnotations(AnnotationStoreDb.java:181)
> 2011-09-30 08:23:44.387 	at be.landc.services.search.server.db.baseserver.AnnotationStoreDb.processDocument(AnnotationStoreDb.java:333)
> 2011-09-30 08:23:44.387 	at be.landc.services.search.server.db.baseserver.AnnotationStoreDb.processDocument(AnnotationStoreDb.java:290)
> 2011-09-30 08:23:44.387 	at be.landc.services.search.server.base.SearchServerDelegatingBase.processDocument(SearchServerDelegatingBase.java:93)
> 2011-09-30 08:23:44.388 	at sun.reflect.GeneratedMethodAccessor133.invoke(Unknown Source)
> 2011-09-30 08:23:44.388 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
> 2011-09-30 08:23:44.388 	at java.lang.reflect.Method.invoke(Method.java:597)
> 2011-09-30 08:23:44.388 	at be.landc.framework.service.server.shared.Server.execute(Server.java:134)
> 2011-09-30 08:23:44.389 	at be.landc.framework.service.server.rmi.RMIServer.execute(RMIServer.java:80)
> 2011-09-30 08:23:44.389 	at sun.reflect.GeneratedMethodAccessor108.invoke(Unknown Source)
> 2011-09-30 08:23:44.389 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
> 2011-09-30 08:23:44.389 	at java.lang.reflect.Method.invoke(Method.java:597)
> 2011-09-30 08:23:44.390 	at sun.rmi.server.UnicastServerRef.dispatch(UnicastServerRef.java:305)
> 2011-09-30 08:23:44.390 	at sun.rmi.transport.Transport$1.run(Transport.java:159)
> 2011-09-30 08:23:44.390 	at java.security.AccessController.doPrivileged(Native Method)
> 2011-09-30 08:23:44.390 	at sun.rmi.transport.Transport.serviceCall(Transport.java:155)
> 2011-09-30 08:23:44.391 	at sun.rmi.transport.tcp.TCPTransport.handleMessages(TCPTransport.java:535)
> 2011-09-30 08:23:44.391 	at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(TCPTransport.java:790)
> 2011-09-30 08:23:44.391 	at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run(TCPTransport.java:649)
> 2011-09-30 08:23:44.391 	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> 2011-09-30 08:23:44.392 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> 2011-09-30 08:23:44.392 	at java.lang.Thread.run(Thread.java:662)
> 2011-09-30 08:24:00.970 Received records to store with total count : 27928
> 2011-09-30 08:24:02.191 Document stored with count records : 27928 Part DB (ms) : 1220
> 2011-09-30 08:24:02.191 Document stored with count records : 27928 Time elapsed (ms) : 1339
> 2011-09-30 08:24:07.851 Received records to store with total count : 23548
> 2011-09-30 08:24:08.241 Received records to store with total count : 27686
> 2011-09-30 08:24:08.910 Document stored with count records : 23548 Part DB (ms) : 1058
> 2011-09-30 08:24:08.910 Document stored with count records : 23548 Time elapsed (ms) : 1328
> 2011-09-30 08:24:09.193 Received records to store with total count : 26117
> 2011-09-30 08:24:09.507 Document stored with count records : 27686 Part DB (ms) : 1265
> 2011-09-30 08:24:09.507 Document stored with count records : 27686 Time elapsed (ms) : 1495
> 2011-09-30 08:24:10.499 Received records to store with total count : 73102
> 2011-09-30 08:24:10.818 Received records to store with total count : 29409
> 2011-09-30 08:24:11.972 Document stored with count records : 29409 Part DB (ms) : 1153
> 2011-09-30 08:24:11.972 Document stored with count records : 29409 Time elapsed (ms) : 1383
> 2011-09-30 08:24:12.374 Received records to store with total count : 53034
> 2011-09-30 08:24:13.376 Document stored with count records : 73102 Part DB (ms) : 2876
> 2011-09-30 08:24:13.376 Document stored with count records : 73102 Time elapsed (ms) : 3303
> 2011-09-30 08:24:13.515 Received records to store with total count : 70652
> 2011-09-30 08:24:14.220 Received records to store with total count : 47999
> 2011-09-30 08:24:14.984 Document stored with count records : 53034 Part DB (ms) : 2609
> 2011-09-30 08:24:14.984 Document stored with count records : 53034 Time elapsed (ms) : 3133
> 2011-09-30 08:24:16.806 Creating new commitlog segment ../../../../data/capd.cassandra.capd/dbcommitlog/CommitLog-1317363856806.log
> 2011-09-30 08:24:17.153 Received records to store with total count : 9021
> 
> -----Original Message-----
> From: Desimpel, Ignace [mailto:Ignace.Desimpel@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:jbellis@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 <Ig...@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/HintsColumnFam
>> 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(ColumnSort
>> 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(ConcurrentS
>> kipListMap.java:1493)
>> 
>>                 at
>> java.util.concurrent.ConcurrentSkipListMap.<init>(ConcurrentSkipListMa
>> 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$IndexedBlock
>> 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(AbstractIt
>> erator.java:140)
>> 
>>                 at
>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.ja
>> va:135)
>> 
>>                 at
>> org.apache.cassandra.db.columniterator.SSTableSliceIterator.hasNext(SS
>> TableSliceIterator.java:108)
>> 
>>                 at
>> org.apache.commons.collections.iterators.CollatingIterator.set(Collati
>> ngIterator.java:283)
>> 
>>                 at
>> org.apache.commons.collections.iterators.CollatingIterator.least(Colla
>> tingIterator.java:326)
>> 
>>                 at
>> org.apache.commons.collections.iterators.CollatingIterator.next(Collat
>> ingIterator.java:230)
>> 
>>                 at
>> org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingIterat
>> or.java:69)
>> 
>>                 at
>> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIt
>> erator.java:140)
>> 
>>                 at
>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.ja
>> va:135)
>> 
>>                 at
>> org.apache.cassandra.db.filter.SliceQueryFilter.collectReducedColumns(
>> SliceQueryFilter.java:116)
>> 
>>                 at
>> org.apache.cassandra.db.filter.QueryFilter.collectCollatedColumns(Quer
>> yFilter.java:142)
>> 
>>                 at
>> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFam
>> ilyStore.java:1427)
>> 
>>                 at
>> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamily
>> Store.java:1304)
>> 
>>                 at
>> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamily
>> Store.java:1261)
>> 
>>                 at
>> org.apache.cassandra.db.HintedHandOffManager.sendRow(HintedHandOffMana
>> ger.java:155)
>> 
>>                 at
>> org.apache.cassandra.db.HintedHandOffManager.deliverHintsToEndpoint(Hi
>> ntedHandOffManager.java:350)
>> 
>>                 at
>> org.apache.cassandra.db.HintedHandOffManager.access$100(HintedHandOffM
>> anager.java:89)
>> 
>>                 at
>> org.apache.cassandra.db.HintedHandOffManager$2.runMayThrow(HintedHandO
>> ffManager.java:397)
>> 
>>                 at
>> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30
>> )
>> 
>>                 at
>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecu
>> 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(ColumnSort
>> 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


RE: invalid column name length 0

Posted by "Desimpel, Ignace" <Ig...@nuance.com>.
Wanted to see if I still would similar exceptions if I lowered the load by reducing the replication factor.
I saw that in the log files a lot of 'GC parNew' messages and started a complete new run (empty database) with smaller JVM -Xmn1000M and with replication factor 1 (before it was 3).
Currently running, 10 % user cpu, 3% iowait, 'iostat await' 60 on commitlog device, 'iostat await 50' on datafiles device, 'iostat svctm 3', 'vmstat si 0', 'vmstat so 0'
I still get the DEAD/UP messages, less 'GC parNew' messages (some parNew take more than 1 sec), but the system recovers and continues.
But the weird thing is that sometimes I still get java.util.concurrent.TimeoutException exceptions , as well on batch inserts (per 100 rows) and on single row inserts (with just one 64 char row key and one 4 char column name and 64 char column value). Rpc_timeout is 36000 !

I looked up what the other 3 machines were doing at that time. The 'received record...' and 'Document stored ...' are msgs of our application.
Below a stack trace of exceptions occurring at the 'same time', just 3 seconds time difference, on machine 1. 
Machine 1 : 08:23:41 : the timeout exception stack trace as listed at end.
Machine 2:  Nothing special
=========
....
2011-09-30 08:23:20.888 Completed flushing /media/datadrive1/capd.cassandra.capd/dbdatafile/KsFullIdx/ForwardShortValues-g-149-Data.db (131582322 bytes)
2011-09-30 08:23:20.889 Writing Memtable-ForwardLabelValues@260608040(47867232/1983693805 serialized/live bytes, 1150959 ops)
2011-09-30 08:23:23.002 Document stored with count records : 55336 Part DB (ms) : 3666
...
2011-09-30 08:23:33.467 Creating new commitlog segment ../../../../data/capd.cassandra.capd/dbcommitlog/CommitLog-1317363813467.log
2011-09-30 08:23:33.915 Document stored with count records : 9046 Part DB (ms) : 653
....
Machine 3: Is compacting. Also got a timeout exception about 36 seconds before the exception of machine 1
=========
2011-09-30 08:22:34.458 Compacted to /media/datadrive1/capd.cassandra.capd/dbdatafile/KsFullIdx/ReverseStringValues-tmp-g-622-Data.db.  966,991,510 to 965,780,805 (~99% of original) bytes for 74,345 keys.  Time: 208,416ms.
2011-09-30 08:22:37.474 Compacting large row KsFullIdx/ForwardStringValues:3237393737 (723544055 bytes) incrementally
2011-09-30 08:22:37.751 Document stored with count records : 13149 Part DB (ms) : 1613
2011-09-30 08:22:37.751 Document stored with count records : 13149 Time elapsed (ms) : 1748
2011-09-30 08:23:14.149 java.util.concurrent.TimeoutException
2011-09-30 08:23:14.149 	at org.apache.cassandra.service.AbstractWriteResponseHandler.get(AbstractWriteResponseHandler.java:69)
2011-09-30 08:23:14.149 	at org.apache.cassandra.service.StorageProxy.mutate(StorageProxy.java:169)
2011-09-30 08:23:14.150 	at be.landc.services.search.server.db.baseserver.indexsearch.store.cassandra.CassandraStore.storeAnnotationsProxy(CassandraStore.java:1290)
2011-09-30 08:23:33.548 Received records to store with total count : 23277
2011-09-30 08:23:34.705 Received records to store with total count : 9816
2011-09-30 08:23:35.152 Received records to store with total count : 8646
2011-09-30 08:23:35.323 Received records to store with total count : 49888
2011-09-30 08:23:35.475 Document stored with count records : 23277 Part DB (ms) : 1926
....
2011-09-30 08:23:39.627 Received records to store with total count : 8443
2011-09-30 08:23:40.081 Document stored with count records : 26862 Part DB (ms) : 3192
2011-09-30 08:23:40.081 Document stored with count records : 26862 Time elapsed (ms) : 3363
...
Machine 4: Nothing special, but also got the timeout exception for a single row insert at 08:22:30
=========
2011-09-30 08:22:12.569 Compacted to /media/datadrive1/capd.cassandra.capd/dbdatafile/KsFullIdx/ForwardIntegerValues-tmp-g-150-Data.db.  586,039,224 to 586,036,960 (~99% of original) bytes for 36 keys.  Time: 152,823ms.
2011-09-30 08:22:30.713 TimedOutException()
2011-09-30 08:22:30.714 	at org.apache.cassandra.thrift.Cassandra$insert_result.read(Cassandra.java:15214)
2011-09-30 08:22:30.714 	at org.apache.cassandra.thrift.Cassandra$Client.recv_insert(Cassandra.java:858)
2011-09-30 08:22:30.715 	at org.apache.cassandra.thrift.Cassandra$Client.insert(Cassandra.java:830)
....
2011-09-30 08:23:18.902 Document stored with count records : 9186 Time elapsed (ms) : 2705
2011-09-30 08:23:19.240 Received records to store with total count : 8931
2011-09-30 08:23:19.898 flushing high-traffic column family ColumnFamilyStore(table='KsFullIdx', columnFamily='ForwardStringValues')
2011-09-30 08:23:19.898 Enqueuing flush of Memtable-ForwardStringValues@1365818075(53315582/2559687119 serialized/live bytes, 1158851 ops)
2011-09-30 08:23:19.899 Writing Memtable-ForwardStringValues@1365818075(53315582/2559687119 serialized/live bytes, 1158851 ops)
2011-09-30 08:23:21.725 Document stored with count records : 8931 Part DB (ms) : 2484
....
2011-09-30 08:23:27.158 Document stored with count records : 12923 Time elapsed (ms) : 1126
2011-09-30 08:23:28.319 Completed flushing /media/datadrive1/capd.cassandra.capd/dbdatafile/KsFullIdx/ForwardStringValues-g-517-Data.db (136428318 bytes)
2011-09-30 08:23:30.719 Received records to store with total count : 41543
...
2011-09-30 08:23:37.734 Received records to store with total count : 13653
2011-09-30 08:23:40.508 Document stored with count records : 13653 Part DB (ms) : 2774
...


Machine 1 : 08:23:41 : the timeout exception as listed below
=========
First one is a StorageProxy.mutate of 100 rows. 
Second one is a single row insert using thrift for a simple and small row/column/value. 
Third is again the same batch of 100 rows. 
Fourth is another single row insert (small row/column/value).

2011-09-30 08:22:59.967 Received records to store with total count : 10224
2011-09-30 08:23:00.231 Document stored with count records : 22179 Part DB (ms) : 1120
2011-09-30 08:23:00.232 Document stored with count records : 22179 Time elapsed (ms) : 1362
2011-09-30 08:23:00.521 Document stored with count records : 10224 Part DB (ms) : 553
2011-09-30 08:23:00.521 Document stored with count records : 10224 Time elapsed (ms) : 604
2011-09-30 08:23:02.291 Received records to store with total count : 8169
2011-09-30 08:23:02.825 Document stored with count records : 8169 Part DB (ms) : 533
2011-09-30 08:23:02.825 Document stored with count records : 8169 Time elapsed (ms) : 604
2011-09-30 08:23:03.762 Received records to store with total count : 15874
2011-09-30 08:23:04.861 Document stored with count records : 15874 Part DB (ms) : 1098
2011-09-30 08:23:04.861 Document stored with count records : 15874 Time elapsed (ms) : 1257
2011-09-30 08:23:04.888 Received records to store with total count : 20536
2011-09-30 08:23:06.082 Received records to store with total count : 18046
2011-09-30 08:23:06.244 Received records to store with total count : 67283
2011-09-30 08:23:06.408 Received records to store with total count : 41107
2011-09-30 08:23:38.592 Compacted to /media/datadrive1/capd.cassandra.capd/dbdatafile/KsFullIdx/ReverseStringValues-tmp-g-622-Data.db.  958,447,377 to 958,447,377 (~100% of original) bytes for 74,860 keys.  Time: 148,318ms.
2011-09-30 08:23:41.553 java.util.concurrent.TimeoutException
2011-09-30 08:23:41.554 	at org.apache.cassandra.service.AbstractWriteResponseHandler.get(AbstractWriteResponseHandler.java:69)
2011-09-30 08:23:41.554 	at org.apache.cassandra.service.StorageProxy.mutate(StorageProxy.java:169)
2011-09-30 08:23:41.554 	at be.landc.services.search.server.db.baseserver.indexsearch.store.cassandra.CassandraStore.storeAnnotationsProxy(CassandraStore.java:1280)
2011-09-30 08:23:41.554 	at be.landc.services.search.server.db.baseserver.indexsearch.store.cassandra.CassandraStore.storeAnnotations(CassandraStore.java:1018)
2011-09-30 08:23:41.555 	at be.landc.services.search.server.db.baseserver.AnnotationStoreDb.storeAnnotations(AnnotationStoreDb.java:181)
2011-09-30 08:23:41.555 	at be.landc.services.search.server.db.baseserver.AnnotationStoreDb.processDocument(AnnotationStoreDb.java:333)
2011-09-30 08:23:41.555 	at be.landc.services.search.server.db.baseserver.AnnotationStoreDb.processDocument(AnnotationStoreDb.java:290)
2011-09-30 08:23:41.555 	at be.landc.services.search.server.base.SearchServerDelegatingBase.processDocument(SearchServerDelegatingBase.java:93)
2011-09-30 08:23:41.556 	at sun.reflect.GeneratedMethodAccessor133.invoke(Unknown Source)
2011-09-30 08:23:41.556 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
2011-09-30 08:23:41.556 	at java.lang.reflect.Method.invoke(Method.java:597)
2011-09-30 08:23:41.556 	at be.landc.framework.service.server.shared.Server.execute(Server.java:134)
2011-09-30 08:23:41.557 	at be.landc.framework.service.server.rmi.RMIServer.execute(RMIServer.java:80)
2011-09-30 08:23:41.557 	at sun.reflect.GeneratedMethodAccessor108.invoke(Unknown Source)
2011-09-30 08:23:41.557 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
2011-09-30 08:23:41.557 	at java.lang.reflect.Method.invoke(Method.java:597)
2011-09-30 08:23:41.558 	at sun.rmi.server.UnicastServerRef.dispatch(UnicastServerRef.java:305)
2011-09-30 08:23:41.558 	at sun.rmi.transport.Transport$1.run(Transport.java:159)
2011-09-30 08:23:41.558 	at java.security.AccessController.doPrivileged(Native Method)
2011-09-30 08:23:41.558 	at sun.rmi.transport.Transport.serviceCall(Transport.java:155)
2011-09-30 08:23:41.559 	at sun.rmi.transport.tcp.TCPTransport.handleMessages(TCPTransport.java:535)
2011-09-30 08:23:41.559 	at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(TCPTransport.java:790)
2011-09-30 08:23:41.559 	at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run(TCPTransport.java:649)
2011-09-30 08:23:41.559 	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
2011-09-30 08:23:41.560 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
2011-09-30 08:23:41.560 	at java.lang.Thread.run(Thread.java:662)
2011-09-30 08:23:42.285 TimedOutException()
2011-09-30 08:23:42.285 	at org.apache.cassandra.thrift.Cassandra$insert_result.read(Cassandra.java:15214)
2011-09-30 08:23:42.286 	at org.apache.cassandra.thrift.Cassandra$Client.recv_insert(Cassandra.java:858)
2011-09-30 08:23:42.286 	at org.apache.cassandra.thrift.Cassandra$Client.insert(Cassandra.java:830)
2011-09-30 08:23:42.286 	at be.landc.services.search.server.db.baseserver.indexsearch.store.cassandra.CassandraStore.storeAnnotationsProxy(CassandraStore.java:1236)
2011-09-30 08:23:42.286 	at be.landc.services.search.server.db.baseserver.indexsearch.store.cassandra.CassandraStore.storeAnnotations(CassandraStore.java:1018)
2011-09-30 08:23:42.286 	at be.landc.services.search.server.db.baseserver.AnnotationStoreDb.storeAnnotations(AnnotationStoreDb.java:181)
2011-09-30 08:23:42.286 	at be.landc.services.search.server.db.baseserver.AnnotationStoreDb.processDocument(AnnotationStoreDb.java:333)
2011-09-30 08:23:42.287 	at be.landc.services.search.server.db.baseserver.AnnotationStoreDb.processDocument(AnnotationStoreDb.java:290)
2011-09-30 08:23:42.287 	at be.landc.services.search.server.base.SearchServerDelegatingBase.processDocument(SearchServerDelegatingBase.java:93)
2011-09-30 08:23:42.287 	at sun.reflect.GeneratedMethodAccessor133.invoke(Unknown Source)
2011-09-30 08:23:42.287 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
2011-09-30 08:23:42.287 	at java.lang.reflect.Method.invoke(Method.java:597)
2011-09-30 08:23:42.287 	at be.landc.framework.service.server.shared.Server.execute(Server.java:134)
2011-09-30 08:23:42.288 	at be.landc.framework.service.server.rmi.RMIServer.execute(RMIServer.java:80)
2011-09-30 08:23:42.288 	at sun.reflect.GeneratedMethodAccessor108.invoke(Unknown Source)
2011-09-30 08:23:42.288 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
2011-09-30 08:23:42.288 	at java.lang.reflect.Method.invoke(Method.java:597)
2011-09-30 08:23:42.288 	at sun.rmi.server.UnicastServerRef.dispatch(UnicastServerRef.java:305)
2011-09-30 08:23:42.288 	at sun.rmi.transport.Transport$1.run(Transport.java:159)
2011-09-30 08:23:42.289 	at java.security.AccessController.doPrivileged(Native Method)
2011-09-30 08:23:42.289 	at sun.rmi.transport.Transport.serviceCall(Transport.java:155)
2011-09-30 08:23:42.289 	at sun.rmi.transport.tcp.TCPTransport.handleMessages(TCPTransport.java:535)
2011-09-30 08:23:42.289 	at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(TCPTransport.java:790)
2011-09-30 08:23:42.289 	at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run(TCPTransport.java:649)
2011-09-30 08:23:42.290 	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
2011-09-30 08:23:42.290 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
2011-09-30 08:23:42.290 	at java.lang.Thread.run(Thread.java:662)
2011-09-30 08:23:43.305 java.util.concurrent.TimeoutException
2011-09-30 08:23:43.305 	at org.apache.cassandra.service.AbstractWriteResponseHandler.get(AbstractWriteResponseHandler.java:69)
2011-09-30 08:23:43.306 	at org.apache.cassandra.service.StorageProxy.mutate(StorageProxy.java:169)
2011-09-30 08:23:43.306 	at be.landc.services.search.server.db.baseserver.indexsearch.store.cassandra.CassandraStore.storeAnnotationsProxy(CassandraStore.java:1280)
2011-09-30 08:23:43.306 	at be.landc.services.search.server.db.baseserver.indexsearch.store.cassandra.CassandraStore.storeAnnotations(CassandraStore.java:1018)
2011-09-30 08:23:43.306 	at be.landc.services.search.server.db.baseserver.AnnotationStoreDb.storeAnnotations(AnnotationStoreDb.java:181)
2011-09-30 08:23:43.306 	at be.landc.services.search.server.db.baseserver.AnnotationStoreDb.processDocument(AnnotationStoreDb.java:333)
2011-09-30 08:23:43.307 	at be.landc.services.search.server.db.baseserver.AnnotationStoreDb.processDocument(AnnotationStoreDb.java:290)
2011-09-30 08:23:43.307 	at be.landc.services.search.server.base.SearchServerDelegatingBase.processDocument(SearchServerDelegatingBase.java:93)
2011-09-30 08:23:43.307 	at sun.reflect.GeneratedMethodAccessor133.invoke(Unknown Source)
2011-09-30 08:23:43.307 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
2011-09-30 08:23:43.308 	at java.lang.reflect.Method.invoke(Method.java:597)
2011-09-30 08:23:43.308 	at be.landc.framework.service.server.shared.Server.execute(Server.java:134)
2011-09-30 08:23:43.308 	at be.landc.framework.service.server.rmi.RMIServer.execute(RMIServer.java:80)
2011-09-30 08:23:43.308 	at sun.reflect.GeneratedMethodAccessor108.invoke(Unknown Source)
2011-09-30 08:23:43.309 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
2011-09-30 08:23:43.309 	at java.lang.reflect.Method.invoke(Method.java:597)
2011-09-30 08:23:43.309 	at sun.rmi.server.UnicastServerRef.dispatch(UnicastServerRef.java:305)
2011-09-30 08:23:43.309 	at sun.rmi.transport.Transport$1.run(Transport.java:159)
2011-09-30 08:23:43.309 	at java.security.AccessController.doPrivileged(Native Method)
2011-09-30 08:23:43.310 	at sun.rmi.transport.Transport.serviceCall(Transport.java:155)
2011-09-30 08:23:43.310 	at sun.rmi.transport.tcp.TCPTransport.handleMessages(TCPTransport.java:535)
2011-09-30 08:23:43.310 	at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(TCPTransport.java:790)
2011-09-30 08:23:43.310 	at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run(TCPTransport.java:649)
2011-09-30 08:23:43.310 	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
2011-09-30 08:23:43.311 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
2011-09-30 08:23:43.311 	at java.lang.Thread.run(Thread.java:662)
2011-09-30 08:23:44.385 TimedOutException()
2011-09-30 08:23:44.385 	at org.apache.cassandra.thrift.Cassandra$insert_result.read(Cassandra.java:15214)
2011-09-30 08:23:44.386 	at org.apache.cassandra.thrift.Cassandra$Client.recv_insert(Cassandra.java:858)
2011-09-30 08:23:44.386 	at org.apache.cassandra.thrift.Cassandra$Client.insert(Cassandra.java:830)
2011-09-30 08:23:44.386 	at be.landc.services.search.server.db.baseserver.indexsearch.store.cassandra.CassandraStore.storeAnnotationsProxy(CassandraStore.java:1242)
2011-09-30 08:23:44.386 	at be.landc.services.search.server.db.baseserver.indexsearch.store.cassandra.CassandraStore.storeAnnotations(CassandraStore.java:1018)
2011-09-30 08:23:44.387 	at be.landc.services.search.server.db.baseserver.AnnotationStoreDb.storeAnnotations(AnnotationStoreDb.java:181)
2011-09-30 08:23:44.387 	at be.landc.services.search.server.db.baseserver.AnnotationStoreDb.processDocument(AnnotationStoreDb.java:333)
2011-09-30 08:23:44.387 	at be.landc.services.search.server.db.baseserver.AnnotationStoreDb.processDocument(AnnotationStoreDb.java:290)
2011-09-30 08:23:44.387 	at be.landc.services.search.server.base.SearchServerDelegatingBase.processDocument(SearchServerDelegatingBase.java:93)
2011-09-30 08:23:44.388 	at sun.reflect.GeneratedMethodAccessor133.invoke(Unknown Source)
2011-09-30 08:23:44.388 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
2011-09-30 08:23:44.388 	at java.lang.reflect.Method.invoke(Method.java:597)
2011-09-30 08:23:44.388 	at be.landc.framework.service.server.shared.Server.execute(Server.java:134)
2011-09-30 08:23:44.389 	at be.landc.framework.service.server.rmi.RMIServer.execute(RMIServer.java:80)
2011-09-30 08:23:44.389 	at sun.reflect.GeneratedMethodAccessor108.invoke(Unknown Source)
2011-09-30 08:23:44.389 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
2011-09-30 08:23:44.389 	at java.lang.reflect.Method.invoke(Method.java:597)
2011-09-30 08:23:44.390 	at sun.rmi.server.UnicastServerRef.dispatch(UnicastServerRef.java:305)
2011-09-30 08:23:44.390 	at sun.rmi.transport.Transport$1.run(Transport.java:159)
2011-09-30 08:23:44.390 	at java.security.AccessController.doPrivileged(Native Method)
2011-09-30 08:23:44.390 	at sun.rmi.transport.Transport.serviceCall(Transport.java:155)
2011-09-30 08:23:44.391 	at sun.rmi.transport.tcp.TCPTransport.handleMessages(TCPTransport.java:535)
2011-09-30 08:23:44.391 	at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(TCPTransport.java:790)
2011-09-30 08:23:44.391 	at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run(TCPTransport.java:649)
2011-09-30 08:23:44.391 	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
2011-09-30 08:23:44.392 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
2011-09-30 08:23:44.392 	at java.lang.Thread.run(Thread.java:662)
2011-09-30 08:24:00.970 Received records to store with total count : 27928
2011-09-30 08:24:02.191 Document stored with count records : 27928 Part DB (ms) : 1220
2011-09-30 08:24:02.191 Document stored with count records : 27928 Time elapsed (ms) : 1339
2011-09-30 08:24:07.851 Received records to store with total count : 23548
2011-09-30 08:24:08.241 Received records to store with total count : 27686
2011-09-30 08:24:08.910 Document stored with count records : 23548 Part DB (ms) : 1058
2011-09-30 08:24:08.910 Document stored with count records : 23548 Time elapsed (ms) : 1328
2011-09-30 08:24:09.193 Received records to store with total count : 26117
2011-09-30 08:24:09.507 Document stored with count records : 27686 Part DB (ms) : 1265
2011-09-30 08:24:09.507 Document stored with count records : 27686 Time elapsed (ms) : 1495
2011-09-30 08:24:10.499 Received records to store with total count : 73102
2011-09-30 08:24:10.818 Received records to store with total count : 29409
2011-09-30 08:24:11.972 Document stored with count records : 29409 Part DB (ms) : 1153
2011-09-30 08:24:11.972 Document stored with count records : 29409 Time elapsed (ms) : 1383
2011-09-30 08:24:12.374 Received records to store with total count : 53034
2011-09-30 08:24:13.376 Document stored with count records : 73102 Part DB (ms) : 2876
2011-09-30 08:24:13.376 Document stored with count records : 73102 Time elapsed (ms) : 3303
2011-09-30 08:24:13.515 Received records to store with total count : 70652
2011-09-30 08:24:14.220 Received records to store with total count : 47999
2011-09-30 08:24:14.984 Document stored with count records : 53034 Part DB (ms) : 2609
2011-09-30 08:24:14.984 Document stored with count records : 53034 Time elapsed (ms) : 3133
2011-09-30 08:24:16.806 Creating new commitlog segment ../../../../data/capd.cassandra.capd/dbcommitlog/CommitLog-1317363856806.log
2011-09-30 08:24:17.153 Received records to store with total count : 9021

-----Original Message-----
From: Desimpel, Ignace [mailto:Ignace.Desimpel@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:jbellis@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 <Ig...@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/HintsColumnFam
> 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(ColumnSort
> 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(ConcurrentS
> kipListMap.java:1493)
>
>                 at
> java.util.concurrent.ConcurrentSkipListMap.<init>(ConcurrentSkipListMa
> 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$IndexedBlock
> 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(AbstractIt
> erator.java:140)
>
>                 at
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.ja
> va:135)
>
>                 at
> org.apache.cassandra.db.columniterator.SSTableSliceIterator.hasNext(SS
> TableSliceIterator.java:108)
>
>                 at
> org.apache.commons.collections.iterators.CollatingIterator.set(Collati
> ngIterator.java:283)
>
>                 at
> org.apache.commons.collections.iterators.CollatingIterator.least(Colla
> tingIterator.java:326)
>
>                 at
> org.apache.commons.collections.iterators.CollatingIterator.next(Collat
> ingIterator.java:230)
>
>                 at
> org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingIterat
> or.java:69)
>
>                 at
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIt
> erator.java:140)
>
>                 at
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.ja
> va:135)
>
>                 at
> org.apache.cassandra.db.filter.SliceQueryFilter.collectReducedColumns(
> SliceQueryFilter.java:116)
>
>                 at
> org.apache.cassandra.db.filter.QueryFilter.collectCollatedColumns(Quer
> yFilter.java:142)
>
>                 at
> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFam
> ilyStore.java:1427)
>
>                 at
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamily
> Store.java:1304)
>
>                 at
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamily
> Store.java:1261)
>
>                 at
> org.apache.cassandra.db.HintedHandOffManager.sendRow(HintedHandOffMana
> ger.java:155)
>
>                 at
> org.apache.cassandra.db.HintedHandOffManager.deliverHintsToEndpoint(Hi
> ntedHandOffManager.java:350)
>
>                 at
> org.apache.cassandra.db.HintedHandOffManager.access$100(HintedHandOffM
> anager.java:89)
>
>                 at
> org.apache.cassandra.db.HintedHandOffManager$2.runMayThrow(HintedHandO
> ffManager.java:397)
>
>                 at
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30
> )
>
>                 at
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecu
> 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(ColumnSort
> 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

RE: invalid column name length 0

Posted by "Desimpel, Ignace" <Ig...@nuance.com>.
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:jbellis@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 <Ig...@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/HintsColumnFam
> 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(ColumnSort
> 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(ConcurrentS
> kipListMap.java:1493)
>
>                 at
> java.util.concurrent.ConcurrentSkipListMap.<init>(ConcurrentSkipListMa
> 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$IndexedBlock
> 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(AbstractIt
> erator.java:140)
>
>                 at
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.ja
> va:135)
>
>                 at
> org.apache.cassandra.db.columniterator.SSTableSliceIterator.hasNext(SS
> TableSliceIterator.java:108)
>
>                 at
> org.apache.commons.collections.iterators.CollatingIterator.set(Collati
> ngIterator.java:283)
>
>                 at
> org.apache.commons.collections.iterators.CollatingIterator.least(Colla
> tingIterator.java:326)
>
>                 at
> org.apache.commons.collections.iterators.CollatingIterator.next(Collat
> ingIterator.java:230)
>
>                 at
> org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingIterat
> or.java:69)
>
>                 at
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIt
> erator.java:140)
>
>                 at
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.ja
> va:135)
>
>                 at
> org.apache.cassandra.db.filter.SliceQueryFilter.collectReducedColumns(
> SliceQueryFilter.java:116)
>
>                 at
> org.apache.cassandra.db.filter.QueryFilter.collectCollatedColumns(Quer
> yFilter.java:142)
>
>                 at
> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFam
> ilyStore.java:1427)
>
>                 at
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamily
> Store.java:1304)
>
>                 at
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamily
> Store.java:1261)
>
>                 at
> org.apache.cassandra.db.HintedHandOffManager.sendRow(HintedHandOffMana
> ger.java:155)
>
>                 at
> org.apache.cassandra.db.HintedHandOffManager.deliverHintsToEndpoint(Hi
> ntedHandOffManager.java:350)
>
>                 at
> org.apache.cassandra.db.HintedHandOffManager.access$100(HintedHandOffM
> anager.java:89)
>
>                 at
> org.apache.cassandra.db.HintedHandOffManager$2.runMayThrow(HintedHandO
> ffManager.java:397)
>
>                 at
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30
> )
>
>                 at
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecu
> 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(ColumnSort
> 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

Re: invalid column name length 0

Posted by Jonathan Ellis <jb...@gmail.com>.
Is this upgraded from an earlier version?

On Tue, Sep 27, 2011 at 8:51 AM, Desimpel, Ignace
<Ig...@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/HintsColumnFamily-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(ColumnSortedMap.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(ConcurrentSkipListMap.java:1493)
>
>                 at
> java.util.concurrent.ConcurrentSkipListMap.<init>(ConcurrentSkipListMap.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$IndexedBlockFetcher.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(AbstractIterator.java:140)
>
>                 at
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:135)
>
>                 at
> org.apache.cassandra.db.columniterator.SSTableSliceIterator.hasNext(SSTableSliceIterator.java:108)
>
>                 at
> org.apache.commons.collections.iterators.CollatingIterator.set(CollatingIterator.java:283)
>
>                 at
> org.apache.commons.collections.iterators.CollatingIterator.least(CollatingIterator.java:326)
>
>                 at
> org.apache.commons.collections.iterators.CollatingIterator.next(CollatingIterator.java:230)
>
>                 at
> org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingIterator.java:69)
>
>                 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.filter.SliceQueryFilter.collectReducedColumns(SliceQueryFilter.java:116)
>
>                 at
> org.apache.cassandra.db.filter.QueryFilter.collectCollatedColumns(QueryFilter.java:142)
>
>                 at
> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1427)
>
>                 at
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1304)
>
>                 at
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1261)
>
>                 at
> org.apache.cassandra.db.HintedHandOffManager.sendRow(HintedHandOffManager.java:155)
>
>                 at
> org.apache.cassandra.db.HintedHandOffManager.deliverHintsToEndpoint(HintedHandOffManager.java:350)
>
>                 at
> org.apache.cassandra.db.HintedHandOffManager.access$100(HintedHandOffManager.java:89)
>
>                 at
> org.apache.cassandra.db.HintedHandOffManager$2.runMayThrow(HintedHandOffManager.java:397)
>
>                 at
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
>
>                 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: 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(ColumnSortedMap.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