cassandra-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From albert_e <dongz...@gmail.com>
Subject Re: Hintedhandoff will never complete when a BIG rowmutation
Date Mon, 28 Jun 2010 16:31:05 GMT
In 0.6.2, HH sending MUTATION message using the same OutboundTcpConnection
with READ message. When HH transfering big mutation data, read operation
will be blocked and read storm may cause 100% disk I/O of the dest node.


2010/6/28 Jonathan Ellis <jbellis@gmail.com>

> Yes, you should increase your timeout if you are hinting big mutations
> (or big rows that were built from smaller mutations).
>
> 2010/6/28 Lu Ming <xluke@live.com>:
> > Hi:
> >      These days I found my Cassandra is strange, much slower than before.
> > And I Spent much time to figure it out and today I got the answer.
> >
> >     Some bad buy keeps on writing many data day and night, then made a
> very
> > big row mutation which size is about 140M.
> > In this period I restarted some Cassandra nodes, and when the nodes is
> alive
> > again, them got some hintedhandoff messages.
> > HintedHandOffManager.sendMessage() will send the rowmutations to these
> > nodes, but the rowmutation is too big to finish transferring in
> > 8 seconds (defined in DatabaseDescriptor.getRpcTimeout()), and
> sendMessage()
> > return false when got a TimeoutException.
> >
> > Every one hour HintedHandOffManager will check hintedhandoff ColumnFamily
> > then send out the big rowmutations to alive nodes,
> > It fails again because of the TimeoutException, so the task will never
> > finish and the big rowmutation is sending again and again.
> >
> >    In multi-datacenters,  a big rowmutation can not be transferred
> > in several seconds. so It is a potential risk when  a big
> > rowmutation occurs.
> >
> >
> >
> > Luke
>
>
>
> --
> Jonathan Ellis
> Project Chair, Apache Cassandra
> co-founder of Riptano, the source for professional Cassandra support
> http://riptano.com
>

Mime
View raw message