41
41
import weakref
42
42
from weakref import WeakValueDictionary
43
43
44
- from cassandra import (ConsistencyLevel , AuthenticationFailed ,
44
+ from cassandra import (ConsistencyLevel , AuthenticationFailed , InvalidRequest ,
45
45
OperationTimedOut , UnsupportedOperation ,
46
46
SchemaTargetType , DriverException , ProtocolVersion ,
47
47
UnresolvableContactPoints )
79
79
named_tuple_factory , dict_factory , tuple_factory , FETCH_SIZE_UNSET ,
80
80
HostTargetingStatement )
81
81
from cassandra .marshal import int64_pack
82
+ from cassandra .tablets import Tablet , Tablets
82
83
from cassandra .timestamps import MonotonicTimestampGenerator
83
84
from cassandra .compat import Mapping
84
85
from cassandra .util import _resolve_contact_points_to_string_map , Version
@@ -938,6 +939,17 @@ def default_retry_policy(self, policy):
938
939
establish connection pools. This can cause a rush of connections and queries if not mitigated with this factor.
939
940
"""
940
941
942
+ experimental_tablet_refresh_time = 60
943
+ """
944
+ The time at which the system.tablets table is polled periodically.
945
+ This field is experimental and may be changed / removed in the future.
946
+ """
947
+
948
+ experimental_tablet_feature_enabled = False
949
+ """
950
+ If true tablet experimental feature is enabled, its interface and use may change.
951
+ """
952
+
941
953
prepare_on_all_hosts = True
942
954
"""
943
955
Specifies whether statements should be prepared on all hosts, or just one.
@@ -1124,6 +1136,8 @@ def __init__(self,
1124
1136
schema_metadata_page_size = 1000 ,
1125
1137
address_translator = None ,
1126
1138
status_event_refresh_window = 2 ,
1139
+ experimental_tablet_refresh_time = 60 , # This field is experimental and may be changed / removed in the future
1140
+ experimental_tablet_feature_enabled = False , # This field is experimental and may be changed / removed in the future
1127
1141
prepare_on_all_hosts = True ,
1128
1142
reprepare_on_up = True ,
1129
1143
execution_profiles = None ,
@@ -1364,6 +1378,8 @@ def __init__(self,
1364
1378
self .schema_event_refresh_window = schema_event_refresh_window
1365
1379
self .topology_event_refresh_window = topology_event_refresh_window
1366
1380
self .status_event_refresh_window = status_event_refresh_window
1381
+ self .experimental_tablet_refresh_time = experimental_tablet_refresh_time
1382
+ self .experimental_tablet_feature_enabled = experimental_tablet_feature_enabled
1367
1383
self .connect_timeout = connect_timeout
1368
1384
self .prepare_on_all_hosts = prepare_on_all_hosts
1369
1385
self .reprepare_on_up = reprepare_on_up
@@ -1416,7 +1432,7 @@ def __init__(self,
1416
1432
self .control_connection = ControlConnection (
1417
1433
self , self .control_connection_timeout ,
1418
1434
self .schema_event_refresh_window , self .topology_event_refresh_window ,
1419
- self .status_event_refresh_window ,
1435
+ self .status_event_refresh_window , self . experimental_tablet_refresh_time ,
1420
1436
schema_metadata_enabled , token_metadata_enabled ,
1421
1437
schema_meta_page_size = schema_metadata_page_size )
1422
1438
@@ -2381,6 +2397,9 @@ def add_prepared(self, query_id, prepared_statement):
2381
2397
with self ._prepared_statement_lock :
2382
2398
self ._prepared_statements [query_id ] = prepared_statement
2383
2399
2400
+ # Experimental, this interface and use may change
2401
+ def check_tablets_enabled (self ):
2402
+ return self .experimental_tablet_feature_enabled
2384
2403
2385
2404
class Session (object ):
2386
2405
"""
@@ -3513,6 +3532,8 @@ class ControlConnection(object):
3513
3532
_SELECT_PEERS_NO_TOKENS_V2 = "SELECT host_id, peer, peer_port, data_center, rack, native_address, native_port, release_version, schema_version FROM system.peers_v2"
3514
3533
_SELECT_SCHEMA_PEERS_V2 = "SELECT host_id, peer, peer_port, native_address, native_port, schema_version FROM system.peers_v2"
3515
3534
3535
+ _SELECT_TABLETS = "SELECT * FROM system.tablets"
3536
+
3516
3537
_MINIMUM_NATIVE_ADDRESS_DSE_VERSION = Version ("6.0.0" )
3517
3538
3518
3539
class PeersQueryType (object ):
@@ -3527,6 +3548,7 @@ class PeersQueryType(object):
3527
3548
_schema_event_refresh_window = None
3528
3549
_topology_event_refresh_window = None
3529
3550
_status_event_refresh_window = None
3551
+ _tablet_refresh_time = 60
3530
3552
3531
3553
_schema_meta_enabled = True
3532
3554
_token_meta_enabled = True
@@ -3541,6 +3563,7 @@ def __init__(self, cluster, timeout,
3541
3563
schema_event_refresh_window ,
3542
3564
topology_event_refresh_window ,
3543
3565
status_event_refresh_window ,
3566
+ tablet_refresh_time = 60 ,
3544
3567
schema_meta_enabled = True ,
3545
3568
token_meta_enabled = True ,
3546
3569
schema_meta_page_size = 1000 ):
@@ -3553,6 +3576,7 @@ def __init__(self, cluster, timeout,
3553
3576
self ._schema_event_refresh_window = schema_event_refresh_window
3554
3577
self ._topology_event_refresh_window = topology_event_refresh_window
3555
3578
self ._status_event_refresh_window = status_event_refresh_window
3579
+ self ._tablet_refresh_time = tablet_refresh_time
3556
3580
self ._schema_meta_enabled = schema_meta_enabled
3557
3581
self ._token_meta_enabled = token_meta_enabled
3558
3582
self ._schema_meta_page_size = schema_meta_page_size
@@ -3658,6 +3682,18 @@ def _try_connect(self, host):
3658
3682
"SCHEMA_CHANGE" : partial (_watch_callback , self_weakref , '_handle_schema_change' )
3659
3683
}, register_timeout = self ._timeout )
3660
3684
3685
+ if self ._cluster .check_tablets_enabled ():
3686
+ try :
3687
+ self ._refresh_tablets (connection )
3688
+ except InvalidRequest :
3689
+ self ._cluster .experimental_tablet_feature_enabled = False
3690
+ # The information about tablets is passed to load_balancing_policy during `populate` that happens
3691
+ # before this check, so if there is an error during `_refresh_tablets` we need to change the field
3692
+ # in the cluster, but also propagate this information to the load_balancing_policy using `populate`
3693
+ self ._cluster .load_balancing_policy .populate (self ._cluster , self ._cluster .metadata .all_hosts ())
3694
+ else :
3695
+ self ._cluster .scheduler .schedule_unique (self ._tablet_refresh_time , self ._refresh_tablets_periodically , connection )
3696
+
3661
3697
sel_peers = self ._get_peers_query (self .PeersQueryType .PEERS , connection )
3662
3698
sel_local = self ._SELECT_LOCAL if self ._token_meta_enabled else self ._SELECT_LOCAL_NO_TOKENS
3663
3699
peers_query = QueryMessage (query = sel_peers , consistency_level = ConsistencyLevel .ONE )
@@ -3964,6 +4000,40 @@ def _refresh_node_list_and_token_map(self, connection, preloaded_results=None,
3964
4000
log .debug ("[control connection] Rebuilding token map due to topology changes" )
3965
4001
self ._cluster .metadata .rebuild_token_map (partitioner , token_map )
3966
4002
4003
+ def _refresh_tablets_periodically (self , connection ):
4004
+ try :
4005
+ self ._refresh_tablets (connection )
4006
+ except ReferenceError :
4007
+ return
4008
+ except Exception :
4009
+ log .debug ("[control connection] Error refreshing tablets" , exc_info = True )
4010
+ self ._signal_error ()
4011
+ self ._cluster .scheduler .schedule_unique (self ._tablet_refresh_time , self ._refresh_tablets , connection )
4012
+
4013
+ # Experimental, this interface and use may change
4014
+ def _refresh_tablets (self , connection ):
4015
+ sel_tablets = self ._SELECT_TABLETS
4016
+ tablets_query = QueryMessage (query = sel_tablets , consistency_level = ConsistencyLevel .ONE )
4017
+ tablets_result = connection .wait_for_response (
4018
+ tablets_query , timeout = self ._timeout )
4019
+ tablets_result = dict_factory (tablets_result .column_names , tablets_result .parsed_rows )
4020
+
4021
+ tablets = {}
4022
+ for row in tablets_result :
4023
+ tablet = Tablet .from_row (row )
4024
+ if tablet is None :
4025
+ log .warning (
4026
+ "Found an invalid row for tablet (%s). Ignoring tablet." %
4027
+ _NodeInfo .get_broadcast_rpc_address (row ))
4028
+ continue
4029
+
4030
+ keyspace_name = row .get ("keyspace_name" )
4031
+ table_name = row .get ("table_name" )
4032
+
4033
+ tablets .setdefault ((keyspace_name , table_name ), []).append (tablet )
4034
+
4035
+ self ._cluster .metadata ._tablets = Tablets (tablets )
4036
+
3967
4037
@staticmethod
3968
4038
def _is_valid_peer (row ):
3969
4039
return bool (_NodeInfo .get_broadcast_rpc_address (row ) and row .get ("host_id" ) and
@@ -4571,7 +4641,10 @@ def _query(self, host, message=None, cb=None):
4571
4641
connection = None
4572
4642
try :
4573
4643
# TODO get connectTimeout from cluster settings
4574
- connection , request_id = pool .borrow_connection (timeout = 2.0 , routing_key = self .query .routing_key if self .query else None )
4644
+ if self .query :
4645
+ connection , request_id = pool .borrow_connection (timeout = 2.0 , routing_key = self .query .routing_key , keyspace = self .query .keyspace , table = self .query .table )
4646
+ else :
4647
+ connection , request_id = pool .borrow_connection (timeout = 2.0 )
4575
4648
self ._connection = connection
4576
4649
result_meta = self .prepared_statement .result_metadata if self .prepared_statement else []
4577
4650
0 commit comments