[ 
https://issues.apache.org/jira/browse/CASSANDRA-7144?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13992841#comment-13992841
 ] 

Maxime Lamothe-Brassard commented on CASSANDRA-7144:
----------------------------------------------------

I was not using prepared statements. I was doing INSERT and SELECT, no DELETE.

I have some more info now. As I said I rebuilt the data on the node and it 
eliminated the problem. However, a day later, I found myself killing my 
ingestor (doing lots of INSERT), it's a python script using the new cassandra 
python-driver. When I did that I got the exception above. Thinking it was a 
"runtime" bug, I just kept going. Then the hinted-handoff started timing out on 
that box, so I restarted cassandra. From that point on, I would get the same 
exception without ever killing ingestors, at random interval. It seems as if 
killing the script during a query ended up sending data to cassandra that made 
it corrupt something on disk and that from that point on whenever it reached 
that part of the data on disk (I use that liberally, I just mean NOT directly 
from the script doing the ingestion) it would throw the exact same exception, 
leading to the timing out again. Restart of the cassandra node did nothing, I 
had to rebuild the data again. So now I'm very paranoid about killing my 
ingestion.

The ingestion uses UNLOGGED BATCH for some of the data for performance as well 
as normal INSERT.

> CassandraDaemon RowMutation exception
> -------------------------------------
>
>                 Key: CASSANDRA-7144
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-7144
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>         Environment: Ubuntu 12.04 w/ Oracle JVM, 5 nodes cluster. Nodes 2GB / 
> 2 Cores in DigitalOcean.
>            Reporter: Maxime Lamothe-Brassard
>
> First time reporting a bug here, apologies if I'm not posting it in the right 
> space.
> At what seems like random interval, on random nodes in random situations I 
> will get the following exception. After this the hinted handoff start timing 
> out and the node stops participating in the cluster.
> I started seeing these after switching to the Cassandra Python-Driver from 
> the Python-CQL driver.
> {noformat}
> ERROR [WRITE-/10.128.180.108] 2014-05-03 13:45:12,843 CassandraDaemon.java 
> (line 198) Exception in thread Thread[WRITE-/10.128.180.108,5,main]
> java.lang.AssertionError
>       at 
> org.apache.cassandra.db.RowMutation$RowMutationSerializer.serialize(RowMutation.java:271)
>       at 
> org.apache.cassandra.db.RowMutation$RowMutationSerializer.serialize(RowMutation.java:259)
>       at org.apache.cassandra.net.MessageOut.serialize(MessageOut.java:120)
>       at 
> org.apache.cassandra.net.OutboundTcpConnection.writeInternal(OutboundTcpConnection.java:251)
>       at 
> org.apache.cassandra.net.OutboundTcpConnection.writeConnected(OutboundTcpConnection.java:203)
>       at 
> org.apache.cassandra.net.OutboundTcpConnection.run(OutboundTcpConnection.java:151)
> ERROR [WRITE-/10.128.194.70] 2014-05-03 13:45:12,843 CassandraDaemon.java 
> (line 198) Exception in thread Thread[WRITE-/10.128.194.70,5,main]
> java.lang.AssertionError
>       at 
> org.apache.cassandra.db.RowMutation$RowMutationSerializer.serialize(RowMutation.java:271)
>       at 
> org.apache.cassandra.db.RowMutation$RowMutationSerializer.serialize(RowMutation.java:259)
>       at org.apache.cassandra.net.MessageOut.serialize(MessageOut.java:120)
>       at 
> org.apache.cassandra.net.OutboundTcpConnection.writeInternal(OutboundTcpConnection.java:251)
>       at 
> org.apache.cassandra.net.OutboundTcpConnection.writeConnected(OutboundTcpConnection.java:203)
>       at 
> org.apache.cassandra.net.OutboundTcpConnection.run(OutboundTcpConnection.java:151)
> ERROR [MutationStage:118] 2014-05-03 13:45:15,048 CassandraDaemon.java (line 
> 198) Exception in thread Thread[MutationStage:118,5,main]
> java.lang.AssertionError
>       at 
> org.apache.cassandra.db.RowMutation$RowMutationSerializer.serialize(RowMutation.java:271)
>       at 
> org.apache.cassandra.db.RowMutation$RowMutationSerializer.serialize(RowMutation.java:259)
>       at 
> org.apache.cassandra.utils.FBUtilities.serialize(FBUtilities.java:654)
>       at 
> org.apache.cassandra.db.HintedHandOffManager.hintFor(HintedHandOffManager.java:137)
>       at 
> org.apache.cassandra.service.StorageProxy.writeHintForMutation(StorageProxy.java:908)
>       at 
> org.apache.cassandra.service.StorageProxy$6.runMayThrow(StorageProxy.java:881)
>       at 
> org.apache.cassandra.service.StorageProxy$HintRunnable.run(StorageProxy.java:1981)
>       at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
>       at java.util.concurrent.FutureTask.run(FutureTask.java:262)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>       at java.lang.Thread.run(Thread.java:744)
> ERROR [MutationStage:117] 2014-05-03 13:45:15,048 CassandraDaemon.java (line 
> 198) Exception in thread Thread[MutationStage:117,5,main]
> java.lang.AssertionError
>       at 
> org.apache.cassandra.db.RowMutation$RowMutationSerializer.serialize(RowMutation.java:271)
>       at 
> org.apache.cassandra.db.RowMutation$RowMutationSerializer.serialize(RowMutation.java:259)
>       at 
> org.apache.cassandra.utils.FBUtilities.serialize(FBUtilities.java:654)
>       at 
> org.apache.cassandra.db.HintedHandOffManager.hintFor(HintedHandOffManager.java:137)
>       at 
> org.apache.cassandra.service.StorageProxy.writeHintForMutation(StorageProxy.java:908)
>       at 
> org.apache.cassandra.service.StorageProxy$6.runMayThrow(StorageProxy.java:881)
>       at 
> org.apache.cassandra.service.StorageProxy$HintRunnable.run(StorageProxy.java:1981)
>       at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
>       at java.util.concurrent.FutureTask.run(FutureTask.java:262)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>       at java.lang.Thread.run(Thread.java:744)
> {noformat}
> The service must be restarted for the node to come back online. Let me know 
> any additional configuration details needed.



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Reply via email to