[ 
https://issues.apache.org/jira/browse/CASSANDRA-7510?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14179082#comment-14179082
 ] 

Tyler Hobbs commented on CASSANDRA-7510:
----------------------------------------

I tested this out with a ccm cluster, and there's still a pretty large gap (8 
seconds) between when the notification is sent and when the native protocol 
server is actually started.  It looks like most of that is due to waiting for 
gossip to settle:

{noformat}
 INFO [main] 2014-10-21 15:43:50,430 StorageService.java (line 1515) Node 
/127.0.0.4 state jump to normal
 INFO [main] 2014-10-21 15:43:50,444 CassandraDaemon.java (line 543) Waiting 
for gossip to settle before accepting client requests...
 INFO [CompactionExecutor:7] 2014-10-21 15:43:53,066 ColumnFamilyStore.java 
(line 794) Enqueuing flush of Memtable-compactions_in_progress@1446920230(0/0 
serialized/live bytes, 1 ops)
 INFO [FlushWriter:1] 2014-10-21 15:43:53,067 Memtable.java (line 355) Writing 
Memtable-compactions_in_progress@1446920230(0/0 serialized/live bytes, 1 ops)
 INFO [FlushWriter:1] 2014-10-21 15:43:53,084 Memtable.java (line 395) 
Completed flushing 
/home/thobbs/.ccm/devcluster/node4/data/system/compactions_in_progress/system-compactions_in_progress-jb-2-Data.db
 (42 bytes) for commitlog position ReplayPosition(segmentId=1413924189183, 
position=13357643)
 INFO [CompactionExecutor:7] 2014-10-21 15:43:53,091 CompactionTask.java (line 
287) Compacted 7 sstables to 
[/home/thobbs/.ccm/devcluster/node4/data/duration_test/ints/duration_test-ints-jb-8,].
  2,672,491 bytes to 2,098,615 (~78% of original) in 2,755ms = 0.726459MB/s.  
4,167 total partitions merged to 3,325.  Partition merge counts were {1:2497, 
2:814, 3:14, }
 INFO [OptionalTasks:1] 2014-10-21 15:43:56,468 MeteredFlusher.java (line 58) 
flushing high-traffic column family CFS(Keyspace='duration_test', 
ColumnFamily='ints') (estimated 20392963 bytes)
 INFO [OptionalTasks:1] 2014-10-21 15:43:56,469 ColumnFamilyStore.java (line 
794) Enqueuing flush of Memtable-ints@1788959623(3053505/20392963 
serialized/live bytes, 136710 ops)
 INFO [FlushWriter:1] 2014-10-21 15:43:56,470 Memtable.java (line 355) Writing 
Memtable-ints@1788959623(3053505/20392963 serialized/live bytes, 136710 ops)
 INFO [FlushWriter:1] 2014-10-21 15:43:56,963 Memtable.java (line 395) 
Completed flushing 
/home/thobbs/.ccm/devcluster/node4/data/duration_test/ints/duration_test-ints-jb-9-Data.db
 (281136 bytes) for commitlog position ReplayPosition(segmentId=1413924189183, 
position=14376181)
 INFO [main] 2014-10-21 15:43:58,445 CassandraDaemon.java (line 575) No gossip 
backlog; proceeding
 INFO [main] 2014-10-21 15:43:58,551 Server.java (line 156) Starting listening 
for CQL clients on /127.0.0.4:9042...
{noformat}

I expect that in larger clusters the gap will be even larger.

I'm not sure that we can do anything about this without adding something new to 
gossip, though.

> Notify clients that bootstrap is finished over binary protocol
> --------------------------------------------------------------
>
>                 Key: CASSANDRA-7510
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-7510
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Joost Reuzel
>            Assignee: Brandon Williams
>            Priority: Minor
>             Fix For: 2.0.12
>
>         Attachments: 7510.txt
>
>
> Currently, Cassandra will notify clients when a new node is added to a 
> cluster. However, that node is typically not usable yet. It first needs to 
> gossip its key range and finish loading all its assigned data before it 
> allows clients to connect. Depending on the amount of data this may take 
> quite a while. The clients in the mean time have no clue about the bootstrap 
> status of that node. The only thing they can do is periodically check if it 
> will accept a connection. 
> My proposal would be to send an additional UP event when the bootstrap is 
> done, this allows clients to mark the node initially as down/unavailable and 
> simply wait for the UP event to arrive.
> Kind regards,
> Joost



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to