Skip to content

Commit dfccfff

Browse files
authored
Merge pull request #249 from sylwiaszunejko/introduce_tablets
Introduce support for tablets
2 parents a27751b + eaa9eb1 commit dfccfff

File tree

16 files changed

+486
-25
lines changed

16 files changed

+486
-25
lines changed

.github/workflows/integration-tests.yml

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -32,4 +32,11 @@ jobs:
3232
- name: Test with pytest
3333
run: |
3434
export EVENT_LOOP_MANAGER=${{ matrix.event_loop_manager }}
35+
export SCYLLA_VERSION='release:5.1'
3536
./ci/run_integration_test.sh tests/integration/standard/ tests/integration/cqlengine/
37+
38+
- name: Test tablets
39+
run: |
40+
export EVENT_LOOP_MANAGER=${{ matrix.event_loop_manager }}
41+
export SCYLLA_VERSION='unstable/master:2024-01-03T08:06:57Z'
42+
./ci/run_integration_test.sh tests/integration/experiments/

README.rst

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@ Features
2626
* `Concurrent execution utilities <http://python-driver.docs.scylladb.com/stable/api/cassandra/concurrent.html>`_
2727
* `Object mapper <http://python-driver.docs.scylladb.com/stable/object-mapper.html>`_
2828
* `Shard awareness <http://python-driver.docs.scylladb.com/stable/scylla-specific.html#shard-awareness>`_
29+
* `Tablet awareness <http://python-driver.docs.scylladb.com/stable/scylla-specific.html#tablet-awareness>`_
2930

3031
Installation
3132
------------

cassandra/cluster.py

Lines changed: 31 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -41,7 +41,7 @@
4141
import weakref
4242
from weakref import WeakValueDictionary
4343

44-
from cassandra import (ConsistencyLevel, AuthenticationFailed,
44+
from cassandra import (ConsistencyLevel, AuthenticationFailed, InvalidRequest,
4545
OperationTimedOut, UnsupportedOperation,
4646
SchemaTargetType, DriverException, ProtocolVersion,
4747
UnresolvableContactPoints)
@@ -51,6 +51,7 @@
5151
EndPoint, DefaultEndPoint, DefaultEndPointFactory,
5252
ContinuousPagingState, SniEndPointFactory, ConnectionBusy)
5353
from cassandra.cqltypes import UserType
54+
import cassandra.cqltypes as types
5455
from cassandra.encoder import Encoder
5556
from cassandra.protocol import (QueryMessage, ResultMessage,
5657
ErrorMessage, ReadTimeoutErrorMessage,
@@ -79,6 +80,7 @@
7980
named_tuple_factory, dict_factory, tuple_factory, FETCH_SIZE_UNSET,
8081
HostTargetingStatement)
8182
from cassandra.marshal import int64_pack
83+
from cassandra.tablets import Tablet, Tablets
8284
from cassandra.timestamps import MonotonicTimestampGenerator
8385
from cassandra.compat import Mapping
8486
from cassandra.util import _resolve_contact_points_to_string_map, Version
@@ -1784,6 +1786,14 @@ def connect(self, keyspace=None, wait_for_all_pools=False):
17841786
self.shutdown()
17851787
raise
17861788

1789+
# Update the information about tablet support after connection handshake.
1790+
self.load_balancing_policy._tablets_routing_v1 = self.control_connection._tablets_routing_v1
1791+
child_policy = self.load_balancing_policy.child_policy if hasattr(self.load_balancing_policy, 'child_policy') else None
1792+
while child_policy is not None:
1793+
if hasattr(child_policy, '_tablet_routing_v1'):
1794+
child_policy._tablet_routing_v1 = self.control_connection._tablets_routing_v1
1795+
child_policy = child_policy.child_policy if hasattr(child_policy, 'child_policy') else None
1796+
17871797
self.profile_manager.check_supported() # todo: rename this method
17881798

17891799
if self.idle_heartbeat_interval:
@@ -2398,7 +2408,6 @@ def add_prepared(self, query_id, prepared_statement):
23982408
with self._prepared_statement_lock:
23992409
self._prepared_statements[query_id] = prepared_statement
24002410

2401-
24022411
class Session(object):
24032412
"""
24042413
A collection of connection pools for each host in the cluster.
@@ -3550,6 +3559,7 @@ class PeersQueryType(object):
35503559
_schema_meta_page_size = 1000
35513560

35523561
_uses_peers_v2 = True
3562+
_tablets_routing_v1 = False
35533563

35543564
# for testing purposes
35553565
_time = time
@@ -3683,6 +3693,8 @@ def _try_connect(self, host):
36833693
# If sharding information is available, it's a ScyllaDB cluster, so do not use peers_v2 table.
36843694
if connection.features.sharding_info is not None:
36853695
self._uses_peers_v2 = False
3696+
3697+
self._tablets_routing_v1 = connection.features.tablets_routing_v1
36863698

36873699
# use weak references in both directions
36883700
# _clear_watcher will be called when this ControlConnection is about to be finalized
@@ -4609,7 +4621,10 @@ def _query(self, host, message=None, cb=None):
46094621
connection = None
46104622
try:
46114623
# TODO get connectTimeout from cluster settings
4612-
connection, request_id = pool.borrow_connection(timeout=2.0, routing_key=self.query.routing_key if self.query else None)
4624+
if self.query:
4625+
connection, request_id = pool.borrow_connection(timeout=2.0, routing_key=self.query.routing_key, keyspace=self.query.keyspace, table=self.query.table)
4626+
else:
4627+
connection, request_id = pool.borrow_connection(timeout=2.0)
46134628
self._connection = connection
46144629
result_meta = self.prepared_statement.result_metadata if self.prepared_statement else []
46154630

@@ -4728,6 +4743,19 @@ def _set_result(self, host, connection, pool, response):
47284743
self._warnings = getattr(response, 'warnings', None)
47294744
self._custom_payload = getattr(response, 'custom_payload', None)
47304745

4746+
if self._custom_payload and self.session.cluster.control_connection._tablets_routing_v1 and 'tablets-routing-v1' in self._custom_payload:
4747+
protocol = self.session.cluster.protocol_version
4748+
info = self._custom_payload.get('tablets-routing-v1')
4749+
ctype = types.lookup_casstype('TupleType(LongType, LongType, ListType(TupleType(UUIDType, Int32Type)))')
4750+
tablet_routing_info = ctype.from_binary(info, protocol)
4751+
first_token = tablet_routing_info[0]
4752+
last_token = tablet_routing_info[1]
4753+
tablet_replicas = tablet_routing_info[2]
4754+
tablet = Tablet.from_row(first_token, last_token, tablet_replicas)
4755+
keyspace = self.query.keyspace
4756+
table = self.query.table
4757+
self.session.cluster.metadata._tablets.add_tablet(keyspace, table, tablet)
4758+
47314759
if isinstance(response, ResultMessage):
47324760
if response.kind == RESULT_KIND_SET_KEYSPACE:
47334761
session = getattr(self, 'session', None)

cassandra/metadata.py

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -44,6 +44,7 @@
4444
from cassandra.pool import HostDistance
4545
from cassandra.connection import EndPoint
4646
from cassandra.compat import Mapping
47+
from cassandra.tablets import Tablets
4748

4849
log = logging.getLogger(__name__)
4950

@@ -126,6 +127,7 @@ def __init__(self):
126127
self._hosts = {}
127128
self._host_id_by_endpoint = {}
128129
self._hosts_lock = RLock()
130+
self._tablets = Tablets({})
129131

130132
def export_schema_as_string(self):
131133
"""

cassandra/policies.py

Lines changed: 15 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -335,6 +335,7 @@ class TokenAwarePolicy(LoadBalancingPolicy):
335335

336336
_child_policy = None
337337
_cluster_metadata = None
338+
_tablets_routing_v1 = False
338339
shuffle_replicas = False
339340
"""
340341
Yield local replicas in a random order.
@@ -346,6 +347,7 @@ def __init__(self, child_policy, shuffle_replicas=False):
346347

347348
def populate(self, cluster, hosts):
348349
self._cluster_metadata = cluster.metadata
350+
self._tablets_routing_v1 = cluster.control_connection._tablets_routing_v1
349351
self._child_policy.populate(cluster, hosts)
350352

351353
def check_supported(self):
@@ -376,7 +378,19 @@ def make_query_plan(self, working_keyspace=None, query=None):
376378
for host in child.make_query_plan(keyspace, query):
377379
yield host
378380
else:
379-
replicas = self._cluster_metadata.get_replicas(keyspace, routing_key)
381+
replicas = []
382+
if self._tablets_routing_v1:
383+
tablet = self._cluster_metadata._tablets.get_tablet_for_key(keyspace, query.table, self._cluster_metadata.token_map.token_class.from_key(routing_key))
384+
385+
if tablet is not None:
386+
replicas_mapped = set(map(lambda r: r[0], tablet.replicas))
387+
child_plan = child.make_query_plan(keyspace, query)
388+
389+
replicas = [host for host in child_plan if host.host_id in replicas_mapped]
390+
391+
if replicas == []:
392+
replicas = self._cluster_metadata.get_replicas(keyspace, routing_key)
393+
380394
if self.shuffle_replicas:
381395
shuffle(replicas)
382396
for replica in replicas:

cassandra/pool.py

Lines changed: 23 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -392,6 +392,8 @@ class HostConnection(object):
392392
# the number below, all excess connections will be closed.
393393
max_excess_connections_per_shard_multiplier = 3
394394

395+
tablets_routing_v1 = False
396+
395397
def __init__(self, host, host_distance, session):
396398
self.host = host
397399
self.host_distance = host_distance
@@ -436,10 +438,11 @@ def __init__(self, host, host_distance, session):
436438
if first_connection.features.sharding_info and not self._session.cluster.shard_aware_options.disable:
437439
self.host.sharding_info = first_connection.features.sharding_info
438440
self._open_connections_for_all_shards(first_connection.features.shard_id)
441+
self.tablets_routing_v1 = first_connection.features.tablets_routing_v1
439442

440443
log.debug("Finished initializing connection for host %s", self.host)
441444

442-
def _get_connection_for_routing_key(self, routing_key=None):
445+
def _get_connection_for_routing_key(self, routing_key=None, keyspace=None, table=None):
443446
if self.is_shutdown:
444447
raise ConnectionException(
445448
"Pool for %s is shutdown" % (self.host,), self.host)
@@ -450,7 +453,22 @@ def _get_connection_for_routing_key(self, routing_key=None):
450453
shard_id = None
451454
if not self._session.cluster.shard_aware_options.disable and self.host.sharding_info and routing_key:
452455
t = self._session.cluster.metadata.token_map.token_class.from_key(routing_key)
453-
shard_id = self.host.sharding_info.shard_id_from_token(t.value)
456+
457+
shard_id = None
458+
if self.tablets_routing_v1 and table is not None:
459+
if keyspace is None:
460+
keyspace = self._keyspace
461+
462+
tablet = self._session.cluster.metadata._tablets.get_tablet_for_key(keyspace, table, t)
463+
464+
if tablet is not None:
465+
for replica in tablet.replicas:
466+
if replica[0] == self.host.host_id:
467+
shard_id = replica[1]
468+
break
469+
470+
if shard_id is None:
471+
shard_id = self.host.sharding_info.shard_id_from_token(t.value)
454472

455473
conn = self._connections.get(shard_id)
456474

@@ -496,15 +514,15 @@ def _get_connection_for_routing_key(self, routing_key=None):
496514
return random.choice(active_connections)
497515
return random.choice(list(self._connections.values()))
498516

499-
def borrow_connection(self, timeout, routing_key=None):
500-
conn = self._get_connection_for_routing_key(routing_key)
517+
def borrow_connection(self, timeout, routing_key=None, keyspace=None, table=None):
518+
conn = self._get_connection_for_routing_key(routing_key, keyspace, table)
501519
start = time.time()
502520
remaining = timeout
503521
last_retry = False
504522
while True:
505523
if conn.is_closed:
506524
# The connection might have been closed in the meantime - if so, try again
507-
conn = self._get_connection_for_routing_key(routing_key)
525+
conn = self._get_connection_for_routing_key(routing_key, keyspace, table)
508526
with conn.lock:
509527
if (not conn.is_closed or last_retry) and conn.in_flight < conn.max_request_id:
510528
# On last retry we ignore connection status, since it is better to return closed connection than

cassandra/protocol_features.py

Lines changed: 11 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -6,22 +6,26 @@
66

77

88
RATE_LIMIT_ERROR_EXTENSION = "SCYLLA_RATE_LIMIT_ERROR"
9+
TABLETS_ROUTING_V1 = "TABLETS_ROUTING_V1"
910

1011
class ProtocolFeatures(object):
1112
rate_limit_error = None
1213
shard_id = 0
1314
sharding_info = None
15+
tablets_routing_v1 = False
1416

15-
def __init__(self, rate_limit_error=None, shard_id=0, sharding_info=None):
17+
def __init__(self, rate_limit_error=None, shard_id=0, sharding_info=None, tablets_routing_v1=False):
1618
self.rate_limit_error = rate_limit_error
1719
self.shard_id = shard_id
1820
self.sharding_info = sharding_info
21+
self.tablets_routing_v1 = tablets_routing_v1
1922

2023
@staticmethod
2124
def parse_from_supported(supported):
2225
rate_limit_error = ProtocolFeatures.maybe_parse_rate_limit_error(supported)
2326
shard_id, sharding_info = ProtocolFeatures.parse_sharding_info(supported)
24-
return ProtocolFeatures(rate_limit_error, shard_id, sharding_info)
27+
tablets_routing_v1 = ProtocolFeatures.parse_tablets_info(supported)
28+
return ProtocolFeatures(rate_limit_error, shard_id, sharding_info, tablets_routing_v1)
2529

2630
@staticmethod
2731
def maybe_parse_rate_limit_error(supported):
@@ -43,6 +47,8 @@ def get_cql_extension_field(vals, key):
4347
def add_startup_options(self, options):
4448
if self.rate_limit_error is not None:
4549
options[RATE_LIMIT_ERROR_EXTENSION] = ""
50+
if self.tablets_routing_v1:
51+
options[TABLETS_ROUTING_V1] = ""
4652

4753
@staticmethod
4854
def parse_sharding_info(options):
@@ -63,3 +69,6 @@ def parse_sharding_info(options):
6369
shard_aware_port, shard_aware_port_ssl)
6470

6571

72+
@staticmethod
73+
def parse_tablets_info(options):
74+
return TABLETS_ROUTING_V1 in options

cassandra/query.py

Lines changed: 11 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -253,6 +253,13 @@ class Statement(object):
253253
.. versionadded:: 2.1.3
254254
"""
255255

256+
table = None
257+
"""
258+
The string name of the table this query acts on. This is used when the tablet
259+
experimental feature is enabled and in the same time :class`~.TokenAwarePolicy`
260+
is configured in the profile load balancing policy.
261+
"""
262+
256263
custom_payload = None
257264
"""
258265
:ref:`custom_payload` to be passed to the server.
@@ -272,7 +279,7 @@ class Statement(object):
272279

273280
def __init__(self, retry_policy=None, consistency_level=None, routing_key=None,
274281
serial_consistency_level=None, fetch_size=FETCH_SIZE_UNSET, keyspace=None, custom_payload=None,
275-
is_idempotent=False):
282+
is_idempotent=False, table=None):
276283
if retry_policy and not hasattr(retry_policy, 'on_read_timeout'): # just checking one method to detect positional parameter errors
277284
raise ValueError('retry_policy should implement cassandra.policies.RetryPolicy')
278285
if retry_policy is not None:
@@ -286,6 +293,8 @@ def __init__(self, retry_policy=None, consistency_level=None, routing_key=None,
286293
self.fetch_size = fetch_size
287294
if keyspace is not None:
288295
self.keyspace = keyspace
296+
if table is not None:
297+
self.table = table
289298
if custom_payload is not None:
290299
self.custom_payload = custom_payload
291300
self.is_idempotent = is_idempotent
@@ -548,6 +557,7 @@ def __init__(self, prepared_statement, retry_policy=None, consistency_level=None
548557
meta = prepared_statement.column_metadata
549558
if meta:
550559
self.keyspace = meta[0].keyspace_name
560+
self.table = meta[0].table_name
551561

552562
Statement.__init__(self, retry_policy, consistency_level, routing_key,
553563
serial_consistency_level, fetch_size, keyspace, custom_payload,

0 commit comments

Comments
 (0)