This means a node was too busy with something else to send out its
heartbeat. Sometimes this is STW GC. Other times it is a bug (one was
fixed for 0.7.6 in
https://issues.apache.org/jira/browse/CASSANDRA-2554).

On Thu, Apr 28, 2011 at 3:57 AM, Sheng Chen <chensheng2...@gmail.com> wrote:
> Thank you for your advice. Rf>=2 is a good work around.
> I was using 0.7.4 and have updated to the latest 0.7 branch, which includes
> 2554 patch.
> But it doesn't help. I still get lots of UnavailableException after the
> following logs,
>  INFO [GossipTasks:1] 2011-04-28 16:12:17,661 Gossiper.java (line 228)
> InetAddress /192.168.125.49 is now dead.
>  INFO [GossipStage:1] 2011-04-28 16:12:19,627 Gossiper.java (line 609)
> InetAddress /192.168.125.49 is now UP
>  INFO [HintedHandoff:1] 2011-04-28 16:13:11,452 HintedHandOffManager.java
> (line 304) Started hinted handoff for endpoint /192.168.125.49
>  INFO [HintedHandoff:1] 2011-04-28 16:13:11,453 HintedHandOffManager.java
> (line 360) Finished hinted handoff of 0 rows to endpoint /192.168.125.49
> It seems that the gossip failure detection is too sensitive. Is there any
> configuration?
>
>
>
>
>
> 2011/4/27 Sylvain Lebresne <sylv...@datastax.com>
>>
>> On Wed, Apr 27, 2011 at 10:32 AM, Sheng Chen <chensheng2...@gmail.com>
>> wrote:
>> > I succeeded to insert 1 billion records into a single node cassandra,
>> >>> bin/stress -d cas01 -o insert -n 1000000000 -c 5 -S 34 -C5 -t 20
>> > Inserts finished in about 14 hours at a speed of 20k/sec.
>> > But when I added another node, tests always failed with
>> > UnavailableException
>> > in an hour.
>> >>> bin/stress -d cas01,cas02 -o insert -n 1000000000 -c 5 -S 34 -C5 -t 20
>> > Writes speed is also 20k/sec because of the bottleneck in the client, so
>> > the
>> > pressure on each server node should be 50% of the single node test.
>> > Why couldn't they handle?
>> > By default, rf=1, consistency=ONE
>> > Some information that may be helpful,
>> > 1. no warn/error in log file, the cluster is still alive after those
>> > exception
>> > 2. the last logs on both nodes happen to be a compaction complete info
>> > 3. gossip log shows one node is dead and then up again in 3 seconds
>>
>> That's your problem. Once marked down (and since rf=1), when an update for
>> cas02 reach cas01 and cas01 has marked cas02 down, it will throw the
>> UnavailableException.
>>
>> Now, it shouldn't have been marked down and I suspect this is due to
>> https://issues.apache.org/jira/browse/CASSANDRA-2554
>> (even though you didn't tell which version you're using, I suppose
>> this is a 0.7.*).
>>
>> If you apply this patch or use the svn current 0.7 branch, that should
>> hopefully
>> not happen again.
>>
>> Note that if you had rf >= 2, the node would still have been marked down
>> wrongly
>> for 3 seconds, but that would have been transparent to the stress test.
>>
>> > 4. I set hinted_handoff_enabled: false, but still see lots of handoff
>> > logs
>>
>> What are those saying ?
>>
>> --
>> Sylvain
>
>



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

Reply via email to