@@ -3441,6 +3441,43 @@ def _clear_watcher(conn, expiring_weakref):
34413441 pass
34423442
34433443
3444+ def _fetch_all_pages (connection , query_msg , timeout , fail_on_error = True ):
3445+ response = connection .wait_for_response (query_msg , timeout = timeout , fail_on_error = fail_on_error )
3446+
3447+ if not fail_on_error :
3448+ success , result = response
3449+ if not success :
3450+ return response
3451+ else :
3452+ result = response
3453+
3454+ if not result or not result .paging_state :
3455+ return response if not fail_on_error else result
3456+
3457+ all_rows = list (result .parsed_rows ) if result .parsed_rows else []
3458+
3459+ # Fetch remaining pages
3460+ while result and result .paging_state :
3461+ query_msg .paging_state = result .paging_state
3462+ page_response = connection .wait_for_response (query_msg , timeout = timeout , fail_on_error = fail_on_error )
3463+
3464+ if not fail_on_error :
3465+ page_success , page_result = page_response
3466+ if not page_success :
3467+ return page_response
3468+ result = page_result
3469+ else :
3470+ result = page_response
3471+
3472+ if result and result .parsed_rows :
3473+ all_rows .extend (result .parsed_rows )
3474+
3475+ if result :
3476+ result .parsed_rows = all_rows
3477+
3478+ return (True , result ) if not fail_on_error else result
3479+
3480+
34443481class ControlConnection (object ):
34453482 """
34463483 Internal
@@ -3634,23 +3671,27 @@ def _try_connect(self, host):
36343671 sel_peers = self ._get_peers_query (self .PeersQueryType .PEERS , connection )
36353672 sel_local = self ._SELECT_LOCAL if self ._token_meta_enabled else self ._SELECT_LOCAL_NO_TOKENS
36363673 peers_query = QueryMessage (query = maybe_add_timeout_to_query (sel_peers , self ._metadata_request_timeout ),
3637- consistency_level = ConsistencyLevel .ONE )
3674+ consistency_level = ConsistencyLevel .ONE ,
3675+ fetch_size = self ._schema_meta_page_size )
36383676 local_query = QueryMessage (query = maybe_add_timeout_to_query (sel_local , self ._metadata_request_timeout ),
3639- consistency_level = ConsistencyLevel .ONE )
3640- (peers_success , peers_result ), (local_success , local_result ) = connection .wait_for_responses (
3641- peers_query , local_query , timeout = self ._timeout , fail_on_error = False )
3677+ consistency_level = ConsistencyLevel .ONE ,
3678+ fetch_size = self ._schema_meta_page_size )
3679+
3680+ local_success , local_result = _fetch_all_pages (connection , local_query , self ._timeout , fail_on_error = False )
36423681
36433682 if not local_success :
36443683 raise local_result
36453684
3685+ peers_success , peers_result = _fetch_all_pages (connection , peers_query , self ._timeout , fail_on_error = False )
3686+
36463687 if not peers_success :
36473688 # error with the peers v2 query, fallback to peers v1
36483689 self ._uses_peers_v2 = False
36493690 sel_peers = self ._get_peers_query (self .PeersQueryType .PEERS , connection )
36503691 peers_query = QueryMessage (query = maybe_add_timeout_to_query (sel_peers , self ._metadata_request_timeout ),
3651- consistency_level = ConsistencyLevel .ONE )
3652- peers_result = connection . wait_for_response (
3653- peers_query , timeout = self ._timeout )
3692+ consistency_level = ConsistencyLevel .ONE ,
3693+ fetch_size = self . _schema_meta_page_size )
3694+ peers_result = _fetch_all_pages ( connection , peers_query , self ._timeout )
36543695
36553696 shared_results = (peers_result , local_result )
36563697 self ._refresh_node_list_and_token_map (connection , preloaded_results = shared_results )
@@ -3793,11 +3834,17 @@ def _refresh_node_list_and_token_map(self, connection, preloaded_results=None,
37933834 log .debug ("[control connection] Refreshing node list and token map" )
37943835 sel_local = self ._SELECT_LOCAL
37953836 peers_query = QueryMessage (query = maybe_add_timeout_to_query (sel_peers , self ._metadata_request_timeout ),
3796- consistency_level = cl )
3837+ consistency_level = cl ,
3838+ fetch_size = self ._schema_meta_page_size )
37973839 local_query = QueryMessage (query = maybe_add_timeout_to_query (sel_local , self ._metadata_request_timeout ),
3798- consistency_level = cl )
3799- peers_result , local_result = connection .wait_for_responses (
3800- peers_query , local_query , timeout = self ._timeout )
3840+ consistency_level = cl ,
3841+ fetch_size = self ._schema_meta_page_size )
3842+
3843+ # Fetch all pages for both queries
3844+ # Note: system.local always has exactly 1 row, so it will never have additional pages
3845+ # system.peers might have multiple pages for very large clusters (>1000 nodes)
3846+ peers_result = _fetch_all_pages (connection , peers_query , self ._timeout )
3847+ local_result = _fetch_all_pages (connection , local_query , self ._timeout )
38013848
38023849 peers_result = dict_factory (peers_result .column_names , peers_result .parsed_rows )
38033850
@@ -3852,9 +3899,11 @@ def _refresh_node_list_and_token_map(self, connection, preloaded_results=None,
38523899 # in system.local. See CASSANDRA-9436.
38533900 local_rpc_address_query = QueryMessage (
38543901 query = maybe_add_timeout_to_query (self ._SELECT_LOCAL_NO_TOKENS_RPC_ADDRESS , self ._metadata_request_timeout ),
3855- consistency_level = ConsistencyLevel .ONE )
3856- success , local_rpc_address_result = connection .wait_for_response (
3857- local_rpc_address_query , timeout = self ._timeout , fail_on_error = False )
3902+ consistency_level = ConsistencyLevel .ONE ,
3903+ fetch_size = self ._schema_meta_page_size )
3904+ # Fetch all pages (system.local table always contains exactly one row, so this is effectively a no-op)
3905+ success , local_rpc_address_result = _fetch_all_pages (
3906+ connection , local_rpc_address_query , self ._timeout , fail_on_error = False )
38583907 if success :
38593908 row = dict_factory (
38603909 local_rpc_address_result .column_names ,
@@ -4088,13 +4137,19 @@ def wait_for_schema_agreement(self, connection=None, preloaded_results=None, wai
40884137
40894138 while elapsed < total_timeout :
40904139 peers_query = QueryMessage (query = maybe_add_timeout_to_query (select_peers_query , self ._metadata_request_timeout ),
4091- consistency_level = cl )
4140+ consistency_level = cl ,
4141+ fetch_size = self ._schema_meta_page_size )
40924142 local_query = QueryMessage (query = maybe_add_timeout_to_query (self ._SELECT_SCHEMA_LOCAL , self ._metadata_request_timeout ),
4093- consistency_level = cl )
4143+ consistency_level = cl ,
4144+ fetch_size = self ._schema_meta_page_size )
40944145 try :
40954146 timeout = min (self ._timeout , total_timeout - elapsed )
4096- peers_result , local_result = connection .wait_for_responses (
4097- peers_query , local_query , timeout = timeout )
4147+
4148+ # Fetch all pages if there are more results
4149+ # Note: system.local always has exactly 1 row, so it will never have additional pages
4150+ # system.peers might have multiple pages for very large clusters (>1000 nodes)
4151+ peers_result = _fetch_all_pages (connection , peers_query , timeout )
4152+ local_result = _fetch_all_pages (connection , local_query , timeout )
40984153 except OperationTimedOut as timeout :
40994154 log .debug ("[control connection] Timed out waiting for "
41004155 "response during schema agreement check: %s" , timeout )
0 commit comments