cassandra-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Jonathan Ellis <jbel...@gmail.com>
Subject Re: MESSAGE-DESERIALIZER-POOL is slow down by SliceFromReadCommand
Date Mon, 28 Jun 2010 15:44:05 GMT
If you're getting to where you have thousands of backed-up commands on
RRS then it doesn't really matter what executor behavior is, because
you're severely screwed. :)

Often this means you are i/o bound and you should look into row or key
caching.  Possibly you may simply need to add capacity to your
cluster.

2010/6/28 Lu Ming <xluke@live.com>:
> OK, I found the reason.
>
>  The task queue for ROW-READ-STAGE ThreadPoolExecutor  is a  Bounded queue,
> and its size is 4096 (defined by DatabaseDescriptor.getStageQueueSize())
>  When the pending tasks of  ROW-READ-STAGE ThreadPoolExecutor   is more than
> 4096,  the follow-up submitted task will be rejected because the task queue
> is full and maximumPoolSize threads are busy.  In
> ThreadPoolExecutor.CallerRunsPolicy, the thread that invokes execute itself
> runs the rejected task.
> Therefore, the invoking thread MESSAGE-DESERIALIZER-POOL-1 executes the
> SliceFromReadCommand task.
> ReadCommand is much slower than message de-serialization and costs many
> milliseconds, so It look like that MESSAGE-DESERIALIZER-POOL is slow down by
> SliceFromReadCommand.
>
>
> From: Lu Ming
> Sent: Wednesday, June 23, 2010 11:50 PM
> To: user@cassandra.apache.org
> Subject: MESSAGE-DESERIALIZER-POOL is slow down by SliceFromReadCommand
> Yesterday and today , I found there are many pending message (>100,000) to
> be de-serialized in MESSAGE-DESERIALIZER-POOL.
> and I do the thread dump, It seems that some "SliceRead" is NOT executed by
> StageManager.READ_STAGE and executed  by MESSAGE-DESERIALIZER-POOL.
> It is somewhat strange. Read from file  is much slower than deserialization,
> so MESSAGE-DESERIALIZER-POOL slow down.
>
> According to MessageService.receive(), when the MessageType() of a Message
> is not found in StageManager, its VerbHandle will be executed by
> MESSAGE-DESERIALIZER-POOL.
> So the question is:   which type of  SliceFromReadCommand will executed by
> MESSAGE-DESERIALIZER-POOL?
>
>
> "MESSAGE-DESERIALIZER-POOL:1" - Thread t@23
>    java.lang.Thread.State: RUNNABLE
>  at java.io.RandomAccessFile.readBytes(Native Method)
>  at java.io.RandomAccessFile.read(RandomAccessFile.java:322)
>  at
> org.apache.cassandra.io.util.BufferedRandomAccessFile.fillBuffer(BufferedRandomAccessFile.java:210)
>  at
> org.apache.cassandra.io.util.BufferedRandomAccessFile.seek(BufferedRandomAccessFile.java:247)
>  at
> org.apache.cassandra.io.SSTableReader.getFileDataInput(SSTableReader.java:515)
>  at
> org.apache.cassandra.db.filter.SSTableSliceIterator.<init>(SSTableSliceIterator.java:54)
>  at
> org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:68)
>  at
> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:862)
>  at
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:773)
>  at
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:742)
>  at org.apache.cassandra.db.Table.getRow(Table.java:381)
>  at
> org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:65)
>  at org.apache.cassandra.db.ReadVerbHandler.doVerb(ReadVerbHandler.java:70)
>  at
> org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:40)
>  at
> java.util.concurrent.ThreadPoolExecutor$CallerRunsPolicy.rejectedExecution(ThreadPoolExecutor.java:1746)
>  at
> java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:767)
>  at
> java.util.concurrent.ThreadPoolExecutor.execute(ThreadPoolExecutor.java:658)
>  at
> org.apache.cassandra.net.MessagingService.receive(MessagingService.java:365)
>  at
> org.apache.cassandra.net.MessageDeserializationTask.run(MessageDeserializationTask.java:55)
>  at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
>  at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>  at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>  at
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>  at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>  at java.lang.Thread.run(Thread.java:619)
>



-- 
Jonathan Ellis
Project Chair, Apache Cassandra
co-founder of Riptano, the source for professional Cassandra support
http://riptano.com

Mime
View raw message