adutra commented on code in PR #1896:
URL: 
https://github.com/apache/cassandra-java-driver/pull/1896#discussion_r1435650327


##########
core/src/main/java/com/datastax/oss/driver/api/core/config/TypedDriverOption.java:
##########
@@ -886,6 +886,16 @@ public String toString() {
               
DefaultDriverOption.LOAD_BALANCING_DC_FAILOVER_ALLOW_FOR_LOCAL_CONSISTENCY_LEVELS,
               GenericType.BOOLEAN);
 
+  /**
+   * Ordered preference list of remote dc's optionally supplied for automatic 
failover and included
+   * in query plan. This feature is enabled only when max-nodes-per-remote-dc 
is greater than 0

Review Comment:
   ```suggestion
      * in query plan. This feature is enabled only when 
max-nodes-per-remote-dc is greater than 0.
   ```



##########
core/src/main/java/com/datastax/oss/driver/internal/core/loadbalancing/BasicLoadBalancingPolicy.java:
##########
@@ -131,6 +135,15 @@ public BasicLoadBalancingPolicy(@NonNull DriverContext 
context, @NonNull String
         this.context
             .getConsistencyLevelRegistry()
             
.nameToLevel(profile.getString(DefaultDriverOption.REQUEST_CONSISTENCY));
+
+    final List<String> remoteDcs =

Review Comment:
   For improved efficiency, I'd suggest that you store the preferred DCs in a 
`String[]`:
   
   ```java
       Set<String> remoteDcs =
           new LinkedHashSet<>(
               Objects.requireNonNull(
                   profile.getStringList(
                       
DefaultDriverOption.LOAD_BALANCING_DC_FAILOVER_PREFERRED_REMOTE_DCS,
                       new ArrayList<>())));
       preferredRemoteDcs = remoteDcs.toArray(new String[0]);
   ```



##########
core/src/main/java/com/datastax/oss/driver/internal/core/loadbalancing/BasicLoadBalancingPolicy.java:
##########
@@ -117,6 +120,7 @@ public class BasicLoadBalancingPolicy implements 
LoadBalancingPolicy {
   private volatile NodeDistanceEvaluator nodeDistanceEvaluator;
   private volatile String localDc;
   private volatile NodeSet liveNodes;
+  private volatile Set<String> preferredRemoteLocalDcs = new LinkedHashSet<>();

Review Comment:
   Rename to `preferredRemoteDcs`. Also, the field should be final.



##########
core/src/main/java/com/datastax/oss/driver/internal/core/loadbalancing/BasicLoadBalancingPolicy.java:
##########
@@ -325,24 +338,63 @@ protected Queue<Node> maybeAddDcFailover(@Nullable 
Request request, @NonNull Que
 
           @Override
           protected Object[] computeNodes() {
-            Object[] remoteNodes =
-                liveNodes.dcs().stream()
-                    .filter(Predicates.not(Predicates.equalTo(localDc)))
-                    .flatMap(dc -> 
liveNodes.dc(dc).stream().limit(maxNodesPerRemoteDc))
-                    .toArray();
-
-            int remoteNodesLength = remoteNodes.length;
-            if (remoteNodesLength == 0) {
-              return EMPTY_NODES;
+            if (!preferredRemoteLocalDcs.isEmpty()) {

Review Comment:
   Isn't this unnecessarily complex? I think the same effect could be achieved 
with a sorted stream:
   
   ```java
   Object[] remoteNodes =
       liveNodes.dcs().stream()
           .filter(Predicates.not(Predicates.equalTo(localDc)))
           .sorted(
               Comparator.comparingInt(
                   dc -> {
                     int i = Arrays.binarySearch(preferredRemoteDcs, dc);
                     return i < 0 ? Integer.MAX_VALUE : i;
                   }))
           .flatMap(dc -> liveNodes.dc(dc).stream().limit(maxNodesPerRemoteDc))
           .toArray();
   ```
   
   And indeed, in presence of preferred dcs, it probably doesn't make sense to 
shuffle the remote nodes:
   
   ```java
   if (preferredRemoteDcs.length == 0) {
     shuffleHead(remoteNodes, remoteNodesLength);
   }
   ```
   



##########
core/src/test/java/com/datastax/oss/driver/internal/core/loadbalancing/BasicLoadBalancingPolicyRemoteDcTest.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.datastax.oss.driver.internal.core.loadbalancing;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.BDDMockito.then;
+import static org.mockito.Mockito.atLeast;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.datastax.oss.driver.api.core.config.DefaultDriverOption;
+import com.datastax.oss.driver.api.core.config.DriverExecutionProfile;
+import com.datastax.oss.driver.api.core.metadata.Node;
+import com.datastax.oss.driver.internal.core.metadata.DefaultNode;
+import com.datastax.oss.driver.shaded.guava.common.collect.ImmutableMap;
+import com.datastax.oss.driver.shaded.guava.common.collect.ImmutableSet;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.junit.Test;
+import org.mockito.Mock;
+
+public class BasicLoadBalancingPolicyRemoteDcTest extends 
BasicLoadBalancingPolicyQueryPlanTest {

Review Comment:
   Rename to `BasicLoadBalancingPolicyPreferredRemoteDcsTest`. Also, maybe it 
would have been simpler to extend `BasicLoadBalancingPolicyDcFailoverTest`.



##########
core/src/main/java/com/datastax/oss/driver/internal/core/loadbalancing/BasicLoadBalancingPolicy.java:
##########
@@ -325,24 +338,63 @@ protected Queue<Node> maybeAddDcFailover(@Nullable 
Request request, @NonNull Que
 
           @Override
           protected Object[] computeNodes() {
-            Object[] remoteNodes =
-                liveNodes.dcs().stream()
-                    .filter(Predicates.not(Predicates.equalTo(localDc)))
-                    .flatMap(dc -> 
liveNodes.dc(dc).stream().limit(maxNodesPerRemoteDc))
-                    .toArray();
-
-            int remoteNodesLength = remoteNodes.length;
-            if (remoteNodesLength == 0) {
-              return EMPTY_NODES;
+            if (!preferredRemoteLocalDcs.isEmpty()) {
+              final List<Object> nodes = new ArrayList<>();
+
+              preferredRemoteLocalDcs.forEach(dc -> addRemoteNodes(nodes, dc));
+
+              liveNodes
+                  .dcs()
+                  .forEach(
+                      dc -> {
+                        if (!preferredRemoteLocalDcs.contains(dc) && 
!dc.equals(localDc)) {
+                          addRemoteNodes(nodes, dc);
+                        }
+                      });
+
+              if (!nodes.isEmpty()) {
+                return nodes.toArray();
+              } else {
+                return EMPTY_NODES;
+              }
+            } else {
+              Object[] remoteNodes =
+                  liveNodes.dcs().stream()
+                      .filter(Predicates.not(Predicates.equalTo(localDc)))
+                      .flatMap(dc -> 
liveNodes.dc(dc).stream().limit(maxNodesPerRemoteDc))
+                      .toArray();
+
+              int remoteNodesLength = remoteNodes.length;
+              if (remoteNodesLength == 0) {
+                return EMPTY_NODES;
+              }
+              shuffleHead(remoteNodes, remoteNodesLength);
+              return remoteNodes;
+            }
+          }
+
+          private void addRemoteNodes(final List<Object> nodes, final String 
dc) {
+            final Object[] remoteNodesPerDc =
+                liveNodes.dc(dc).stream().limit(maxNodesPerRemoteDc).toArray();
+            int remoteNodesLength = remoteNodesPerDc.length;
+            if (remoteNodesLength > 0) {
+              shuffleHeadRemote(remoteNodesPerDc, remoteNodesLength);
+
+              for (int i = 0; i < remoteNodesLength; i++) {
+                nodes.add(remoteNodesPerDc[i]);
+              }
             }
-            shuffleHead(remoteNodes, remoteNodesLength);
-            return remoteNodes;
           }
         };
 
     return new CompositeQueryPlan(local, remote);
   }
 
+  /** Exposed as a protected method so that it can be accessed by tests */
+  protected void shuffleHeadRemote(Object[] currentNodes, int headLength) {

Review Comment:
   Not a huge fan of this tbh. I believe you can distinguish calls to 
`shuffleHead` by inspecting their arguments. See here for an example:
   
   
https://github.com/apache/cassandra-java-driver/blob/16260261d3df50fcf24fac1fc2d37896c4a111bf/core/src/test/java/com/datastax/oss/driver/internal/core/loadbalancing/BasicLoadBalancingPolicyDcFailoverTest.java#L67-L70



##########
core/src/test/java/com/datastax/oss/driver/internal/core/loadbalancing/BasicLoadBalancingPolicyRemoteDcTest.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.datastax.oss.driver.internal.core.loadbalancing;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.BDDMockito.then;
+import static org.mockito.Mockito.atLeast;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.datastax.oss.driver.api.core.config.DefaultDriverOption;
+import com.datastax.oss.driver.api.core.config.DriverExecutionProfile;
+import com.datastax.oss.driver.api.core.metadata.Node;
+import com.datastax.oss.driver.internal.core.metadata.DefaultNode;
+import com.datastax.oss.driver.shaded.guava.common.collect.ImmutableMap;
+import com.datastax.oss.driver.shaded.guava.common.collect.ImmutableSet;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.junit.Test;
+import org.mockito.Mock;
+
+public class BasicLoadBalancingPolicyRemoteDcTest extends 
BasicLoadBalancingPolicyQueryPlanTest {
+
+  @Mock protected DefaultNode node6;
+  @Mock protected DefaultNode node7;
+  @Mock protected DefaultNode node8;
+  @Mock protected DefaultNode node9;
+  @Mock protected DefaultNode node10;
+  @Mock protected DefaultNode node11;
+  @Mock protected DefaultNode node12;
+
+  @Override
+  @Test
+  public void
+      
should_use_round_robin_when_token_map_returns_no_replicas_using_request_keyspace_and_routing_key()
 {
+    when(request.getRoutingKeyspace()).thenReturn(KEYSPACE);
+    when(request.getRoutingKey()).thenReturn(ROUTING_KEY);
+    when(tokenMap.getReplicas(KEYSPACE, 
ROUTING_KEY)).thenReturn(Collections.emptySet());
+
+    assertRoundRobinQueryPlans();
+
+    then(tokenMap).should(atLeast(1)).getReplicas(KEYSPACE, ROUTING_KEY);
+  }
+
+  @Override
+  @Test
+  public void should_prioritize_and_shuffle_replicas() {
+    when(request.getRoutingKeyspace()).thenReturn(KEYSPACE);
+    when(request.getRoutingKey()).thenReturn(ROUTING_KEY);
+    when(tokenMap.getReplicas(KEYSPACE, 
ROUTING_KEY)).thenReturn(ImmutableSet.of(node3, node5));
+
+    assertThat(policy.newQueryPlan(request, session))
+        .containsExactly(node3, node5, node1, node2, node4, node9, node10, 
node6, node7, node12);
+    assertThat(policy.newQueryPlan(request, session))
+        .containsExactly(node3, node5, node2, node4, node1, node9, node10, 
node6, node7, node12);
+    assertThat(policy.newQueryPlan(request, session))
+        .containsExactly(node3, node5, node4, node1, node2, node9, node10, 
node6, node7, node12);
+
+    verify(policy, times(3)).shuffleHead(any(), eq(2));
+    // No power of two choices with only two replicas
+    verify(session, never()).getPools();
+  }
+
+  @Override
+  @Test
+  public void should_prioritize_single_replica() {
+    when(request.getRoutingKeyspace()).thenReturn(KEYSPACE);
+    when(request.getRoutingKey()).thenReturn(ROUTING_KEY);
+    when(tokenMap.getReplicas(KEYSPACE, 
ROUTING_KEY)).thenReturn(ImmutableSet.of(node3));
+
+    // node3 always first, round-robin on the rest
+    assertThat(policy.newQueryPlan(request, session))
+        .containsExactly(node3, node1, node2, node4, node5, node9, node10, 
node6, node7, node12);
+    assertThat(policy.newQueryPlan(request, session))
+        .containsExactly(node3, node2, node4, node5, node1, node9, node10, 
node6, node7, node12);
+    assertThat(policy.newQueryPlan(request, session))
+        .containsExactly(node3, node4, node5, node1, node2, node9, node10, 
node6, node7, node12);
+    assertThat(policy.newQueryPlan(request, session))
+        .containsExactly(node3, node5, node1, node2, node4, node9, node10, 
node6, node7, node12);
+
+    // Should not shuffle replicas since there is only one
+    verify(policy, never()).shuffleHead(any(), anyInt());
+  }
+
+  @Override
+  protected void assertRoundRobinQueryPlans() {
+    for (int i = 0; i < 3; i++) {
+      assertThat(policy.newQueryPlan(request, session))
+          .containsExactly(node1, node2, node3, node4, node5, node9, node10, 
node6, node7, node12);
+      assertThat(policy.newQueryPlan(request, session))
+          .containsExactly(node2, node3, node4, node5, node1, node9, node10, 
node6, node7, node12);
+      assertThat(policy.newQueryPlan(request, session))
+          .containsExactly(node3, node4, node5, node1, node2, node9, node10, 
node6, node7, node12);
+      assertThat(policy.newQueryPlan(request, session))
+          .containsExactly(node4, node5, node1, node2, node3, node9, node10, 
node6, node7, node12);
+      assertThat(policy.newQueryPlan(request, session))
+          .containsExactly(node5, node1, node2, node3, node4, node9, node10, 
node6, node7, node12);
+    }
+  }
+
+  @Override
+  protected BasicLoadBalancingPolicy createAndInitPolicy() {
+    when(node4.getDatacenter()).thenReturn("dc1");
+    when(node5.getDatacenter()).thenReturn("dc1");
+    when(node6.getDatacenter()).thenReturn("dc2");
+    when(node7.getDatacenter()).thenReturn("dc2");
+    when(node8.getDatacenter()).thenReturn("dc2");
+    when(node9.getDatacenter()).thenReturn("dc3");
+    when(node10.getDatacenter()).thenReturn("dc3");
+    when(node11.getDatacenter()).thenReturn("dc3");
+    when(node12.getDatacenter()).thenReturn("dc4");
+
+    // Accept 2 nodes per remote DC
+    when(defaultProfile.getInt(
+            
DefaultDriverOption.LOAD_BALANCING_DC_FAILOVER_MAX_NODES_PER_REMOTE_DC))
+        .thenReturn(2);
+    when(defaultProfile.getBoolean(
+            
DefaultDriverOption.LOAD_BALANCING_DC_FAILOVER_ALLOW_FOR_LOCAL_CONSISTENCY_LEVELS))
+        .thenReturn(false);
+
+    final List<String> preferredDcs = new ArrayList<>();
+    preferredDcs.add("dc3");
+    preferredDcs.add("dc2");
+    when(defaultProfile.getStringList(
+            
DefaultDriverOption.LOAD_BALANCING_DC_FAILOVER_PREFERRED_REMOTE_DCS, new 
ArrayList<>()))
+        .thenReturn(preferredDcs);

Review Comment:
   ```suggestion
       when(defaultProfile.getStringList(
               
DefaultDriverOption.LOAD_BALANCING_DC_FAILOVER_PREFERRED_REMOTE_DCS, new 
ArrayList<>()))
           .thenReturn(ImmutableList.of("dc3","dc2"));
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to