Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -1,6 +1,7 @@
package com.datastax.oss.driver.api.core.metadata;

import com.datastax.oss.driver.shaded.guava.common.annotations.Beta;
import java.util.List;
import java.util.Set;

/**
Expand All @@ -25,8 +26,25 @@ public interface Tablet extends Comparable<Tablet> {
*/
public long getLastToken();

/**
* Returns replica nodes in the order reported by the tablets-routing-v1 payload.
*
* <p>This set is immutable
*
* @deprecated Use {@link #getReplicaNodesList()} instead.
*/
@Deprecated
public Set<Node> getReplicaNodes();

/**
* Returns replica nodes in the order reported by the tablets-routing-v1 payload.
*
* <p>This list is immutable.
*
* @return ordered list of replica nodes for this tablet
*/
public List<Node> getReplicaNodesList();

/**
* Looks up the shard number for specific replica Node.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,10 +31,12 @@
import com.datastax.oss.driver.api.core.metadata.token.TokenRange;
import com.datastax.oss.driver.api.core.session.Session;
import com.datastax.oss.driver.api.core.type.codec.TypeCodec;
import com.datastax.oss.driver.shaded.guava.common.collect.ImmutableList;
import com.datastax.oss.driver.shaded.guava.common.collect.ImmutableSet;
import edu.umd.cs.findbugs.annotations.NonNull;
import edu.umd.cs.findbugs.annotations.Nullable;
import java.nio.ByteBuffer;
import java.util.List;
import java.util.Set;

/**
Expand Down Expand Up @@ -114,78 +116,222 @@ default Set<TokenRange> getTokenRanges(@NonNull String keyspaceName, @NonNull No
return getTokenRanges(CqlIdentifier.fromCql(keyspaceName), replica);
}

/** The replicas for a given partition key in the given keyspace. */
/**
* The replicas for a given partition key in the given keyspace.
*
* @deprecated Use {@link #getReplicasList(CqlIdentifier, Partitioner, ByteBuffer)} instead.
*/
@NonNull
@Deprecated
default Set<Node> getReplicas(
@NonNull CqlIdentifier keyspace,
@Nullable Partitioner partitioner,
@NonNull ByteBuffer partitionKey) {
return ImmutableSet.copyOf(getReplicasList(keyspace, partitioner, partitionKey));
}

/**
* The replicas for a given partition key in the given keyspace, in ring order.
*
* @param keyspace the keyspace for which to get replicas
* @param partitioner the partitioner to use or {@code null} for this TokenMap's partitioner
* @param partitionKey the partition key in its serialized form
* @return ordered list of replica nodes for the given partition key
*/
@NonNull
Set<Node> getReplicas(
List<Node> getReplicasList(
@NonNull CqlIdentifier keyspace,
@Nullable Partitioner partitioner,
@NonNull ByteBuffer partitionKey);

/**
* Shortcut for {@link #getReplicas(CqlIdentifier, Partitioner, ByteBuffer) getReplicas(keyspace,
* null, partitionKey)}.
*
* @deprecated Use {@link #getReplicasList(CqlIdentifier, ByteBuffer)} instead.
*/
@NonNull
@Deprecated
default Set<Node> getReplicas(@NonNull CqlIdentifier keyspace, @NonNull ByteBuffer partitionKey) {
return getReplicas(keyspace, null, partitionKey);
}

/**
* Shortcut for {@link #getReplicasList(CqlIdentifier, Partitioner, ByteBuffer)
* getReplicasList(keyspace, null, partitionKey)}.
*
* @param keyspace the keyspace for which to get replicas
* @param partitionKey the partition key in its serialized form
* @return ordered list of replica nodes for the given partition key
*/
@NonNull
default List<Node> getReplicasList(
@NonNull CqlIdentifier keyspace, @NonNull ByteBuffer partitionKey) {
return getReplicasList(keyspace, null, partitionKey);
}

/**
* Shortcut for {@link #getReplicas(CqlIdentifier, Partitioner, ByteBuffer)
* getReplicas(CqlIdentifier.fromCql(keyspaceName), partitioner, partitionKey)}.
*
* @deprecated Use {@link #getReplicasList(String, Partitioner, ByteBuffer)} instead.
*/
@NonNull
@Deprecated
default Set<Node> getReplicas(
@NonNull String keyspaceName,
@Nullable Partitioner partitioner,
@NonNull ByteBuffer partitionKey) {
return getReplicas(CqlIdentifier.fromCql(keyspaceName), partitioner, partitionKey);
}

/**
* Shortcut for {@link #getReplicasList(CqlIdentifier, Partitioner, ByteBuffer)
* getReplicasList(CqlIdentifier.fromCql(keyspaceName), partitioner, partitionKey)}.
*
* @param keyspaceName the keyspace name for which to get replicas
* @param partitioner the partitioner to use or {@code null} for this TokenMap's partitioner
* @param partitionKey the partition key in its serialized form
* @return ordered list of replica nodes for the given partition key
*/
@NonNull
default List<Node> getReplicasList(
@NonNull String keyspaceName,
@Nullable Partitioner partitioner,
@NonNull ByteBuffer partitionKey) {
return getReplicasList(CqlIdentifier.fromCql(keyspaceName), partitioner, partitionKey);
}

/**
* Shortcut for {@link #getReplicas(CqlIdentifier, Partitioner, ByteBuffer)
* getReplicas(CqlIdentifier.fromCql(keyspaceName), null, partitionKey)}.
*
* @deprecated Use {@link #getReplicasList(String, ByteBuffer)} instead.
*/
@NonNull
@Deprecated
default Set<Node> getReplicas(@NonNull String keyspaceName, @NonNull ByteBuffer partitionKey) {
return getReplicas(CqlIdentifier.fromCql(keyspaceName), null, partitionKey);
}

/** The replicas for a given token in the given keyspace. */
/**
* Shortcut for {@link #getReplicasList(CqlIdentifier, Partitioner, ByteBuffer)
* getReplicasList(CqlIdentifier.fromCql(keyspaceName), null, partitionKey)}.
*
* @param keyspaceName the keyspace name for which to get replicas
* @param partitionKey the partition key in its serialized form
* @return ordered list of replica nodes for the given partition key
*/
@NonNull
default List<Node> getReplicasList(
@NonNull String keyspaceName, @NonNull ByteBuffer partitionKey) {
return getReplicasList(CqlIdentifier.fromCql(keyspaceName), null, partitionKey);
}

/**
* The replicas for a given token in the given keyspace.
*
* @deprecated Use {@link #getReplicasList(CqlIdentifier, Token)} instead.
*/
@NonNull
@Deprecated
Set<Node> getReplicas(@NonNull CqlIdentifier keyspace, @NonNull Token token);

/**
* The replicas for a given token in the given keyspace, in ring order.
*
* @param keyspace the keyspace for which to get replicas
* @param token the token for which to get replicas
* @return ordered list of replica nodes for the given token
*/
@NonNull
default List<Node> getReplicasList(@NonNull CqlIdentifier keyspace, @NonNull Token token) {
return ImmutableList.copyOf(getReplicas(keyspace, token));
}

/**
* Shortcut for {@link #getReplicas(CqlIdentifier, Token)
* getReplicas(CqlIdentifier.fromCql(keyspaceName), token)}.
*
* @deprecated Use {@link #getReplicasList(String, Token)} instead.
*/
@NonNull
@Deprecated
default Set<Node> getReplicas(@NonNull String keyspaceName, @NonNull Token token) {
return getReplicas(CqlIdentifier.fromCql(keyspaceName), token);
}

/**
* Shortcut for {@link #getReplicasList(CqlIdentifier, Token)
* getReplicasList(CqlIdentifier.fromCql(keyspaceName), token)}.
*
* @param keyspaceName the keyspace name for which to get replicas
* @param token the token for which to get replicas
* @return ordered list of replica nodes for the given token
*/
@NonNull
default List<Node> getReplicasList(@NonNull String keyspaceName, @NonNull Token token) {
return getReplicasList(CqlIdentifier.fromCql(keyspaceName), token);
}

/**
* The replicas for a given range in the given keyspace.
*
* <p>It is assumed that the input range does not overlap across multiple node ranges. If the
* range extends over multiple nodes, it only returns the nodes that are replicas for the last
* token of the range. In other words, this method is a shortcut for {@code getReplicas(keyspace,
* range.getEnd())}.
*
* @deprecated Use {@link #getReplicasList(CqlIdentifier, TokenRange)} instead.
*/
@NonNull
@Deprecated
default Set<Node> getReplicas(@NonNull CqlIdentifier keyspace, @NonNull TokenRange range) {
return getReplicas(keyspace, range.getEnd());
}

/**
* The replicas for a given range in the given keyspace, in ring order.
*
* <p>It is assumed that the input range does not overlap across multiple node ranges. If the
* range extends over multiple nodes, it only returns the nodes that are replicas for the last
* token of the range. In other words, this method is a shortcut for {@code
* getReplicasList(keyspace, range.getEnd())}.
*
* @param keyspace the keyspace for which to get replicas
* @param range the token range for which to get replicas
* @return ordered list of replica nodes for the given token range
*/
@NonNull
default List<Node> getReplicasList(@NonNull CqlIdentifier keyspace, @NonNull TokenRange range) {
return getReplicasList(keyspace, range.getEnd());
}

/**
* Shortcut for {@link #getReplicas(CqlIdentifier, TokenRange)
* getReplicas(CqlIdentifier.fromCql(keyspaceName), range)}.
*
* @deprecated Use {@link #getReplicasList(String, TokenRange)} instead.
*/
@NonNull
@Deprecated
default Set<Node> getReplicas(@NonNull String keyspaceName, @NonNull TokenRange range) {
return getReplicas(CqlIdentifier.fromCql(keyspaceName), range);
}

/**
* Shortcut for {@link #getReplicasList(CqlIdentifier, TokenRange)
* getReplicasList(CqlIdentifier.fromCql(keyspaceName), range)}.
*
* @param keyspaceName the keyspace name for which to get replicas
* @param range the token range for which to get replicas
* @return ordered list of replica nodes for the given token range
*/
@NonNull
default List<Node> getReplicasList(@NonNull String keyspaceName, @NonNull TokenRange range) {
return getReplicasList(CqlIdentifier.fromCql(keyspaceName), range);
}

/** The name of the partitioner class in use, as reported by the Cassandra nodes. */
@NonNull
String getPartitionerName();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,12 +57,12 @@
import com.datastax.oss.driver.internal.core.util.collection.QueryPlan;
import com.datastax.oss.driver.internal.core.util.collection.SimpleQueryPlan;
import com.datastax.oss.driver.shaded.guava.common.base.Predicates;
import com.datastax.oss.driver.shaded.guava.common.collect.ImmutableList;
import com.datastax.oss.driver.shaded.guava.common.collect.Lists;
import com.datastax.oss.driver.shaded.guava.common.collect.Sets;
import edu.umd.cs.findbugs.annotations.NonNull;
import edu.umd.cs.findbugs.annotations.Nullable;
import java.nio.ByteBuffer;
import java.util.Collections;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Map;
Expand Down Expand Up @@ -263,7 +263,7 @@ public Queue<Node> newQueryPlan(@Nullable Request request, @Nullable Session ses
// Take a snapshot since the set is concurrent:
Object[] currentNodes = liveNodes.dc(localDc).toArray();

Set<Node> allReplicas = getReplicas(request, session);
List<Node> allReplicas = getReplicas(request, session);
int replicaCount = 0; // in currentNodes

if (!allReplicas.isEmpty()) {
Expand Down Expand Up @@ -295,9 +295,9 @@ public Queue<Node> newQueryPlan(@Nullable Request request, @Nullable Session ses
}

@NonNull
protected Set<Node> getReplicas(@Nullable Request request, @Nullable Session session) {
protected List<Node> getReplicas(@Nullable Request request, @Nullable Session session) {
if (request == null || session == null) {
return Collections.emptySet();
return ImmutableList.of();
}

Optional<TokenMap> maybeTokenMap = context.getMetadataManager().getMetadata().getTokenMap();
Expand All @@ -321,15 +321,15 @@ protected Set<Node> getReplicas(@Nullable Request request, @Nullable Session ses
keyspace = session.getKeyspace().get();
}
if (keyspace == null) {
return Collections.emptySet();
return ImmutableList.of();
}

table = request.getRoutingTable();

token = request.getRoutingToken();
key = (token == null) ? request.getRoutingKey() : null;
if (token == null && key == null) {
return Collections.emptySet();
return ImmutableList.of();
}

partitioner = request.getPartitioner();
Expand All @@ -339,7 +339,7 @@ protected Set<Node> getReplicas(@Nullable Request request, @Nullable Session ses
} catch (Exception e) {
// Protect against poorly-implemented Request instances
LOG.error("Unexpected error while trying to compute query plan", e);
return Collections.emptySet();
return ImmutableList.of();
}

if (token == null && partitioner != null) {
Expand All @@ -350,25 +350,25 @@ protected Set<Node> getReplicas(@Nullable Request request, @Nullable Session ses
context.getMetadataManager().getMetadata().getKeyspace(keyspace);
if (ksMetadata.isPresent() && ksMetadata.get().isUsingTablets() && maybeTabletMap.isPresent()) {
if (table == null) {
return Collections.emptySet();
return ImmutableList.of();
}
if (token instanceof TokenLong64) {
Tablet targetTablet =
maybeTabletMap.get().getTablet(keyspace, table, ((TokenLong64) token).getValue());
if (targetTablet != null) {
return targetTablet.getReplicaNodes();
return targetTablet.getReplicaNodesList();
}
}
return Collections.emptySet();
return ImmutableList.of();
}

if (!maybeTokenMap.isPresent()) {
return Collections.emptySet();
return ImmutableList.of();
}
TokenMap tokenMap = maybeTokenMap.get();
return token != null
? tokenMap.getReplicas(keyspace, token)
: tokenMap.getReplicas(keyspace, partitioner, key);
? tokenMap.getReplicasList(keyspace, token)
: tokenMap.getReplicasList(keyspace, partitioner, key);
}

@NonNull
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,12 +38,12 @@
import edu.umd.cs.findbugs.annotations.NonNull;
import edu.umd.cs.findbugs.annotations.Nullable;
import java.util.BitSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.OptionalLong;
import java.util.Queue;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
Expand Down Expand Up @@ -138,7 +138,7 @@ public Queue<Node> newQueryPlan(@Nullable Request request, @Nullable Session ses
// Take a snapshot since the set is concurrent:
Object[] currentNodes = getLiveNodes().dc(getLocalDatacenter()).toArray();

Set<Node> allReplicas = getReplicas(request, session);
List<Node> allReplicas = getReplicas(request, session);
int replicaCount = 0; // in currentNodes
int localRackReplicaCount = 0; // in currentNodes
String localRack = getLocalRack();
Expand Down
Loading
Loading