[ 
https://issues.apache.org/jira/browse/CASSANDRA-6592?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sylvain Lebresne updated CASSANDRA-6592:
----------------------------------------

    Attachment: 6592-2.0.txt

Summing up where we are:
* Tyler tested the 2nd patch which seems to fix the issue.
* It seems MemoryMeter can return pretty funky results (~100MB for a statement 
that Tyler confirmed as really small) when measuring CFMetaData. While there is 
some theory, it's no yet entirely clear why it can produce such big error.
* While the 2nd patch is likely fine, given the previous point and to take 0 
risk with 1.2, we're reverted CASSANDRA-6107 there.

Now, for 2.0, attaching a rebase of the patch with correct handling of null 
values (which was missing from the first patch -- could maybe be slightly 
cleaner to make MemoryMeter.measureDeep handle null on the long run).

I'll note that while the patch seems to properly fix this issue, it might be 
worth digging a bit more separatly to understand what was confusing MemoryMeter 
exactly, since we use it in other places.


> IllegalArgumentException when Preparing Statements
> --------------------------------------------------
>
>                 Key: CASSANDRA-6592
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-6592
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>            Reporter: Tyler Hobbs
>            Assignee: Lyuben Todorov
>            Priority: Critical
>             Fix For: 1.2.14, 2.0.5
>
>         Attachments: 0001-Remove-concurrent-use-of-MemoryMeter-instance.txt, 
> 0001-Switch-to-adding-fields-manually-in-measureForPrepared.txt, 6592-2.0.txt
>
>
> When preparing a lot of statements with the python native driver, I 
> occasionally get an error response with an error that corresponds to the 
> following stacktrace in the cassandra logs:
> {noformat}
> ERROR [Native-Transport-Requests:126] 2014-01-11 13:58:05,503 
> ErrorMessage.java (line 210) Unexpected exception during request
> java.lang.IllegalArgumentException
>         at 
> com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap.checkArgument(ConcurrentLinkedHashMap.java:259)
>         at 
> com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$BoundedEntryWeigher.weightOf(ConcurrentLinkedHashMap.java:1448)
>         at 
> com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap.put(ConcurrentLinkedHashMap.java:764)
>         at 
> com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap.put(ConcurrentLinkedHashMap.java:743)
>         at 
> org.apache.cassandra.cql3.QueryProcessor.storePreparedStatement(QueryProcessor.java:255)
>         at 
> org.apache.cassandra.cql3.QueryProcessor.prepare(QueryProcessor.java:221)
>         at 
> org.apache.cassandra.transport.messages.PrepareMessage.execute(PrepareMessage.java:77)
>         at 
> org.apache.cassandra.transport.Message$Dispatcher.messageReceived(Message.java:287)
>         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$Worker.runTask(ThreadPoolExecutor.java:895)
>         at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:918)
>         at java.lang.Thread.run(Thread.java:662)
> {noformat}
> Looking at the CLHM source, this means we're giving the statement a weight 
> that's less than 1.  I'll also note that these errors frequently happen in 
> clumps of 2 or 3 at a time.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)

Reply via email to