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
73
75
NeverRetryPolicy )
74
76
from cassandra .pool import (Host , _ReconnectionHandler , _HostReconnectionHandler ,
75
77
HostConnectionPool , HostConnection ,
76
- NoConnectionsAvailable )
78
+ NoConnectionsAvailable , Tablet )
77
79
from cassandra .query import (SimpleStatement , PreparedStatement , BoundStatement ,
78
80
BatchStatement , bind_params , QueryTrace , TraceUnavailable ,
79
81
named_tuple_factory , dict_factory , tuple_factory , FETCH_SIZE_UNSET ,
@@ -139,6 +141,9 @@ def _is_gevent_monkey_patched():
139
141
except ImportError :
140
142
from cassandra .io .asyncorereactor import AsyncoreConnection as DefaultConnection # NOQA
141
143
144
+ # If true tablet experimental feature is enabled, its interface and use may change
145
+ EXPERIMENTAL_TABLETS = os .environ .get ('EXPERIMENTAL_TABLETS' , '' ).lower () == 'true'
146
+
142
147
# Forces load of utf8 encoding module to avoid deadlock that occurs
143
148
# if code that is being imported tries to import the module in a seperate
144
149
# thread.
@@ -938,6 +943,12 @@ def default_retry_policy(self, policy):
938
943
establish connection pools. This can cause a rush of connections and queries if not mitigated with this factor.
939
944
"""
940
945
946
+ tablet_refresh_time = 60
947
+ """
948
+ The time at which the system.tablets table is polled periodically.
949
+ This field is experimental and may be changed / removed in the future.
950
+ """
951
+
941
952
prepare_on_all_hosts = True
942
953
"""
943
954
Specifies whether statements should be prepared on all hosts, or just one.
@@ -1124,6 +1135,7 @@ def __init__(self,
1124
1135
schema_metadata_page_size = 1000 ,
1125
1136
address_translator = None ,
1126
1137
status_event_refresh_window = 2 ,
1138
+ tablet_refresh_time = 60 , # This field is experimental and may be changed / removed in the future
1127
1139
prepare_on_all_hosts = True ,
1128
1140
reprepare_on_up = True ,
1129
1141
execution_profiles = None ,
@@ -1364,6 +1376,7 @@ def __init__(self,
1364
1376
self .schema_event_refresh_window = schema_event_refresh_window
1365
1377
self .topology_event_refresh_window = topology_event_refresh_window
1366
1378
self .status_event_refresh_window = status_event_refresh_window
1379
+ self .tablet_refresh_time = tablet_refresh_time
1367
1380
self .connect_timeout = connect_timeout
1368
1381
self .prepare_on_all_hosts = prepare_on_all_hosts
1369
1382
self .reprepare_on_up = reprepare_on_up
@@ -1416,7 +1429,7 @@ def __init__(self,
1416
1429
self .control_connection = ControlConnection (
1417
1430
self , self .control_connection_timeout ,
1418
1431
self .schema_event_refresh_window , self .topology_event_refresh_window ,
1419
- self .status_event_refresh_window ,
1432
+ self .status_event_refresh_window , self . tablet_refresh_time ,
1420
1433
schema_metadata_enabled , token_metadata_enabled ,
1421
1434
schema_meta_page_size = schema_metadata_page_size )
1422
1435
@@ -2381,6 +2394,15 @@ def add_prepared(self, query_id, prepared_statement):
2381
2394
with self ._prepared_statement_lock :
2382
2395
self ._prepared_statements [query_id ] = prepared_statement
2383
2396
2397
+ # Experimental, this interface and use may change
2398
+ def check_tablets_support (self , connection , timeout ):
2399
+ try :
2400
+ sel_tablets = "SELECT * FROM system.tablets"
2401
+ tablets_query = QueryMessage (query = sel_tablets , consistency_level = ConsistencyLevel .ONE )
2402
+ connection .wait_for_response (tablets_query , timeout = timeout )
2403
+ except :
2404
+ return False
2405
+ return True
2384
2406
2385
2407
class Session (object ):
2386
2408
"""
@@ -3513,6 +3535,8 @@ class ControlConnection(object):
3513
3535
_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
3536
_SELECT_SCHEMA_PEERS_V2 = "SELECT host_id, peer, peer_port, native_address, native_port, schema_version FROM system.peers_v2"
3515
3537
3538
+ _SELECT_TABLETS = "SELECT * FROM system.tablets"
3539
+
3516
3540
_MINIMUM_NATIVE_ADDRESS_DSE_VERSION = Version ("6.0.0" )
3517
3541
3518
3542
class PeersQueryType (object ):
@@ -3527,6 +3551,7 @@ class PeersQueryType(object):
3527
3551
_schema_event_refresh_window = None
3528
3552
_topology_event_refresh_window = None
3529
3553
_status_event_refresh_window = None
3554
+ _tablet_refresh_time = 60
3530
3555
3531
3556
_schema_meta_enabled = True
3532
3557
_token_meta_enabled = True
@@ -3541,6 +3566,7 @@ def __init__(self, cluster, timeout,
3541
3566
schema_event_refresh_window ,
3542
3567
topology_event_refresh_window ,
3543
3568
status_event_refresh_window ,
3569
+ tablet_refresh_time = 60 ,
3544
3570
schema_meta_enabled = True ,
3545
3571
token_meta_enabled = True ,
3546
3572
schema_meta_page_size = 1000 ):
@@ -3553,6 +3579,7 @@ def __init__(self, cluster, timeout,
3553
3579
self ._schema_event_refresh_window = schema_event_refresh_window
3554
3580
self ._topology_event_refresh_window = topology_event_refresh_window
3555
3581
self ._status_event_refresh_window = status_event_refresh_window
3582
+ self ._tablet_refresh_time = tablet_refresh_time
3556
3583
self ._schema_meta_enabled = schema_meta_enabled
3557
3584
self ._token_meta_enabled = token_meta_enabled
3558
3585
self ._schema_meta_page_size = schema_meta_page_size
@@ -3616,7 +3643,7 @@ def _reconnect_internal(self):
3616
3643
raise DriverException ("[control connection] Reconnection in progress during shutdown" )
3617
3644
3618
3645
raise NoHostAvailable ("Unable to connect to any servers" , errors )
3619
-
3646
+
3620
3647
def _try_connect (self , host ):
3621
3648
"""
3622
3649
Creates a new Connection, registers for pushed events, and refreshes
@@ -3656,8 +3683,12 @@ def _try_connect(self, host):
3656
3683
"TOPOLOGY_CHANGE" : partial (_watch_callback , self_weakref , '_handle_topology_change' ),
3657
3684
"STATUS_CHANGE" : partial (_watch_callback , self_weakref , '_handle_status_change' ),
3658
3685
"SCHEMA_CHANGE" : partial (_watch_callback , self_weakref , '_handle_schema_change' )
3686
+ # "TABLET_CHANGE": partial(_watch_callback, self_weakref, '_handle_tablet_change')
3659
3687
}, register_timeout = self ._timeout )
3660
3688
3689
+ if EXPERIMENTAL_TABLETS and self ._cluster .check_tablets_support (connection , self ._timeout ):
3690
+ self ._cluster .scheduler .schedule_unique (self ._tablet_refresh_time , self ._refresh_tablets , connection )
3691
+
3661
3692
sel_peers = self ._get_peers_query (self .PeersQueryType .PEERS , connection )
3662
3693
sel_local = self ._SELECT_LOCAL if self ._token_meta_enabled else self ._SELECT_LOCAL_NO_TOKENS
3663
3694
peers_query = QueryMessage (query = sel_peers , consistency_level = ConsistencyLevel .ONE )
@@ -3679,6 +3710,11 @@ def _try_connect(self, host):
3679
3710
shared_results = (peers_result , local_result )
3680
3711
self ._refresh_node_list_and_token_map (connection , preloaded_results = shared_results )
3681
3712
self ._refresh_schema (connection , preloaded_results = shared_results , schema_agreement_wait = - 1 )
3713
+
3714
+ if EXPERIMENTAL_TABLETS and self ._cluster .check_tablets_support (connection , self ._timeout ):
3715
+ tablets_query = QueryMessage (query = self ._SELECT_TABLETS , consistency_level = ConsistencyLevel .ONE )
3716
+ tablets_result = connection .wait_for_response (tablets_query , timeout = self ._timeout )
3717
+ self ._refresh_tablets (connection , preloaded_result = tablets_result )
3682
3718
except Exception :
3683
3719
connection .close ()
3684
3720
raise
@@ -3964,12 +4000,53 @@ 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
+ # Experimental, this interface and use may change
4004
+ def _refresh_tablets (self , connection , preloaded_result = None ,
4005
+ force_token_rebuild = False ):
4006
+ if preloaded_result :
4007
+ log .debug ("[control connection] Refreshing tablet list using preloaded result" )
4008
+ tablets_result = preloaded_result
4009
+ else :
4010
+ sel_tablets = self ._SELECT_TABLETS
4011
+ tablets_query = QueryMessage (query = sel_tablets , consistency_level = ConsistencyLevel .ONE )
4012
+ tablets_result = connection .wait_for_response (
4013
+ tablets_query , timeout = self ._timeout )
4014
+ tablets_result = dict_factory (tablets_result .column_names , tablets_result .parsed_rows )
4015
+
4016
+ tablets = []
4017
+ for row in tablets_result :
4018
+ if not self ._is_valid_tablet (row ):
4019
+ log .warning (
4020
+ "Found an invalid row for tablet (%s). Ignoring tablet." %
4021
+ _NodeInfo .get_broadcast_rpc_address (row ))
4022
+ continue
4023
+
4024
+ tablet = Tablet ()
4025
+ tablet .keyspace_name = row .get ("keyspace_name" )
4026
+ tablet .table_name = row .get ("table_name" )
4027
+ tablet .table_id = row .get ("table_id" )
4028
+ tablet .tablet_count = row .get ("tablet_count" )
4029
+ tablet .last_token = row .get ("last_token" )
4030
+ tablet .new_replicas = row .get ("new_replicas" )
4031
+ tablet .replicas = row .get ("replicas" )
4032
+ tablet .stage = row .get ("stage" )
4033
+
4034
+ tablets .append (tablet )
4035
+
4036
+ self ._cluster .metadata .set_tablets (tablets )
4037
+ return ""
4038
+
3967
4039
@staticmethod
3968
4040
def _is_valid_peer (row ):
3969
4041
return bool (_NodeInfo .get_broadcast_rpc_address (row ) and row .get ("host_id" ) and
3970
4042
row .get ("data_center" ) and row .get ("rack" ) and
3971
4043
('tokens' not in row or row .get ('tokens' )))
3972
4044
4045
+ # Experimental, this interface and use may change
4046
+ @staticmethod
4047
+ def _is_valid_tablet (row ):
4048
+ return bool (row .get ("replicas" ) is not None and len (row .get ("replicas" )) != 0 and row .get ("table_name" ) is not None )
4049
+
3973
4050
def _update_location_info (self , host , datacenter , rack ):
3974
4051
if host .datacenter == datacenter and host .rack == rack :
3975
4052
return False
@@ -4571,7 +4648,10 @@ def _query(self, host, message=None, cb=None):
4571
4648
connection = None
4572
4649
try :
4573
4650
# 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 )
4651
+ if self .query :
4652
+ connection , request_id = pool .borrow_connection (timeout = 2.0 , routing_key = self .query .routing_key , keyspace = self .query .keyspace , table = self .query .table )
4653
+ else :
4654
+ connection , request_id = pool .borrow_connection (timeout = 2.0 )
4575
4655
self ._connection = connection
4576
4656
result_meta = self .prepared_statement .result_metadata if self .prepared_statement else []
4577
4657
0 commit comments