[ https://issues.apache.org/jira/browse/CASSANDRA-6592?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Sylvain Lebresne updated CASSANDRA-6592: ---------------------------------------- Attachment: 0001-Switch-to-adding-fields-manually-in-measureForPrepared.txt 0001-Remove-concurrent-use-of-MemoryMeter-instance.txt bq. It's possible the object being metered is being concurrently modified? Certainly the CFMetaData permits this. Yes it's possible and yes it's definitively dodgy. But one would hope that MemoryMeter don't return completely random results just because the measured object is mutated (but maybe that assumption is wrong). So that the error due to a concurrent modification of CFMetaData shouldn't exceed a couple of KB. Which might explain occasional negative size computations, but doesn't entirely explain to me why Tyler is seeing "a lot of" statement being measured at more than 15MB. Unless Tyler is building "a lot of" super huge prepared batch statements that is, which I've assumed is not the case but maybe I'm wrong on that part. In any case, I can agree that the current subtraction is misguided (it just doesn't seem to explain it all). Adding each fields individually in the measure method is slightly annoying to maintain so this is what this was trying to avoid but well, that's a bad idea so attaching a patch that changes that. [~thobbs] can you check if that solves both of the errors you see? > 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 > > > 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)