Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[3.x] RackAwareRoundRobinPolicy seems incompatible with ReplicaOrdering.RANDOM in TokenAwarePolicy #369

Open
Bouncheck opened this issue Nov 19, 2024 · 6 comments
Labels
bug Something isn't working

Comments

@Bouncheck
Copy link
Collaborator

Bouncheck commented Nov 19, 2024

See the following snippet from TokenAwarePolicy.java

@Override
        protected Host computeNext() {
          while (replicasIterator.hasNext()) {
            Host host = replicasIterator.next();
            if (host.isUp() && childPolicy.distance(host) == HostDistance.LOCAL) return host;
          }
...

This is how we calculate the first hosts in query plan in case of ReplicaOrdering.RANDOM. We prioritise replicas (which have been random shuffled beforehand) and select first those that according to child policy are local replicas. The problem is with distance implementation of RackAwareRoundRobinPolicy

@Override
  public HostDistance distance(Host host) {
    String dc = dc(host);
    if (dc == UNSET || dc.equals(localDc)) return HostDistance.LOCAL;

    CopyOnWriteArrayList<Host> dcHosts = perDcLiveHosts.get(dc);
    if (dcHosts == null || usedHostsPerRemoteDc == 0) return HostDistance.IGNORED;

    // We need to clone, otherwise our subList call is not thread safe
    dcHosts = cloneList(dcHosts);
    return dcHosts.subList(0, Math.min(dcHosts.size(), usedHostsPerRemoteDc)).contains(host)
        ? HostDistance.REMOTE
        : HostDistance.IGNORED;
  }

On its own this policy prioritizes first local rack, then remote racks but in local DC then remote DCs. However, HostDistance right now only differentiates between LOCAL, REMOTE and IGNORED. If this policy is used with random ordering token aware policy then if remote rack but local dc replica comes first in the list of replicas, then it will be considered LOCAL according to this code snippet, invalidating the property of local rack going first. It's effectively DCAware and not RackAware in that configuration.

@dkropachev
Copy link
Collaborator

On that front I also want to point out that both 3.x and 4.x ignore LoadBalancingPolicy and return all the replicas in the same order they get it from vnodes or tablets for LWT statements and it seems wrong to me:

  • Java 3.x
    if (statement.getHost() != null) {
    this.queryPlan = new QueryPlan(Iterators.singletonIterator(statement.getHost()));
    } else if (statement.isLWT()) {
    this.queryPlan =
    new QueryPlan(
    getReplicas(
    manager.poolsState.keyspace,
    statement,
    manager
    .loadBalancingPolicy()
    .newQueryPlan(manager.poolsState.keyspace, statement)));
    } else {
    this.queryPlan =
    new QueryPlan(
    manager.loadBalancingPolicy().newQueryPlan(manager.poolsState.keyspace, statement));
    }
  • Java 4.x
    if (this.initialStatement.getNode() != null) {
    queryPlan = new SimpleQueryPlan(this.initialStatement.getNode());
    } else if (this.initialStatement.isLWT()) {
    queryPlan =
    getReplicas(
    session.getKeyspace().orElse(null),
    this.initialStatement,
    context
    .getLoadBalancingPolicyWrapper()
    .newQueryPlan(initialStatement, executionProfile.getName(), session));
    } else {
    queryPlan =
    context
    .getLoadBalancingPolicyWrapper()
    .newQueryPlan(initialStatement, executionProfile.getName(), session);
    }
    sendRequest(initialStatement, null, queryPlan, 0, 0, true);
    }

@Bouncheck
Copy link
Collaborator Author

Bouncheck commented Nov 19, 2024

To me it seems like an appropriate decision. Those structures are supposed just to tell what are the replicas and the LBP's job is to balance the load.

I reread the comment and now I see that you are talking about LWT

@Bouncheck
Copy link
Collaborator Author

This seems like a separate issue though. I'm not convinced that it is wrong since IIRC LWT queries have its own rules. I think it's meant not to follow LBP 100%.

@dkropachev
Copy link
Collaborator

This seems like a separate issue though. I'm not convinced that it is wrong since IIRC LWT queries have its own rules. I think it's meant not to follow LBP 100%.

True, but how do you force them target particular rack, datacenter ?

@dkropachev
Copy link
Collaborator

This seems like a separate issue though. I'm not convinced that it is wrong since IIRC LWT queries have its own rules. I think it's meant not to follow LBP 100%.

What to do with racks then ?

@Bouncheck
Copy link
Collaborator Author

This seems like a separate issue though. I'm not convinced that it is wrong since IIRC LWT queries have its own rules. I think it's meant not to follow LBP 100%.

What to do with racks then ?

I'm not sure. We probably should care about where the primary replica is rather than target particular rack. I'd need to brush up on how LWT is supposed to work, especially with tablets and raft.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something isn't working
Projects
None yet
Development

No branches or pull requests

2 participants