Re: Heavy writes ok for single node, but failed for cluster

2011-04-29 Thread aaron morton
Can you try turning up the logging to TRACE, if you know how turn it up for the 
org.apache.cassandra.gms.FailureDetector and org.apache.cassandra.gms.Gossiper 
loggers

You'll see a lot of messages. Some of the interesting ones will be...

FailureDetector.java (line 128) reporting /127.0.0.2  - logged when the end 
point reports 
FailureDetector.java (line 149) PHI for /127.0.0.2 : 0.9127981004059462 - 
logged when an end point is checked before possibly marking dead

There will also be messages from the gossiper about receiving and sending.

Aaron
 
On 29 Apr 2011, at 14:50, Sheng Chen wrote:

 Thank you for your patch. 
 I believe the latter version I used (the latest 0.7 branch) includes the 
 patch, but the problem remains.
 
 Is there anything else that may block this heartbeat, like gc?
 Here are some logs during heartbeat failure.
 
  INFO [GossipTasks:1] 2011-04-29 07:25:09,716 Gossiper.java (line 228) 
 InetAddress /192.168.125.51 is now dead.
  INFO [GossipStage:1] 2011-04-29 07:25:20,128 Gossiper.java (line 609) 
 InetAddress /192.168.125.51 is now UP
 --
  INFO [FlushWriter:1] 2011-04-29 07:25:00,135 Memtable.java (line 172) 
 Completed flushing /data/cassandra/data/Keyspace1/Standard1-f-4041-Data.db 
 (76170930 bytes)
  INFO [COMMIT-LOG-WRITER] 2011-04-29 07:25:00,140 CommitLog.java (line 440) 
 Discarding obsolete commit 
 log:CommitLogSegment(/commitlog/CommitLog-1304033061025.log)
  INFO [ScheduledTasks:1] 2011-04-29 07:25:20,399 GCInspector.java (line 128) 
 GC for ParNew: 511 ms, -71757976 reclaimed leaving 6614730672 used; max is 
 8466202624
  INFO [ScheduledTasks:1] 2011-04-29 07:25:34,632 GCInspector.java (line 128) 
 GC for ParNew: 238 ms, 612197280 reclaimed leaving 4429643304 used; max is 
 8466202624
 --
 
  INFO [GossipTasks:1] 2011-04-29 07:17:48,833 Gossiper.java (line 228) 
 InetAddress /192.168.125.51 is now dead.
  INFO [GossipStage:1] 2011-04-29 07:17:48,854 Gossiper.java (line 609) 
 InetAddress /192.168.125.51 is now UP
 --
  INFO [GossipTasks:1] 2011-04-29 07:17:33,503 Gossiper.java (line 228) 
 InetAddress /192.168.125.49 is now dead.
  INFO [MutationStage:3] 2011-04-29 07:17:47,751 ColumnFamilyStore.java (line 
 1064) Enqueuing flush of Memtable-Standard1@461979209(61555223 bytes, 1218975 
 operations)
  INFO [FlushWriter:1] 2011-04-29 07:17:47,751 Memtable.java (line 157) 
 Writing Memtable-Standard1@461979209(61555223 bytes, 1218975 operations)
  INFO [GossipStage:1] 2011-04-29 07:17:49,123 Gossiper.java (line 609) 
 InetAddress /192.168.125.49 is now UP
 --
 
  INFO [GossipTasks:1] 2011-04-29 07:09:46,072 Gossiper.java (line 228) 
 InetAddress /192.168.125.51 is now dead.
  INFO [GossipStage:1] 2011-04-29 07:09:59,587 Gossiper.java (line 609) 
 InetAddress /192.168.125.51 is now UP
 --
  INFO [FlushWriter:1] 2011-04-29 07:09:36,917 Memtable.java (line 172) 
 Completed flushing /data/cassandra/data/Keyspace1/Standard1-f-4005-Data.db 
 (76199863 bytes)
  INFO [COMMIT-LOG-WRITER] 2011-04-29 07:09:36,921 CommitLog.java (line 440) 
 Discarding obsolete commit 
 log:CommitLogSegment(/commitlog/CommitLog-1304032122510.log)
  INFO [COMMIT-LOG-WRITER] 2011-04-29 07:09:38,089 CommitLogSegment.java (line 
 50) Creating new commitlog segment /commitlog/CommitLog-1304032178089.log
  INFO [ScheduledTasks:1] 2011-04-29 07:09:59,858 GCInspector.java (line 128) 
 GC for ParNew: 511 ms, -61586864 reclaimed leaving 6496071272 used; max is 
 8466202624
  WARN [ScheduledTasks:1] 2011-04-29 07:09:59,858 MessagingService.java (line 
 504) Dropped 1 REQUEST_RESPONSE messages in the last 5000ms
  INFO [ScheduledTasks:1] 2011-04-29 07:09:59,859 StatusLogger.java (line 51) 
 Pool NameActive   Pending
  INFO [ScheduledTasks:1] 2011-04-29 07:09:59,859 StatusLogger.java (line 66) 
 ReadStage 0 0
  INFO [ScheduledTasks:1] 2011-04-29 07:09:59,859 StatusLogger.java (line 66) 
 RequestResponseStage  0 0
  INFO [ScheduledTasks:1] 2011-04-29 07:09:59,860 StatusLogger.java (line 66) 
 ReadRepairStage   0 0
  INFO [ScheduledTasks:1] 2011-04-29 07:09:59,860 StatusLogger.java (line 66) 
 MutationStage 0 0
  INFO [ScheduledTasks:1] 2011-04-29 07:09:59,860 StatusLogger.java (line 66) 
 GossipStage   0 0
  INFO [ScheduledTasks:1] 2011-04-29 07:09:59,860 StatusLogger.java (line 66) 
 AntiEntropyStage  0 0
  INFO [ScheduledTasks:1] 2011-04-29 07:09:59,861 StatusLogger.java (line 66) 
 MigrationStage0 0
  INFO [ScheduledTasks:1] 2011-04-29 07:09:59,861 StatusLogger.java (line 66) 
 StreamStage   0 0
  INFO [ScheduledTasks:1] 2011-04-29 07:09:59,861 StatusLogger.java (line 66) 
 MemtablePostFlusher   0 0
  INFO [ScheduledTasks:1] 2011-04-29 07:09:59,861 StatusLogger.java (line 66) 
 FILEUTILS-DELETE-POOL 0   

Re: Heavy writes ok for single node, but failed for cluster

2011-04-28 Thread Sheng Chen
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 10 -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 10 -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



Re: Heavy writes ok for single node, but failed for cluster

2011-04-28 Thread Jonathan Ellis
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 10 -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 10 -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


Re: Heavy writes ok for single node, but failed for cluster

2011-04-28 Thread Sheng Chen
Thank you for your patch.
I believe the latter version I used (the latest 0.7 branch) includes the
patch, but the problem remains.

Is there anything else that may block this heartbeat, like gc?
Here are some logs during heartbeat failure.

 INFO [GossipTasks:1] 2011-04-29 07:25:09,716 Gossiper.java (line 228)
InetAddress /192.168.125.51 is now dead.
 INFO [GossipStage:1] 2011-04-29 07:25:20,128 Gossiper.java (line 609)
InetAddress /192.168.125.51 is now UP
--
 INFO [FlushWriter:1] 2011-04-29 07:25:00,135 Memtable.java (line 172)
Completed flushing /data/cassandra/data/Keyspace1/Standard1-f-4041-Data.db
(76170930 bytes)
 INFO [COMMIT-LOG-WRITER] 2011-04-29 07:25:00,140 CommitLog.java (line 440)
Discarding obsolete commit
log:CommitLogSegment(/commitlog/CommitLog-1304033061025.log)
 INFO [ScheduledTasks:1] 2011-04-29 07:25:20,399 GCInspector.java (line 128)
GC for ParNew: 511 ms, -71757976 reclaimed leaving 6614730672 used; max is
8466202624
 INFO [ScheduledTasks:1] 2011-04-29 07:25:34,632 GCInspector.java (line 128)
GC for ParNew: 238 ms, 612197280 reclaimed leaving 4429643304 used; max is
8466202624
--

 INFO [GossipTasks:1] 2011-04-29 07:17:48,833 Gossiper.java (line 228)
InetAddress /192.168.125.51 is now dead.
 INFO [GossipStage:1] 2011-04-29 07:17:48,854 Gossiper.java (line 609)
InetAddress /192.168.125.51 is now UP
--
 INFO [GossipTasks:1] 2011-04-29 07:17:33,503 Gossiper.java (line 228)
InetAddress /192.168.125.49 is now dead.
 INFO [MutationStage:3] 2011-04-29 07:17:47,751 ColumnFamilyStore.java (line
1064) Enqueuing flush of Memtable-Standard1@461979209(61555223 bytes,
1218975 operations)
 INFO [FlushWriter:1] 2011-04-29 07:17:47,751 Memtable.java (line 157)
Writing Memtable-Standard1@461979209(61555223 bytes, 1218975 operations)
 INFO [GossipStage:1] 2011-04-29 07:17:49,123 Gossiper.java (line 609)
InetAddress /192.168.125.49 is now UP
--

 INFO [GossipTasks:1] 2011-04-29 07:09:46,072 Gossiper.java (line 228)
InetAddress /192.168.125.51 is now dead.
 INFO [GossipStage:1] 2011-04-29 07:09:59,587 Gossiper.java (line 609)
InetAddress /192.168.125.51 is now UP
--
 INFO [FlushWriter:1] 2011-04-29 07:09:36,917 Memtable.java (line 172)
Completed flushing /data/cassandra/data/Keyspace1/Standard1-f-4005-Data.db
(76199863 bytes)
 INFO [COMMIT-LOG-WRITER] 2011-04-29 07:09:36,921 CommitLog.java (line 440)
Discarding obsolete commit
log:CommitLogSegment(/commitlog/CommitLog-1304032122510.log)
 INFO [COMMIT-LOG-WRITER] 2011-04-29 07:09:38,089 CommitLogSegment.java
(line 50) Creating new commitlog segment
/commitlog/CommitLog-1304032178089.log
 INFO [ScheduledTasks:1] 2011-04-29 07:09:59,858 GCInspector.java (line 128)
GC for ParNew: 511 ms, -61586864 reclaimed leaving 6496071272 used; max is
8466202624
 WARN [ScheduledTasks:1] 2011-04-29 07:09:59,858 MessagingService.java (line
504) Dropped 1 REQUEST_RESPONSE messages in the last 5000ms
 INFO [ScheduledTasks:1] 2011-04-29 07:09:59,859 StatusLogger.java (line 51)
Pool NameActive   Pending
 INFO [ScheduledTasks:1] 2011-04-29 07:09:59,859 StatusLogger.java (line 66)
ReadStage 0 0
 INFO [ScheduledTasks:1] 2011-04-29 07:09:59,859 StatusLogger.java (line 66)
RequestResponseStage  0 0
 INFO [ScheduledTasks:1] 2011-04-29 07:09:59,860 StatusLogger.java (line 66)
ReadRepairStage   0 0
 INFO [ScheduledTasks:1] 2011-04-29 07:09:59,860 StatusLogger.java (line 66)
MutationStage 0 0
 INFO [ScheduledTasks:1] 2011-04-29 07:09:59,860 StatusLogger.java (line 66)
GossipStage   0 0
 INFO [ScheduledTasks:1] 2011-04-29 07:09:59,860 StatusLogger.java (line 66)
AntiEntropyStage  0 0
 INFO [ScheduledTasks:1] 2011-04-29 07:09:59,861 StatusLogger.java (line 66)
MigrationStage0 0
 INFO [ScheduledTasks:1] 2011-04-29 07:09:59,861 StatusLogger.java (line 66)
StreamStage   0 0
 INFO [ScheduledTasks:1] 2011-04-29 07:09:59,861 StatusLogger.java (line 66)
MemtablePostFlusher   0 0
 INFO [ScheduledTasks:1] 2011-04-29 07:09:59,861 StatusLogger.java (line 66)
FILEUTILS-DELETE-POOL 0 0
 INFO [ScheduledTasks:1] 2011-04-29 07:09:59,862 StatusLogger.java (line 66)
FlushWriter   0 0
 INFO [ScheduledTasks:1] 2011-04-29 07:09:59,862 StatusLogger.java (line 66)
MiscStage 0 0
 INFO [ScheduledTasks:1] 2011-04-29 07:09:59,862 StatusLogger.java (line 66)
FlushSorter   0 0
 INFO [ScheduledTasks:1] 2011-04-29 07:09:59,863 StatusLogger.java (line 66)
InternalResponseStage 0 0
 INFO [ScheduledTasks:1] 2011-04-29 07:09:59,863 StatusLogger.java (line 66)
HintedHandoff 1 2
 INFO [ScheduledTasks:1] 2011-04-29 07:09:59,863 StatusLogger.java (line 70)
CompactionManager

Heavy writes ok for single node, but failed for cluster

2011-04-27 Thread Sheng Chen
I succeeded to insert 1 billion records into a single node cassandra,
 bin/stress -d cas01 -o insert -n 10 -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 10 -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
4. I set hinted_handoff_enabled: false, but still see lots of handoff logs


Re: Heavy writes ok for single node, but failed for cluster

2011-04-27 Thread Sylvain Lebresne
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 10 -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 10 -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