Skip to content

Commit dbfe82a

Browse files
Bouncheckavelanarius
authored andcommitted
Introduce support for tablets
This PR introduces changes to the driver that are necessary for shard-awareness and token-awareness to work effectively with the tablets feature recently introduced to ScyllaDB. It overwrites the ring-based replica calculations on tablet-enabled keyspaces. Now if driver sends the request to the wrong node/shard it will get the correct tablet information from Scylla in custom payload. It uses this information to obtain target replicas and shard numbers for tables managed by tablet replication. This tablet information is then stored in the driver and is used for correctly routing all next requests.
1 parent 5d2fb2c commit dbfe82a

13 files changed

+718
-12
lines changed

driver-core/src/main/java/com/datastax/driver/core/Connection.java

+9
Original file line numberDiff line numberDiff line change
@@ -474,6 +474,8 @@ public ListenableFuture<Void> apply(Message.Response response) throws Exception
474474
if (lwt != null) {
475475
getHost().setLwtInfo(lwt);
476476
}
477+
TabletInfo tabletInfo = TabletInfo.parseTabletInfo(msg.supported);
478+
getHost().setTabletInfo(tabletInfo);
477479
return MoreFutures.VOID_SUCCESS;
478480
case ERROR:
479481
Responses.Error error = (Responses.Error) response;
@@ -507,6 +509,13 @@ public ListenableFuture<Void> apply(Void input) throws Exception {
507509
if (lwtInfo != null) {
508510
lwtInfo.addOption(extraOptions);
509511
}
512+
TabletInfo tabletInfo = getHost().getTabletInfo();
513+
if (tabletInfo != null
514+
&& tabletInfo.isEnabled()
515+
&& ProtocolFeature.CUSTOM_PAYLOADS.isSupportedBy(protocolVersion)) {
516+
logger.debug("Enabling tablet support in OPTIONS message");
517+
TabletInfo.addOption(extraOptions);
518+
}
510519
Future startupResponseFuture =
511520
write(
512521
new Requests.Startup(

driver-core/src/main/java/com/datastax/driver/core/DefaultResultSetFuture.java

+17
Original file line numberDiff line numberDiff line change
@@ -70,6 +70,23 @@ public void onSet(
7070
switch (response.type) {
7171
case RESULT:
7272
Responses.Result rm = (Responses.Result) response;
73+
74+
if (rm.getCustomPayload() != null
75+
&& rm.getCustomPayload().containsKey(TabletInfo.TABLETS_ROUTING_V1_CUSTOM_PAYLOAD_KEY)
76+
&& (statement instanceof BoundStatement)) {
77+
BoundStatement st = (BoundStatement) statement;
78+
String keyspace = statement.getKeyspace();
79+
String table =
80+
st.preparedStatement().getPreparedId().boundValuesMetadata.variables.getTable(0);
81+
session
82+
.getCluster()
83+
.getMetadata()
84+
.getTabletMap()
85+
.processTabletsRoutingV1Payload(
86+
keyspace,
87+
table,
88+
rm.getCustomPayload().get(TabletInfo.TABLETS_ROUTING_V1_CUSTOM_PAYLOAD_KEY));
89+
}
7390
switch (rm.kind) {
7491
case SET_KEYSPACE:
7592
// propagate the keyspace change to other connections

driver-core/src/main/java/com/datastax/driver/core/Host.java

+11
Original file line numberDiff line numberDiff line change
@@ -70,6 +70,9 @@ public class Host {
7070
// Can be set concurrently but the value should always be the same.
7171
private volatile LwtInfo lwtInfo = null;
7272

73+
// Whether host supports TABLETS_ROUTING_V1
74+
private volatile TabletInfo tabletInfo = null;
75+
7376
enum State {
7477
ADDED,
7578
DOWN,
@@ -450,6 +453,14 @@ public void setLwtInfo(LwtInfo lwtInfo) {
450453
this.lwtInfo = lwtInfo;
451454
}
452455

456+
public TabletInfo getTabletInfo() {
457+
return tabletInfo;
458+
}
459+
460+
public void setTabletInfo(TabletInfo tabletInfo) {
461+
this.tabletInfo = tabletInfo;
462+
}
463+
453464
/**
454465
* Returns whether the host is considered up by the driver.
455466
*

driver-core/src/main/java/com/datastax/driver/core/HostConnectionPool.java

+14-2
Original file line numberDiff line numberDiff line change
@@ -504,7 +504,9 @@ ListenableFuture<Connection> borrowConnection(
504504
TimeUnit unit,
505505
int maxQueueSize,
506506
Token.Factory partitioner,
507-
ByteBuffer routingKey) {
507+
ByteBuffer routingKey,
508+
String keyspace,
509+
String table) {
508510
Phase phase = this.phase.get();
509511
if (phase != Phase.READY)
510512
return Futures.immediateFailedFuture(
@@ -515,7 +517,17 @@ ListenableFuture<Connection> borrowConnection(
515517
if (routingKey != null) {
516518
Metadata metadata = manager.cluster.getMetadata();
517519
Token t = metadata.newToken(partitioner, routingKey);
518-
shardId = host.getShardingInfo().shardId(t);
520+
shardId = -1;
521+
if (keyspace != null && table != null) {
522+
assert t instanceof Token.TokenLong64;
523+
shardId =
524+
Integer.min(
525+
metadata.getShardForTabletToken(keyspace, table, (Token.TokenLong64) t, host),
526+
host.getShardingInfo().getShardsCount());
527+
}
528+
if (shardId == -1) { // means that tablet lookup failed
529+
shardId = host.getShardingInfo().shardId(t);
530+
}
519531
} else {
520532
shardId = RAND.nextInt(host.getShardingInfo().getShardsCount());
521533
}

driver-core/src/main/java/com/datastax/driver/core/Metadata.java

+98-4
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@
2121
*/
2222
package com.datastax.driver.core;
2323

24+
import com.google.common.annotations.Beta;
2425
import com.google.common.collect.ImmutableMap;
2526
import com.google.common.collect.ImmutableSet;
2627
import com.google.common.collect.Maps;
@@ -35,13 +36,15 @@
3536
import java.util.HashSet;
3637
import java.util.List;
3738
import java.util.Map;
39+
import java.util.NavigableSet;
3840
import java.util.Set;
3941
import java.util.TreeSet;
4042
import java.util.UUID;
4143
import java.util.concurrent.ConcurrentHashMap;
4244
import java.util.concurrent.ConcurrentMap;
4345
import java.util.concurrent.CopyOnWriteArrayList;
4446
import java.util.concurrent.locks.ReentrantLock;
47+
import java.util.stream.Collectors;
4548
import org.slf4j.Logger;
4649
import org.slf4j.LoggerFactory;
4750

@@ -60,8 +63,8 @@ public class Metadata {
6063
final ConcurrentMap<String, KeyspaceMetadata> keyspaces =
6164
new ConcurrentHashMap<String, KeyspaceMetadata>();
6265
private volatile TokenMap tokenMap;
63-
6466
final ReentrantLock lock = new ReentrantLock();
67+
private final TabletMap tabletMap;
6568

6669
// See https://github.com/apache/cassandra/blob/trunk/doc/cql3/CQL.textile#appendixA
6770
private static final IntObjectHashMap<List<char[]>> RESERVED_KEYWORDS =
@@ -146,6 +149,7 @@ public class Metadata {
146149

147150
Metadata(Cluster.Manager cluster) {
148151
this.cluster = cluster;
152+
this.tabletMap = TabletMap.emptyMap(cluster);
149153
}
150154

151155
// rebuilds the token map with the current hosts, typically when refreshing schema metadata
@@ -514,21 +518,30 @@ public Set<TokenRange> getTokenRanges(String keyspace, Host host) {
514518
}
515519

516520
/**
517-
* Returns the set of hosts that are replica for a given partition key. Partitioner can be {@code
518-
* null} and then a cluster-wide partitioner will be invoked.
521+
* Extension of legacy method {@link Metadata#getReplicas(String, Token.Factory, ByteBuffer)}.
522+
* Tablets model requires knowledge of the table name to determine the replicas. This method will
523+
* first try to lookup replicas through known tablets metadata. It will default to TokenMap lookup
524+
* if either {@code null} was passed as table name or the tablet lookup is unsuccessful for any
525+
* other reason.
526+
*
527+
* <p>Returns the set of hosts that are replica for a given partition key. Partitioner can be
528+
* {@code null} and then a cluster-wide partitioner will be invoked.
519529
*
520530
* <p>Note that this information is refreshed asynchronously by the control connection, when
521531
* schema or ring topology changes. It might occasionally be stale (or even empty).
522532
*
523533
* @param keyspace the name of the keyspace to get replicas for.
534+
* @param table the name of the table to get replicas for. Necessary for distinction for tablets.
535+
* Unnecessary for regular TokenMap
524536
* @param partitioner the partitioner to use or @{code null} for cluster-wide partitioner.
525537
* @param partitionKey the partition key for which to find the set of replica.
526538
* @return the (immutable) set of replicas for {@code partitionKey} as known by the driver. Note
527539
* that the result might be stale or empty if metadata was explicitly disabled with {@link
528540
* QueryOptions#setMetadataEnabled(boolean)}.
529541
*/
542+
@Beta
530543
public Set<Host> getReplicas(
531-
String keyspace, Token.Factory partitioner, ByteBuffer partitionKey) {
544+
String keyspace, String table, Token.Factory partitioner, ByteBuffer partitionKey) {
532545
keyspace = handleId(keyspace);
533546
TokenMap current = tokenMap;
534547
if (current == null) {
@@ -537,11 +550,40 @@ public Set<Host> getReplicas(
537550
if (partitioner == null) {
538551
partitioner = current.factory;
539552
}
553+
// If possible, try tablet lookup first
554+
if (keyspace != null && table != null) {
555+
Token token = partitioner.hash(partitionKey);
556+
assert (token instanceof Token.TokenLong64);
557+
Set<UUID> hostUuids = tabletMap.getReplicas(keyspace, table, (long) token.getValue());
558+
if (!hostUuids.isEmpty()) {
559+
return hostUuids.stream().map(this::getHost).collect(Collectors.toSet());
560+
}
561+
}
562+
// Fall back to tokenMap
540563
Set<Host> hosts = current.getReplicas(keyspace, partitioner.hash(partitionKey));
541564
return hosts == null ? Collections.<Host>emptySet() : hosts;
542565
}
543566
}
544567

568+
/**
569+
* Returns the set of hosts that are replica for a given partition key. Partitioner can be {@code
570+
* null} and then a cluster-wide partitioner will be invoked.
571+
*
572+
* <p>Note that this information is refreshed asynchronously by the control connection, when
573+
* schema or ring topology changes. It might occasionally be stale (or even empty).
574+
*
575+
* @param keyspace the name of the keyspace to get replicas for.
576+
* @param partitioner the partitioner to use or @{code null} for cluster-wide partitioner.
577+
* @param partitionKey the partition key for which to find the set of replica.
578+
* @return the (immutable) set of replicas for {@code partitionKey} as known by the driver. Note
579+
* that the result might be stale or empty if metadata was explicitly disabled with {@link
580+
* QueryOptions#setMetadataEnabled(boolean)}.
581+
*/
582+
public Set<Host> getReplicas(
583+
String keyspace, Token.Factory partitioner, ByteBuffer partitionKey) {
584+
return getReplicas(keyspace, null, partitioner, partitionKey);
585+
}
586+
545587
/**
546588
* Returns the set of hosts that are replica for a given token range.
547589
*
@@ -860,6 +902,58 @@ void triggerOnMaterializedViewRemoved(MaterializedViewMetadata view) {
860902
}
861903
}
862904

905+
@Beta
906+
public int getShardForTabletToken(
907+
String keyspace, String table, Token.TokenLong64 token, Host host) {
908+
if (tabletMap == null) {
909+
logger.trace(
910+
"Could not determine shard for token {} on host {} because tablets metadata is currently null. "
911+
+ "Returning -1.",
912+
token,
913+
host);
914+
return -1;
915+
}
916+
UUID targetHostUuid = host.getHostId();
917+
long tokenValue = (long) token.getValue();
918+
TabletMap.KeyspaceTableNamePair key = new TabletMap.KeyspaceTableNamePair(keyspace, table);
919+
NavigableSet<TabletMap.Tablet> targetTablets = tabletMap.getMapping().get(key);
920+
if (targetTablets == null) {
921+
logger.trace(
922+
"Could not determine shard for token {} on host {} because table {}.{} is not present in tablets "
923+
+ "metadata. Returning -1.",
924+
token,
925+
host,
926+
keyspace,
927+
table);
928+
return -1;
929+
}
930+
TabletMap.Tablet row = targetTablets.ceiling(TabletMap.Tablet.malformedTablet(tokenValue));
931+
if (row != null && row.getFirstToken() < tokenValue) {
932+
for (TabletMap.HostShardPair hostShardPair : row.getReplicas()) {
933+
if (hostShardPair.getHost().equals(targetHostUuid)) {
934+
return hostShardPair.getShard();
935+
}
936+
}
937+
}
938+
logger.trace(
939+
"Could not find tablet corresponding to token {} on host {} for table {} in keyspace {}. Returning -1.",
940+
token,
941+
host,
942+
table,
943+
keyspace);
944+
return -1;
945+
}
946+
947+
/**
948+
* Getter for current {@link TabletMap}.
949+
*
950+
* @return current {@link TabletMap}
951+
*/
952+
@Beta
953+
public TabletMap getTabletMap() {
954+
return tabletMap;
955+
}
956+
863957
private static class TokenMap {
864958

865959
private final Token.Factory factory;

driver-core/src/main/java/com/datastax/driver/core/RequestHandler.java

+28-2
Original file line numberDiff line numberDiff line change
@@ -110,11 +110,22 @@ private Iterator<Host> getReplicas(
110110
}
111111

112112
Token.Factory partitioner = statement.getPartitioner();
113+
String tableName = null;
114+
ColumnDefinitions defs = null;
115+
if (statement instanceof BoundStatement) {
116+
defs = ((BoundStatement) statement).preparedStatement().getVariables();
117+
} else if (statement instanceof PreparedStatement) {
118+
defs = ((PreparedStatement) statement).getVariables();
119+
}
120+
if (defs != null && defs.size() > 0) {
121+
tableName = defs.getTable(0);
122+
}
123+
113124
final Set<Host> replicas =
114125
manager
115126
.cluster
116127
.getMetadata()
117-
.getReplicas(Metadata.quote(keyspace), partitioner, partitionKey);
128+
.getReplicas(Metadata.quote(keyspace), tableName, partitioner, partitionKey);
118129

119130
// replicas are stored in the right order starting with the primary replica
120131
return replicas.iterator();
@@ -437,13 +448,28 @@ private boolean query(final Host host) {
437448
ByteBuffer routingKey = statement.getRoutingKey(protocolVersion, codecRegistry);
438449

439450
PoolingOptions poolingOptions = manager.configuration().getPoolingOptions();
451+
String statementKeyspace = statement.getKeyspace();
452+
String statementTable = null;
453+
ColumnDefinitions defs = null;
454+
if (statement instanceof PreparedStatement) {
455+
defs = ((PreparedStatement) statement).getVariables();
456+
}
457+
if (statement instanceof BoundStatement) {
458+
defs = ((BoundStatement) statement).statement.getVariables();
459+
}
460+
if (defs != null && defs.size() > 0) {
461+
statementTable = defs.getTable(0);
462+
}
463+
440464
ListenableFuture<Connection> connectionFuture =
441465
pool.borrowConnection(
442466
poolingOptions.getPoolTimeoutMillis(),
443467
TimeUnit.MILLISECONDS,
444468
poolingOptions.getMaxQueueSize(),
445469
statement.getPartitioner(),
446-
routingKey);
470+
routingKey,
471+
statementKeyspace,
472+
statementTable);
447473
GuavaCompatibility.INSTANCE.addCallback(
448474
connectionFuture,
449475
new FutureCallback<Connection>() {

driver-core/src/main/java/com/datastax/driver/core/SessionManager.java

+10-1
Original file line numberDiff line numberDiff line change
@@ -733,13 +733,22 @@ private ListenableFuture<PreparedStatement> prepare(
733733
if (entry.getKey().getEndPoint().equals(toExclude)) continue;
734734

735735
try {
736+
ColumnDefinitions defs = statement.getVariables();
737+
String statementTable = (defs != null && defs.size() > 0 ? defs.getTable(0) : null);
736738
// Preparing is not critical: if it fails, it will fix itself later when the user tries to
737739
// execute
738740
// the prepared query. So don't wait if no connection is available, simply abort.
739741
ListenableFuture<Connection> connectionFuture =
740742
entry
741743
.getValue()
742-
.borrowConnection(0, TimeUnit.MILLISECONDS, 0, null, statement.getRoutingKey());
744+
.borrowConnection(
745+
0,
746+
TimeUnit.MILLISECONDS,
747+
0,
748+
null,
749+
statement.getRoutingKey(),
750+
statement.getQueryKeyspace(),
751+
statementTable);
743752
ListenableFuture<Response> prepareFuture =
744753
GuavaCompatibility.INSTANCE.transformAsync(
745754
connectionFuture,
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,33 @@
1+
package com.datastax.driver.core;
2+
3+
import java.util.List;
4+
import java.util.Map;
5+
6+
public class TabletInfo {
7+
private static final String SCYLLA_TABLETS_STARTUP_OPTION_KEY = "TABLETS_ROUTING_V1";
8+
private static final String SCYLLA_TABLETS_STARTUP_OPTION_VALUE = "";
9+
public static final String TABLETS_ROUTING_V1_CUSTOM_PAYLOAD_KEY = "tablets-routing-v1";
10+
11+
private boolean enabled = false;
12+
13+
private TabletInfo(boolean enabled) {
14+
this.enabled = enabled;
15+
}
16+
17+
// Currently pertains only to TABLETS_ROUTING_V1
18+
public boolean isEnabled() {
19+
return enabled;
20+
}
21+
22+
public static TabletInfo parseTabletInfo(Map<String, List<String>> supported) {
23+
List<String> values = supported.get(SCYLLA_TABLETS_STARTUP_OPTION_KEY);
24+
return new TabletInfo(
25+
values != null
26+
&& values.size() == 1
27+
&& values.get(0).equals(SCYLLA_TABLETS_STARTUP_OPTION_VALUE));
28+
}
29+
30+
public static void addOption(Map<String, String> options) {
31+
options.put(SCYLLA_TABLETS_STARTUP_OPTION_KEY, SCYLLA_TABLETS_STARTUP_OPTION_VALUE);
32+
}
33+
}

0 commit comments

Comments
 (0)