Skip to content

Commit abe2811

Browse files
authored
4.x: Add optional fallback for ControlConnection#reconnect() (#341)
* Add `MockResolverIT#cannot_reconnect_with_resolved_socket()` Adds a method for testing the issues that surface after cluster replacements. Due to the variable, sometimes long runtime it is not added to any of the test groups. * Add optional fallback for `ControlConnection#reconnect()` Adds an experimental option to allow `ControlConnection` to try reconnecting to the original contact points held by `MetadataManager`, in case of getting empty query plan from the load balancing policy. In order to separate this logic from query plans of other queries `LoadBalancingPolicyWrapper#newControlReconnectionQueryPlan()` was introduced and is called during reconnection in place of `newQueryPlan()`.
1 parent d69bdd4 commit abe2811

File tree

9 files changed

+270
-7
lines changed

9 files changed

+270
-7
lines changed

core/src/main/java/com/datastax/oss/driver/api/core/config/DefaultDriverOption.java

+9
Original file line numberDiff line numberDiff line change
@@ -668,6 +668,15 @@ public enum DefaultDriverOption implements DriverOption {
668668
*/
669669
CONTROL_CONNECTION_AGREEMENT_WARN("advanced.control-connection.schema-agreement.warn-on-failure"),
670670

671+
/**
672+
* Whether to forcibly add original contact points held by MetadataManager to the reconnection
673+
* plan, in case there is no live nodes available according to LBP. Experimental.
674+
*
675+
* <p>Value-type: boolean
676+
*/
677+
CONTROL_CONNECTION_RECONNECT_CONTACT_POINTS(
678+
"advanced.control-connection.reconnection.fallback-to-original-contact-points"),
679+
671680
/**
672681
* Whether `Session.prepare` calls should be sent to all nodes in the cluster.
673682
*

core/src/main/java/com/datastax/oss/driver/api/core/config/OptionsMap.java

+1
Original file line numberDiff line numberDiff line change
@@ -360,6 +360,7 @@ protected static void fillWithDriverDefaults(OptionsMap map) {
360360
map.put(TypedDriverOption.CONTROL_CONNECTION_AGREEMENT_INTERVAL, Duration.ofMillis(200));
361361
map.put(TypedDriverOption.CONTROL_CONNECTION_AGREEMENT_TIMEOUT, Duration.ofSeconds(10));
362362
map.put(TypedDriverOption.CONTROL_CONNECTION_AGREEMENT_WARN, true);
363+
map.put(TypedDriverOption.CONTROL_CONNECTION_RECONNECT_CONTACT_POINTS, false);
363364
map.put(TypedDriverOption.PREPARE_ON_ALL_NODES, true);
364365
map.put(TypedDriverOption.REPREPARE_ENABLED, true);
365366
map.put(TypedDriverOption.REPREPARE_CHECK_SYSTEM_TABLE, false);

core/src/main/java/com/datastax/oss/driver/api/core/config/TypedDriverOption.java

+4
Original file line numberDiff line numberDiff line change
@@ -566,6 +566,10 @@ public String toString() {
566566
public static final TypedDriverOption<Boolean> CONTROL_CONNECTION_AGREEMENT_WARN =
567567
new TypedDriverOption<>(
568568
DefaultDriverOption.CONTROL_CONNECTION_AGREEMENT_WARN, GenericType.BOOLEAN);
569+
/** Whether to forcibly try original contacts if no live nodes are available */
570+
public static final TypedDriverOption<Boolean> CONTROL_CONNECTION_RECONNECT_CONTACT_POINTS =
571+
new TypedDriverOption<>(
572+
DefaultDriverOption.CONTROL_CONNECTION_RECONNECT_CONTACT_POINTS, GenericType.BOOLEAN);
569573
/** Whether `Session.prepare` calls should be sent to all nodes in the cluster. */
570574
public static final TypedDriverOption<Boolean> PREPARE_ON_ALL_NODES =
571575
new TypedDriverOption<>(DefaultDriverOption.PREPARE_ON_ALL_NODES, GenericType.BOOLEAN);

core/src/main/java/com/datastax/oss/driver/internal/core/control/ControlConnection.java

+3-2
Original file line numberDiff line numberDiff line change
@@ -300,7 +300,8 @@ private void init(
300300
.withOwnerLogPrefix(logPrefix + "|control")
301301
.build();
302302

303-
Queue<Node> nodes = context.getLoadBalancingPolicyWrapper().newQueryPlan();
303+
Queue<Node> nodes =
304+
context.getLoadBalancingPolicyWrapper().newControlReconnectionQueryPlan();
304305

305306
connect(
306307
nodes,
@@ -336,7 +337,7 @@ private void init(
336337

337338
private CompletionStage<Boolean> reconnect() {
338339
assert adminExecutor.inEventLoop();
339-
Queue<Node> nodes = context.getLoadBalancingPolicyWrapper().newQueryPlan();
340+
Queue<Node> nodes = context.getLoadBalancingPolicyWrapper().newControlReconnectionQueryPlan();
340341
CompletableFuture<Boolean> result = new CompletableFuture<>();
341342
connect(
342343
nodes,

core/src/main/java/com/datastax/oss/driver/internal/core/metadata/LoadBalancingPolicyWrapper.java

+20-2
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717
*/
1818
package com.datastax.oss.driver.internal.core.metadata;
1919

20+
import com.datastax.oss.driver.api.core.config.DefaultDriverOption;
2021
import com.datastax.oss.driver.api.core.config.DriverExecutionProfile;
2122
import com.datastax.oss.driver.api.core.loadbalancing.LoadBalancingPolicy;
2223
import com.datastax.oss.driver.api.core.loadbalancing.NodeDistance;
@@ -161,8 +162,25 @@ public Queue<Node> newQueryPlan(
161162
}
162163

163164
@NonNull
164-
public Queue<Node> newQueryPlan() {
165-
return newQueryPlan(null, DriverExecutionProfile.DEFAULT_NAME, null);
165+
public Queue<Node> newControlReconnectionQueryPlan() {
166+
// First try the original way
167+
Queue<Node> regularQueryPlan = newQueryPlan(null, DriverExecutionProfile.DEFAULT_NAME, null);
168+
if (!regularQueryPlan.isEmpty()) return regularQueryPlan;
169+
170+
if (context
171+
.getConfig()
172+
.getDefaultProfile()
173+
.getBoolean(DefaultDriverOption.CONTROL_CONNECTION_RECONNECT_CONTACT_POINTS)) {
174+
Set<DefaultNode> originalNodes = context.getMetadataManager().getContactPoints();
175+
List<Node> nodes = new ArrayList<>();
176+
for (DefaultNode node : originalNodes) {
177+
nodes.add(new DefaultNode(node.getEndPoint(), context));
178+
}
179+
Collections.shuffle(nodes);
180+
return new ConcurrentLinkedQueue<>(nodes);
181+
} else {
182+
return regularQueryPlan;
183+
}
166184
}
167185

168186
// when it comes in from the outside

core/src/main/resources/reference.conf

+11
Original file line numberDiff line numberDiff line change
@@ -2113,6 +2113,17 @@ datastax-java-driver {
21132113
# Overridable in a profile: no
21142114
warn-on-failure = true
21152115
}
2116+
2117+
reconnection {
2118+
# Whether to forcibly add original contact points held by MetadataManager to the reconnection plan,
2119+
# in case there is no live nodes available according to LBP.
2120+
# Experimental.
2121+
#
2122+
# Required: yes
2123+
# Modifiable at runtime: yes, the new value will be used for checks issued after the change.
2124+
# Overridable in a profile: no
2125+
fallback-to-original-contact-points = false
2126+
}
21162127
}
21172128

21182129
advanced.prepared-statements {

core/src/test/java/com/datastax/oss/driver/internal/core/control/ControlConnectionTestBase.java

+15-1
Original file line numberDiff line numberDiff line change
@@ -132,11 +132,25 @@ public void setup() {
132132
when(defaultProfile.getBoolean(DefaultDriverOption.CONNECTION_WARN_INIT_ERROR))
133133
.thenReturn(false);
134134

135+
when(context.getConfig()).thenReturn(config);
136+
when(config.getDefaultProfile()).thenReturn(defaultProfile);
137+
when(defaultProfile.getBoolean(DefaultDriverOption.CONTROL_CONNECTION_RECONNECT_CONTACT_POINTS))
138+
.thenReturn(false);
139+
135140
controlConnection = new ControlConnection(context);
136141
}
137142

138143
protected void mockQueryPlan(Node... nodes) {
139-
when(loadBalancingPolicyWrapper.newQueryPlan())
144+
when(loadBalancingPolicyWrapper.newControlReconnectionQueryPlan())
145+
.thenAnswer(
146+
i -> {
147+
ConcurrentLinkedQueue<Node> queryPlan = new ConcurrentLinkedQueue<>();
148+
for (Node node : nodes) {
149+
queryPlan.offer(node);
150+
}
151+
return queryPlan;
152+
});
153+
when(loadBalancingPolicyWrapper.newControlReconnectionQueryPlan())
140154
.thenAnswer(
141155
i -> {
142156
ConcurrentLinkedQueue<Node> queryPlan = new ConcurrentLinkedQueue<>();

core/src/test/java/com/datastax/oss/driver/internal/core/metadata/LoadBalancingPolicyWrapperTest.java

+31-2
Original file line numberDiff line numberDiff line change
@@ -118,10 +118,22 @@ public void setup() {
118118
policy3));
119119
}
120120

121+
@Test
122+
public void should_build_control_connection_query_plan_from_contact_points_before_init() {
123+
// When
124+
Queue<Node> queryPlan = wrapper.newControlReconnectionQueryPlan();
125+
126+
// Then
127+
for (LoadBalancingPolicy policy : ImmutableList.of(policy1, policy2, policy3)) {
128+
verify(policy, never()).newQueryPlan(null, null);
129+
}
130+
assertThat(queryPlan).hasSameElementsAs(contactPoints);
131+
}
132+
121133
@Test
122134
public void should_build_query_plan_from_contact_points_before_init() {
123135
// When
124-
Queue<Node> queryPlan = wrapper.newQueryPlan();
136+
Queue<Node> queryPlan = wrapper.newQueryPlan(null, DriverExecutionProfile.DEFAULT_NAME, null);
125137

126138
// Then
127139
for (LoadBalancingPolicy policy : ImmutableList.of(policy1, policy2, policy3)) {
@@ -139,7 +151,24 @@ public void should_fetch_query_plan_from_policy_after_init() {
139151
}
140152

141153
// When
142-
Queue<Node> queryPlan = wrapper.newQueryPlan();
154+
Queue<Node> queryPlan = wrapper.newControlReconnectionQueryPlan();
155+
156+
// Then
157+
// no-arg newQueryPlan() uses the default profile
158+
verify(policy1).newQueryPlan(null, null);
159+
assertThat(queryPlan).isEqualTo(defaultPolicyQueryPlan);
160+
}
161+
162+
@Test
163+
public void should_fetch_control_connection_query_plan_from_policy_after_init() {
164+
// Given
165+
wrapper.init();
166+
for (LoadBalancingPolicy policy : ImmutableList.of(policy1, policy2, policy3)) {
167+
verify(policy).init(anyMap(), any(DistanceReporter.class));
168+
}
169+
170+
// When
171+
Queue<Node> queryPlan = wrapper.newQueryPlan(null, DriverExecutionProfile.DEFAULT_NAME, null);
143172

144173
// Then
145174
// no-arg newQueryPlan() uses the default profile

integration-tests/src/test/java/com/datastax/oss/driver/core/resolver/MockResolverIT.java

+176
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@
2424
package com.datastax.oss.driver.core.resolver;
2525

2626
import static org.assertj.core.api.Assertions.assertThat;
27+
import static org.junit.Assert.assertFalse;
2728
import static org.junit.Assert.assertTrue;
2829
import static org.junit.Assert.fail;
2930

@@ -33,11 +34,14 @@
3334
import com.datastax.oss.driver.api.core.config.TypedDriverOption;
3435
import com.datastax.oss.driver.api.core.cql.ResultSet;
3536
import com.datastax.oss.driver.api.core.cql.Row;
37+
import com.datastax.oss.driver.api.core.cql.SimpleStatement;
38+
import com.datastax.oss.driver.api.core.cql.SimpleStatementBuilder;
3639
import com.datastax.oss.driver.api.core.metadata.Node;
3740
import com.datastax.oss.driver.api.testinfra.ccm.CcmBridge;
3841
import com.datastax.oss.driver.categories.IsolatedTests;
3942
import com.datastax.oss.driver.internal.core.config.typesafe.DefaultProgrammaticDriverConfigLoaderBuilder;
4043
import java.net.InetSocketAddress;
44+
import java.time.Duration;
4145
import java.util.Collection;
4246
import java.util.Collections;
4347
import java.util.Iterator;
@@ -237,4 +241,176 @@ public void run_replace_test_20_times() {
237241
replace_cluster_test();
238242
}
239243
}
244+
245+
// This is too long to run during CI, but is useful for manual investigations.
246+
@SuppressWarnings("unused")
247+
public void cannot_reconnect_with_resolved_socket() {
248+
DriverConfigLoader loader =
249+
new DefaultProgrammaticDriverConfigLoaderBuilder()
250+
.withBoolean(TypedDriverOption.RESOLVE_CONTACT_POINTS.getRawOption(), false)
251+
.withBoolean(TypedDriverOption.RECONNECT_ON_INIT.getRawOption(), true)
252+
.withStringList(
253+
TypedDriverOption.CONTACT_POINTS.getRawOption(),
254+
Collections.singletonList("test.cluster.fake:9042"))
255+
.build();
256+
257+
CqlSessionBuilder builder = new CqlSessionBuilder().withConfigLoader(loader);
258+
CqlSession session;
259+
Collection<Node> nodes;
260+
Set<Node> filteredNodes;
261+
try (CcmBridge ccmBridge = CcmBridge.builder().withNodes(3).withIpPrefix("127.0.1.").build()) {
262+
MultimapHostResolverProvider.removeResolverEntries("test.cluster.fake");
263+
MultimapHostResolverProvider.addResolverEntry(
264+
"test.cluster.fake", ccmBridge.getNodeIpAddress(1));
265+
MultimapHostResolverProvider.addResolverEntry(
266+
"test.cluster.fake", ccmBridge.getNodeIpAddress(2));
267+
MultimapHostResolverProvider.addResolverEntry(
268+
"test.cluster.fake", ccmBridge.getNodeIpAddress(3));
269+
ccmBridge.create();
270+
ccmBridge.start();
271+
session = builder.build();
272+
long endTime = System.currentTimeMillis() + CLUSTER_WAIT_SECONDS * 1000;
273+
while (System.currentTimeMillis() < endTime) {
274+
try {
275+
nodes = session.getMetadata().getNodes().values();
276+
int upNodes = 0;
277+
for (Node node : nodes) {
278+
if (node.getUpSinceMillis() > 0) {
279+
upNodes++;
280+
}
281+
}
282+
if (upNodes == 3) {
283+
break;
284+
}
285+
// session.refreshSchema();
286+
SimpleStatement statement =
287+
new SimpleStatementBuilder("SELECT * FROM system.local")
288+
.setTimeout(Duration.ofSeconds(3))
289+
.build();
290+
session.executeAsync(statement);
291+
Thread.sleep(3000);
292+
} catch (InterruptedException e) {
293+
break;
294+
}
295+
}
296+
ResultSet rs = session.execute("SELECT * FROM system.local");
297+
assertThat(rs).isNotNull();
298+
Row row = rs.one();
299+
assertThat(row).isNotNull();
300+
nodes = session.getMetadata().getNodes().values();
301+
assertThat(nodes).hasSize(3);
302+
Iterator<Node> iterator = nodes.iterator();
303+
while (iterator.hasNext()) {
304+
LOG.trace("Metadata node: " + iterator.next().toString());
305+
}
306+
filteredNodes =
307+
nodes.stream()
308+
.filter(x -> x.toString().contains("test.cluster.fake"))
309+
.collect(Collectors.toSet());
310+
assertThat(filteredNodes).hasSize(1);
311+
}
312+
int counter = 0;
313+
while (filteredNodes.size() == 1) {
314+
counter++;
315+
if (counter == 255) {
316+
LOG.error("Completed 254 runs. Breaking.");
317+
break;
318+
}
319+
LOG.warn(
320+
"Launching another cluster until we lose resolved socket from metadata (run {}).",
321+
counter);
322+
try (CcmBridge ccmBridge =
323+
CcmBridge.builder().withNodes(3).withIpPrefix("127.0." + counter + ".").build()) {
324+
MultimapHostResolverProvider.removeResolverEntries("test.cluster.fake");
325+
MultimapHostResolverProvider.addResolverEntry(
326+
"test.cluster.fake", ccmBridge.getNodeIpAddress(1));
327+
MultimapHostResolverProvider.addResolverEntry(
328+
"test.cluster.fake", ccmBridge.getNodeIpAddress(2));
329+
MultimapHostResolverProvider.addResolverEntry(
330+
"test.cluster.fake", ccmBridge.getNodeIpAddress(3));
331+
ccmBridge.create();
332+
ccmBridge.start();
333+
long endTime = System.currentTimeMillis() + CLUSTER_WAIT_SECONDS * 1000;
334+
while (System.currentTimeMillis() < endTime) {
335+
try {
336+
nodes = session.getMetadata().getNodes().values();
337+
int upNodes = 0;
338+
for (Node node : nodes) {
339+
if (node.getUpSinceMillis() > 0) {
340+
upNodes++;
341+
}
342+
}
343+
if (upNodes == 3) {
344+
break;
345+
}
346+
SimpleStatement statement =
347+
new SimpleStatementBuilder("SELECT * FROM system.local")
348+
.setTimeout(Duration.ofSeconds(3))
349+
.build();
350+
session.executeAsync(statement);
351+
Thread.sleep(3000);
352+
} catch (InterruptedException e) {
353+
break;
354+
}
355+
}
356+
nodes = session.getMetadata().getNodes().values();
357+
assertThat(nodes).hasSize(3);
358+
Iterator<Node> iterator = nodes.iterator();
359+
while (iterator.hasNext()) {
360+
LOG.trace("Metadata node: " + iterator.next().toString());
361+
}
362+
filteredNodes =
363+
nodes.stream()
364+
.filter(x -> x.toString().contains("test.cluster.fake"))
365+
.collect(Collectors.toSet());
366+
if (filteredNodes.size() > 1) {
367+
fail(
368+
"Somehow there is more than 1 node in metadata with unresolved hostname. This should not ever happen.");
369+
}
370+
}
371+
}
372+
Iterator<Node> iterator = nodes.iterator();
373+
while (iterator.hasNext()) {
374+
InetSocketAddress address = (InetSocketAddress) iterator.next().getEndPoint().resolve();
375+
assertFalse(address.isUnresolved());
376+
}
377+
try (CcmBridge ccmBridge = CcmBridge.builder().withNodes(3).withIpPrefix("127.1.1.").build()) {
378+
MultimapHostResolverProvider.removeResolverEntries("test.cluster.fake");
379+
MultimapHostResolverProvider.addResolverEntry(
380+
"test.cluster.fake", ccmBridge.getNodeIpAddress(1));
381+
MultimapHostResolverProvider.addResolverEntry(
382+
"test.cluster.fake", ccmBridge.getNodeIpAddress(2));
383+
MultimapHostResolverProvider.addResolverEntry(
384+
"test.cluster.fake", ccmBridge.getNodeIpAddress(3));
385+
// Now the driver should fail to reconnect since unresolved hostname is gone.
386+
ccmBridge.create();
387+
ccmBridge.start();
388+
long endTime = System.currentTimeMillis() + CLUSTER_WAIT_SECONDS * 1000;
389+
while (System.currentTimeMillis() < endTime) {
390+
try {
391+
nodes = session.getMetadata().getNodes().values();
392+
int upNodes = 0;
393+
for (Node node : nodes) {
394+
if (node.getUpSinceMillis() > 0) {
395+
upNodes++;
396+
}
397+
}
398+
if (upNodes == 3) {
399+
break;
400+
}
401+
// session.refreshSchema();
402+
SimpleStatement statement =
403+
new SimpleStatementBuilder("SELECT * FROM system.local")
404+
.setTimeout(Duration.ofSeconds(3))
405+
.build();
406+
session.executeAsync(statement);
407+
Thread.sleep(3000);
408+
} catch (InterruptedException e) {
409+
break;
410+
}
411+
}
412+
session.execute("SELECT * FROM system.local");
413+
}
414+
session.close();
415+
}
240416
}

0 commit comments

Comments
 (0)