27
27
from itertools import groupby , count , chain
28
28
import json
29
29
import logging
30
+ import os
31
+ import threading
30
32
from warnings import warn
31
33
from random import random
32
34
import re
79
81
named_tuple_factory , dict_factory , tuple_factory , FETCH_SIZE_UNSET ,
80
82
HostTargetingStatement )
81
83
from cassandra .marshal import int64_pack
84
+ from cassandra .tablets import Tablet , Tablets
82
85
from cassandra .timestamps import MonotonicTimestampGenerator
83
86
from cassandra .compat import Mapping
84
87
from cassandra .util import _resolve_contact_points_to_string_map , Version
@@ -938,6 +941,17 @@ def default_retry_policy(self, policy):
938
941
establish connection pools. This can cause a rush of connections and queries if not mitigated with this factor.
939
942
"""
940
943
944
+ experimental_tablet_refresh_time = 60
945
+ """
946
+ The time at which the system.tablets table is polled periodically.
947
+ This field is experimental and may be changed / removed in the future.
948
+ """
949
+
950
+ experimental_tablet_feature_enabled = False
951
+ """
952
+ If true tablet experimental feature is enabled, its interface and use may change.
953
+ """
954
+
941
955
prepare_on_all_hosts = True
942
956
"""
943
957
Specifies whether statements should be prepared on all hosts, or just one.
@@ -1124,6 +1138,8 @@ def __init__(self,
1124
1138
schema_metadata_page_size = 1000 ,
1125
1139
address_translator = None ,
1126
1140
status_event_refresh_window = 2 ,
1141
+ experimental_tablet_refresh_time = 60 , # This field is experimental and may be changed / removed in the future
1142
+ experimental_tablet_feature_enabled = False , # This field is experimental and may be changed / removed in the future
1127
1143
prepare_on_all_hosts = True ,
1128
1144
reprepare_on_up = True ,
1129
1145
execution_profiles = None ,
@@ -1364,6 +1380,8 @@ def __init__(self,
1364
1380
self .schema_event_refresh_window = schema_event_refresh_window
1365
1381
self .topology_event_refresh_window = topology_event_refresh_window
1366
1382
self .status_event_refresh_window = status_event_refresh_window
1383
+ self .experimental_tablet_refresh_time = experimental_tablet_refresh_time
1384
+ self .experimental_tablet_feature_enabled = experimental_tablet_feature_enabled
1367
1385
self .connect_timeout = connect_timeout
1368
1386
self .prepare_on_all_hosts = prepare_on_all_hosts
1369
1387
self .reprepare_on_up = reprepare_on_up
@@ -1416,7 +1434,7 @@ def __init__(self,
1416
1434
self .control_connection = ControlConnection (
1417
1435
self , self .control_connection_timeout ,
1418
1436
self .schema_event_refresh_window , self .topology_event_refresh_window ,
1419
- self .status_event_refresh_window ,
1437
+ self .status_event_refresh_window , self . experimental_tablet_refresh_time ,
1420
1438
schema_metadata_enabled , token_metadata_enabled ,
1421
1439
schema_meta_page_size = schema_metadata_page_size )
1422
1440
@@ -2381,6 +2399,19 @@ def add_prepared(self, query_id, prepared_statement):
2381
2399
with self ._prepared_statement_lock :
2382
2400
self ._prepared_statements [query_id ] = prepared_statement
2383
2401
2402
+ # Experimental, this interface and use may change
2403
+ def check_tablets_support (self , connection , timeout ):
2404
+ try :
2405
+ sel_tablets = ControlConnection ._SELECT_TABLETS
2406
+ tablets_query = QueryMessage (query = sel_tablets , consistency_level = ConsistencyLevel .ONE )
2407
+ connection .wait_for_response (tablets_query , timeout = timeout )
2408
+ except :
2409
+ return False
2410
+ return True
2411
+
2412
+ # Experimental, this interface and use may change
2413
+ def check_tablets_enabled (self ):
2414
+ return self .experimental_tablet_feature_enabled
2384
2415
2385
2416
class Session (object ):
2386
2417
"""
@@ -3513,6 +3544,8 @@ class ControlConnection(object):
3513
3544
_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
3545
_SELECT_SCHEMA_PEERS_V2 = "SELECT host_id, peer, peer_port, native_address, native_port, schema_version FROM system.peers_v2"
3515
3546
3547
+ _SELECT_TABLETS = "SELECT * FROM system.tablets"
3548
+
3516
3549
_MINIMUM_NATIVE_ADDRESS_DSE_VERSION = Version ("6.0.0" )
3517
3550
3518
3551
class PeersQueryType (object ):
@@ -3527,6 +3560,7 @@ class PeersQueryType(object):
3527
3560
_schema_event_refresh_window = None
3528
3561
_topology_event_refresh_window = None
3529
3562
_status_event_refresh_window = None
3563
+ _tablet_refresh_time = 60
3530
3564
3531
3565
_schema_meta_enabled = True
3532
3566
_token_meta_enabled = True
@@ -3541,6 +3575,7 @@ def __init__(self, cluster, timeout,
3541
3575
schema_event_refresh_window ,
3542
3576
topology_event_refresh_window ,
3543
3577
status_event_refresh_window ,
3578
+ tablet_refresh_time = 60 ,
3544
3579
schema_meta_enabled = True ,
3545
3580
token_meta_enabled = True ,
3546
3581
schema_meta_page_size = 1000 ):
@@ -3553,6 +3588,7 @@ def __init__(self, cluster, timeout,
3553
3588
self ._schema_event_refresh_window = schema_event_refresh_window
3554
3589
self ._topology_event_refresh_window = topology_event_refresh_window
3555
3590
self ._status_event_refresh_window = status_event_refresh_window
3591
+ self ._tablet_refresh_time = tablet_refresh_time
3556
3592
self ._schema_meta_enabled = schema_meta_enabled
3557
3593
self ._token_meta_enabled = token_meta_enabled
3558
3594
self ._schema_meta_page_size = schema_meta_page_size
@@ -3658,6 +3694,9 @@ def _try_connect(self, host):
3658
3694
"SCHEMA_CHANGE" : partial (_watch_callback , self_weakref , '_handle_schema_change' )
3659
3695
}, register_timeout = self ._timeout )
3660
3696
3697
+ if self ._cluster .check_tablets_enabled () and self ._cluster .check_tablets_support (connection , self ._timeout ):
3698
+ self ._cluster .scheduler .schedule_unique (self ._tablet_refresh_time , self ._refresh_tablets , connection )
3699
+
3661
3700
sel_peers = self ._get_peers_query (self .PeersQueryType .PEERS , connection )
3662
3701
sel_local = self ._SELECT_LOCAL if self ._token_meta_enabled else self ._SELECT_LOCAL_NO_TOKENS
3663
3702
peers_query = QueryMessage (query = sel_peers , consistency_level = ConsistencyLevel .ONE )
@@ -3679,6 +3718,11 @@ def _try_connect(self, host):
3679
3718
shared_results = (peers_result , local_result )
3680
3719
self ._refresh_node_list_and_token_map (connection , preloaded_results = shared_results )
3681
3720
self ._refresh_schema (connection , preloaded_results = shared_results , schema_agreement_wait = - 1 )
3721
+
3722
+ if self ._cluster .check_tablets_enabled () and self ._cluster .check_tablets_support (connection , self ._timeout ):
3723
+ tablets_query = QueryMessage (query = self ._SELECT_TABLETS , consistency_level = ConsistencyLevel .ONE )
3724
+ tablets_result = connection .wait_for_response (tablets_query , timeout = self ._timeout )
3725
+ self ._refresh_tablets (connection , preloaded_result = tablets_result )
3682
3726
except Exception :
3683
3727
connection .close ()
3684
3728
raise
@@ -3964,12 +4008,59 @@ def _refresh_node_list_and_token_map(self, connection, preloaded_results=None,
3964
4008
log .debug ("[control connection] Rebuilding token map due to topology changes" )
3965
4009
self ._cluster .metadata .rebuild_token_map (partitioner , token_map )
3966
4010
4011
+ # Experimental, this interface and use may change
4012
+ def _refresh_tablets (self , connection , preloaded_result = None ,
4013
+ force_token_rebuild = False ):
4014
+ if preloaded_result :
4015
+ log .debug ("[control connection] Refreshing tablet list using preloaded result" )
4016
+ tablets_result = preloaded_result
4017
+ else :
4018
+ sel_tablets = self ._SELECT_TABLETS
4019
+ tablets_query = QueryMessage (query = sel_tablets , consistency_level = ConsistencyLevel .ONE )
4020
+ tablets_result = connection .wait_for_response (
4021
+ tablets_query , timeout = self ._timeout )
4022
+ tablets_result = dict_factory (tablets_result .column_names , tablets_result .parsed_rows )
4023
+
4024
+ tablets = {}
4025
+ for row in tablets_result :
4026
+ if not self ._is_valid_tablet (row ):
4027
+ log .warning (
4028
+ "Found an invalid row for tablet (%s). Ignoring tablet." %
4029
+ _NodeInfo .get_broadcast_rpc_address (row ))
4030
+ continue
4031
+
4032
+ keyspace_name = row .get ("keyspace_name" )
4033
+ table_name = row .get ("table_name" )
4034
+
4035
+ tablet = Tablet ()
4036
+ tablet .table_id = row .get ("table_id" )
4037
+ tablet .tablet_count = row .get ("tablet_count" )
4038
+ tablet .last_token = row .get ("last_token" )
4039
+ tablet .new_replicas = row .get ("new_replicas" )
4040
+ tablet .replicas = row .get ("replicas" )
4041
+ tablet .stage = row .get ("stage" )
4042
+
4043
+ prev = tablets .get ((keyspace_name , table_name ), [])
4044
+ prev .append (tablet )
4045
+ tablets [(keyspace_name , table_name )] = prev
4046
+
4047
+ if self ._cluster .metadata ._tablets is None :
4048
+ self ._cluster .metadata ._tablets = Tablets ()
4049
+ self ._cluster .metadata ._tablets .set_tablets (tablets )
4050
+ self ._cluster .scheduler .schedule_unique (self ._tablet_refresh_time , self ._refresh_tablets , connection )
4051
+ return ""
4052
+
3967
4053
@staticmethod
3968
4054
def _is_valid_peer (row ):
3969
4055
return bool (_NodeInfo .get_broadcast_rpc_address (row ) and row .get ("host_id" ) and
3970
4056
row .get ("data_center" ) and row .get ("rack" ) and
3971
4057
('tokens' not in row or row .get ('tokens' )))
3972
4058
4059
+ # Experimental, this interface and use may change
4060
+ @staticmethod
4061
+ def _is_valid_tablet (row ):
4062
+ return bool (row .get ("replicas" ) is not None and len (row .get ("replicas" )) != 0 and row .get ("table_name" ) is not None )
4063
+
3973
4064
def _update_location_info (self , host , datacenter , rack ):
3974
4065
if host .datacenter == datacenter and host .rack == rack :
3975
4066
return False
@@ -4571,7 +4662,10 @@ def _query(self, host, message=None, cb=None):
4571
4662
connection = None
4572
4663
try :
4573
4664
# 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 )
4665
+ if self .query :
4666
+ connection , request_id = pool .borrow_connection (timeout = 2.0 , routing_key = self .query .routing_key , keyspace = self .query .keyspace , table = self .query .table )
4667
+ else :
4668
+ connection , request_id = pool .borrow_connection (timeout = 2.0 )
4575
4669
self ._connection = connection
4576
4670
result_meta = self .prepared_statement .result_metadata if self .prepared_statement else []
4577
4671
0 commit comments