Shao-Chuan Wang created CASSANDRA-6879:
------------------------------------------
Summary: ConcurrentModificationException while doing range slice
query.
Key: CASSANDRA-6879
URL: https://issues.apache.org/jira/browse/CASSANDRA-6879
Project: Cassandra
Issue Type: Bug
Components: Core
Environment: 2.0.4
Reporter: Shao-Chuan Wang
The paging read request (either from thrift or native) would sporadically fail
due to the race condition between read repair and requesting thread waiting for
read repair results list. The READ_REPAIR is queued in
ReadCallback.maybeResolveForRepair(), and it does not seem to have guarantee
that its resolve() method (which internally create
RangeSliceResponseResolver.Reducer and doing repairResults.addAll inside
RangeSliceResponseResolver.Reducer) would be invoked before the requesting
thread starts waiting on resolver.repairResults. So, there is a small window
that the list is partially populated, while requesting thread starts waiting on
repairResults. I believe for the most of the time, the requesting thread is
either wait for the entire repair results or not waiting for repair results at
all. The original intent here seems to be waiting for repair results always (if
the repair is triggered by repair chance).
{code}
ERROR [Native-Transport-Requests:70827] 2014-03-18 05:00:12,774
ErrorMessage.java (line 222) Unexpected exception during request
java.util.ConcurrentModificationException
at java.util.ArrayList$Itr.checkForComodification(ArrayList.java:859)
at java.util.ArrayList$Itr.next(ArrayList.java:831)
at
org.apache.cassandra.utils.FBUtilities.waitOnFutures(FBUtilities.java:423)
at
org.apache.cassandra.service.StorageProxy.getRangeSlice(StorageProxy.java:1583)
at
org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:188)
at
org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:163)
at
org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:58)
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}
{code}
ERROR [Thrift:1] 2014-03-18 07:18:02,434 CustomTThreadPoolServer.java (line
212) Error occurred during processing of message.
java.util.ConcurrentModificationException
at java.util.ArrayList$Itr.checkForComodification(ArrayList.java:859)
at java.util.ArrayList$Itr.next(ArrayList.java:831)
at
org.apache.cassandra.utils.FBUtilities.waitOnFutures(FBUtilities.java:423)
at
org.apache.cassandra.service.StorageProxy.getRangeSlice(StorageProxy.java:1583)
at
org.apache.cassandra.service.pager.RangeSliceQueryPager.queryNextPage(RangeSliceQueryPager.java:85)
at
org.apache.cassandra.service.pager.AbstractQueryPager.fetchPage(AbstractQueryPager.java:71)
at
org.apache.cassandra.service.pager.RangeSliceQueryPager.fetchPage(RangeSliceQueryPager.java:36)
at
org.apache.cassandra.cql3.statements.SelectStatement.pageCountQuery(SelectStatement.java:202)
at
org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:169)
at
org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:58)
at
org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:188)
at
org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:222)
at
org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:212)
at
org.apache.cassandra.thrift.CassandraServer.execute_cql3_query(CassandraServer.java:1958)
at
org.apache.cassandra.thrift.Cassandra$Processor$execute_cql3_query.getResult(Cassandra.java:4486)
at
org.apache.cassandra.thrift.Cassandra$Processor$execute_cql3_query.getResult(Cassandra.java:4470)
at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39)
at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39)
at
org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:194)
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}
--
This message was sent by Atlassian JIRA
(v6.2#6252)