Skip to content

Commit 4beebd5

Browse files
committed
Use metadata_request_timeout for all driver queries
1 parent e4a000f commit 4beebd5

File tree

4 files changed

+164
-69
lines changed

4 files changed

+164
-69
lines changed

cassandra/cluster.py

Lines changed: 18 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -83,7 +83,7 @@
8383
from cassandra.marshal import int64_pack
8484
from cassandra.tablets import Tablet, Tablets
8585
from cassandra.timestamps import MonotonicTimestampGenerator
86-
from cassandra.util import _resolve_contact_points_to_string_map, Version
86+
from cassandra.util import _resolve_contact_points_to_string_map, Version, maybe_add_timeout_to_query
8787

8888
from cassandra.datastax.insights.reporter import MonitorReporter
8989
from cassandra.datastax.insights.util import version_supports_insights
@@ -3725,8 +3725,10 @@ def _try_connect(self, host):
37253725

37263726
sel_peers = self._get_peers_query(self.PeersQueryType.PEERS, connection)
37273727
sel_local = self._SELECT_LOCAL if self._token_meta_enabled else self._SELECT_LOCAL_NO_TOKENS
3728-
peers_query = QueryMessage(query=sel_peers, consistency_level=ConsistencyLevel.ONE)
3729-
local_query = QueryMessage(query=sel_local, consistency_level=ConsistencyLevel.ONE)
3728+
peers_query = QueryMessage(query=maybe_add_timeout_to_query(sel_peers, self._metadata_request_timeout),
3729+
consistency_level=ConsistencyLevel.ONE)
3730+
local_query = QueryMessage(query=maybe_add_timeout_to_query(sel_local, self._metadata_request_timeout),
3731+
consistency_level=ConsistencyLevel.ONE)
37303732
(peers_success, peers_result), (local_success, local_result) = connection.wait_for_responses(
37313733
peers_query, local_query, timeout=self._timeout, fail_on_error=False)
37323734

@@ -3737,7 +3739,8 @@ def _try_connect(self, host):
37373739
# error with the peers v2 query, fallback to peers v1
37383740
self._uses_peers_v2 = False
37393741
sel_peers = self._get_peers_query(self.PeersQueryType.PEERS, connection)
3740-
peers_query = QueryMessage(query=sel_peers, consistency_level=ConsistencyLevel.ONE)
3742+
peers_query = QueryMessage(query=maybe_add_timeout_to_query(sel_peers, self._metadata_request_timeout),
3743+
consistency_level=ConsistencyLevel.ONE)
37413744
peers_result = connection.wait_for_response(
37423745
peers_query, timeout=self._timeout)
37433746

@@ -3881,8 +3884,10 @@ def _refresh_node_list_and_token_map(self, connection, preloaded_results=None,
38813884
else:
38823885
log.debug("[control connection] Refreshing node list and token map")
38833886
sel_local = self._SELECT_LOCAL
3884-
peers_query = QueryMessage(query=sel_peers, consistency_level=cl)
3885-
local_query = QueryMessage(query=sel_local, consistency_level=cl)
3887+
peers_query = QueryMessage(query=maybe_add_timeout_to_query(sel_peers, self._metadata_request_timeout),
3888+
consistency_level=cl)
3889+
local_query = QueryMessage(query=maybe_add_timeout_to_query(sel_local, self._metadata_request_timeout),
3890+
consistency_level=cl)
38863891
peers_result, local_result = connection.wait_for_responses(
38873892
peers_query, local_query, timeout=self._timeout)
38883893

@@ -3937,8 +3942,9 @@ def _refresh_node_list_and_token_map(self, connection, preloaded_results=None,
39373942
# local rpc_address has not been queried yet, try to fetch it
39383943
# separately, which might fail because C* < 2.1.6 doesn't have rpc_address
39393944
# in system.local. See CASSANDRA-9436.
3940-
local_rpc_address_query = QueryMessage(query=self._SELECT_LOCAL_NO_TOKENS_RPC_ADDRESS,
3941-
consistency_level=ConsistencyLevel.ONE)
3945+
local_rpc_address_query = QueryMessage(
3946+
query=maybe_add_timeout_to_query(self._SELECT_LOCAL_NO_TOKENS_RPC_ADDRESS, self._metadata_request_timeout),
3947+
consistency_level=ConsistencyLevel.ONE)
39423948
success, local_rpc_address_result = connection.wait_for_response(
39433949
local_rpc_address_query, timeout=self._timeout, fail_on_error=False)
39443950
if success:
@@ -4173,8 +4179,10 @@ def wait_for_schema_agreement(self, connection=None, preloaded_results=None, wai
41734179
select_peers_query = self._get_peers_query(self.PeersQueryType.PEERS_SCHEMA, connection)
41744180

41754181
while elapsed < total_timeout:
4176-
peers_query = QueryMessage(query=select_peers_query, consistency_level=cl)
4177-
local_query = QueryMessage(query=self._SELECT_SCHEMA_LOCAL, consistency_level=cl)
4182+
peers_query = QueryMessage(query=maybe_add_timeout_to_query(select_peers_query, self._metadata_request_timeout),
4183+
consistency_level=cl)
4184+
local_query = QueryMessage(query=maybe_add_timeout_to_query(self._SELECT_SCHEMA_LOCAL, self._metadata_request_timeout),
4185+
consistency_level=cl)
41784186
try:
41794187
timeout = min(self._timeout, total_timeout - elapsed)
41804188
peers_result, local_result = connection.wait_for_responses(

0 commit comments

Comments
 (0)