You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@cassandra.apache.org by aaron morton <aa...@thelastpickle.com> on 2011/10/02 13:25:52 UTC

Re: invalid column name length 0

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