You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Adam Hattrell (JIRA)" <ji...@apache.org> on 2014/04/16 12:42:15 UTC

[jira] [Created] (CASSANDRA-7044) LWT with SERIAL consistency gives misleading message on WriteTimeout

Adam Hattrell created CASSANDRA-7044:
----------------------------------------

             Summary: LWT with SERIAL consistency gives misleading message on WriteTimeout
                 Key: CASSANDRA-7044
                 URL: https://issues.apache.org/jira/browse/CASSANDRA-7044
             Project: Cassandra
          Issue Type: Bug
          Components: Core
            Reporter: Adam Hattrell


I have a user using LWT with SERIAL consistency.  We see the following stack trace:

{code:none}
ERROR [Native-Transport-Requests:61112] 2014-04-16 10:39:05,437 ErrorMessage.java (line 222) Unexpected exception during request
java.lang.UnsupportedOperationException: Invalid consistency level: SERIAL
        at org.apache.cassandra.db.ConsistencyLevel.blockFor(ConsistencyLevel.java:137)
        at org.apache.cassandra.service.StorageProxy.beginAndRepairPaxos(StorageProxy.java:361)
        at org.apache.cassandra.service.StorageProxy.cas(StorageProxy.java:220)
        at org.apache.cassandra.cql3.statements.ModificationStatement.executeWithCondition(ModificationStatement.java:452)
        at org.apache.cassandra.cql3.statements.ModificationStatement.execute(ModificationStatement.java:407)
        at org.apache.cassandra.cql3.QueryProcessor.executeWithHooks(QueryProcessor.java:201)
        at org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:188)
        at org.apache.cassandra.cql3.QueryProcessor.processPrepared(QueryProcessor.java:358)
        at org.apache.cassandra.transport.messages.ExecuteMessage.execute(ExecuteMessage.java:131)
        at org.apache.cassandra.transport.Message$Dispatcher.messageReceived(Message.java:304)
        at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70)
        at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564)
        at org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:791)
        at org.jboss.netty.handler.execution.ChannelUpstreamEventRunnable.doRun(ChannelUpstreamEventRunnable.java:43)
        at org.jboss.netty.handler.execution.ChannelEventRunnable.run(ChannelEventRunnable.java:67)
        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)
{code}

It looks as though when the paxos round timesout we call blockFor() which currently doesn't support SERIAL.  



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