[ https://issues.apache.org/jira/browse/CASSANDRA-6592?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Sylvain Lebresne updated CASSANDRA-6592: ---------------------------------------- Attachment: 0001-Remove-concurrent-use-of-MemoryMeter-instance.txt Not sure what happens here. In particular the "is larger than allowed" exception suggests MemoryMeter is sometimes returning something whack. That being said, this sound like a race (comments above suggests this doesn't reproduce reliably on a specific statement) and we do use the same MemoryMeter instance concurrently. So while it's not clear why doing so would be wrong, it's probably worth removing that variable from the equation and see if that fixes it. So attaching a patch to switch to a ThreadLocal for the MemoryMeter. [~thobbs], since you seem to be able to reproduce relatively easily, can you try said patch and see if it helps or not? If it doesn't, we might want to revert CASSANDRA-6107 until we understand what's going on. > 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 > > > 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)