Skip to content

Commit 0780c7c

Browse files
committed
Correct node config overloading and failover from one protocol to another
1 parent 9326785 commit 0780c7c

File tree

11 files changed

+371
-211
lines changed

11 files changed

+371
-211
lines changed

clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseClientBuilder.java

+87-46
Original file line numberDiff line numberDiff line change
@@ -3,6 +3,7 @@
33
import java.io.Serializable;
44
import java.io.UncheckedIOException;
55
import java.net.ConnectException;
6+
import java.util.Collections;
67
import java.util.HashMap;
78
import java.util.Map;
89
import java.util.Objects;
@@ -32,25 +33,33 @@ public class ClickHouseClientBuilder {
3233
* Dummy client which is only used by {@link Agent}.
3334
*/
3435
static class DummyClient implements ClickHouseClient {
35-
static final ClickHouseConfig CONFIG = new ClickHouseConfig();
36-
static final DummyClient INSTANCE = new DummyClient();
36+
static final ClickHouseConfig DEFAULT_CONFIG = new ClickHouseConfig();
37+
38+
private final ClickHouseConfig config;
39+
40+
DummyClient() {
41+
this(null);
42+
}
43+
44+
DummyClient(ClickHouseConfig config) {
45+
this.config = config != null ? config : DEFAULT_CONFIG;
46+
}
3747

3848
@Override
3949
public boolean accept(ClickHouseProtocol protocol) {
40-
return true;
50+
return false;
4151
}
4252

4353
@Override
4454
public CompletableFuture<ClickHouseResponse> execute(ClickHouseRequest<?> request) {
4555
CompletableFuture<ClickHouseResponse> future = new CompletableFuture<>();
46-
future.completeExceptionally(
47-
new ConnectException("No client available for connecting to: " + request.getServer()));
56+
future.completeExceptionally(new ConnectException("No client available"));
4857
return future;
4958
}
5059

5160
@Override
5261
public ClickHouseConfig getConfig() {
53-
return CONFIG;
62+
return config;
5463
}
5564

5665
@Override
@@ -73,8 +82,8 @@ static final class Agent implements ClickHouseClient {
7382

7483
private final AtomicReference<ClickHouseClient> client;
7584

76-
Agent(ClickHouseClient client) {
77-
this.client = new AtomicReference<>(client != null ? client : DummyClient.INSTANCE);
85+
Agent(ClickHouseClient client, ClickHouseConfig config) {
86+
this.client = new AtomicReference<>(client != null ? client : new DummyClient(config));
7887
}
7988

8089
ClickHouseClient getClient() {
@@ -95,25 +104,27 @@ boolean changeClient(ClickHouseClient currentClient, ClickHouseClient newClient)
95104
return changed;
96105
}
97106

98-
ClickHouseResponse failover(ClickHouseRequest<?> sealedRequest, Throwable cause, int times) {
107+
ClickHouseResponse failover(ClickHouseRequest<?> sealedRequest, ClickHouseException exception, int times) {
99108
for (int i = 1; i <= times; i++) {
100-
log.debug("Failover %d of %d due to: %s", i, times, cause.getMessage());
109+
log.debug("Failover %d of %d due to: %s", i, times, exception.getCause(), null);
101110
ClickHouseNode current = sealedRequest.getServer();
102111
ClickHouseNodeManager manager = current.manager.get();
103112
if (manager == null) {
104113
break;
105114
}
106-
ClickHouseNode next = manager.suggestNode(current, cause);
115+
ClickHouseNode next = manager.suggestNode(current, exception);
107116
if (next == current) {
117+
log.debug("Cancel failover for same node returned from %s", manager.getPolicy());
108118
break;
109119
}
110120
current.update(Status.FAULTY);
111121
next = sealedRequest.changeServer(current, next);
112122
if (next == current) {
123+
log.debug("Cancel failover for no alternative of %s", current);
113124
break;
114125
}
115126

116-
log.info("Switching node from %s to %s due to: %s", current, next, cause.getMessage());
127+
log.info("Switching node from %s to %s due to: %s", current, next, exception.getCause(), null);
117128
final ClickHouseProtocol protocol = next.getProtocol();
118129
final ClickHouseClient currentClient = client.get();
119130
if (!currentClient.accept(protocol)) {
@@ -123,51 +134,50 @@ ClickHouseResponse failover(ClickHouseRequest<?> sealedRequest, Throwable cause,
123134
.config(new ClickHouseConfig(currentClient.getConfig(), next.config))
124135
.nodeSelector(ClickHouseNodeSelector.of(protocol)).build();
125136
} catch (Exception e) {
126-
cause = e;
127-
continue;
137+
exception = ClickHouseException.of(new ConnectException("No client available for " + next),
138+
sealedRequest.getServer());
128139
} finally {
129140
if (newClient != null) {
130141
boolean changed = changeClient(currentClient, newClient);
131-
log.debug("Switching client from %s to %s: %s", currentClient, newClient, changed);
142+
log.info("Switching client from %s to %s: %s", currentClient, newClient, changed);
132143
if (changed) {
133144
sealedRequest.resetCache();
134145
}
135146
}
136147
}
148+
149+
if (newClient == null) {
150+
continue;
151+
}
137152
}
138153

139154
try {
140155
return sendOnce(sealedRequest);
141156
} catch (Exception exp) {
142-
cause = exp.getCause();
143-
if (cause == null) {
144-
cause = exp;
145-
}
157+
exception = ClickHouseException.of(exp.getCause() != null ? exp.getCause() : exp,
158+
sealedRequest.getServer());
146159
}
147160
}
148161

149-
throw new CompletionException(cause);
162+
throw new CompletionException(exception);
150163
}
151164

152-
ClickHouseResponse retry(ClickHouseRequest<?> sealedRequest, Throwable cause, int times) {
165+
ClickHouseResponse retry(ClickHouseRequest<?> sealedRequest, ClickHouseException exception, int times) {
153166
for (int i = 1; i <= times; i++) {
154-
log.debug("Retry %d of %d due to: %s", i, times, cause.getMessage());
167+
log.debug("Retry %d of %d due to: %s", i, times, exception.getMessage());
155168
// TODO retry idempotent query
156-
if (cause instanceof ClickHouseException
157-
&& ((ClickHouseException) cause).getErrorCode() == ClickHouseException.ERROR_NETWORK) {
169+
if (exception.getErrorCode() == ClickHouseException.ERROR_NETWORK) {
158170
log.info("Retry request on %s due to connection issue", sealedRequest.getServer());
159171
try {
160172
return sendOnce(sealedRequest);
161173
} catch (Exception exp) {
162-
cause = exp.getCause();
163-
if (cause == null) {
164-
cause = exp;
165-
}
174+
exception = ClickHouseException.of(exp.getCause() != null ? exp.getCause() : exp,
175+
sealedRequest.getServer());
166176
}
167177
}
168178
}
169179

170-
throw new CompletionException(cause);
180+
throw new CompletionException(exception);
171181
}
172182

173183
ClickHouseResponse handle(ClickHouseRequest<?> sealedRequest, Throwable cause) {
@@ -176,16 +186,18 @@ ClickHouseResponse handle(ClickHouseRequest<?> sealedRequest, Throwable cause) {
176186
cause = ((UncheckedIOException) cause).getCause();
177187
}
178188

189+
log.debug("Handling %s(failover=%d, retry=%d)", cause, sealedRequest.getConfig().getFailover(),
190+
sealedRequest.getConfig().getRetry());
179191
try {
180192
int times = sealedRequest.getConfig().getFailover();
181193
if (times > 0) {
182-
return failover(sealedRequest, cause, times);
194+
return failover(sealedRequest, ClickHouseException.of(cause, sealedRequest.getServer()), times);
183195
}
184196

185197
// different from failover: 1) retry on the same node; 2) never retry on timeout
186198
times = sealedRequest.getConfig().getRetry();
187199
if (times > 0) {
188-
return retry(sealedRequest, cause, times);
200+
return retry(sealedRequest, ClickHouseException.of(cause, sealedRequest.getServer()), times);
189201
}
190202

191203
throw new CompletionException(cause);
@@ -210,8 +222,8 @@ ClickHouseResponse sendOnce(ClickHouseRequest<?> sealedRequest) {
210222
ClickHouseResponse send(ClickHouseRequest<?> sealedRequest) {
211223
try {
212224
return sendOnce(sealedRequest);
213-
} catch (CompletionException e) {
214-
return handle(sealedRequest, e.getCause());
225+
} catch (Exception e) {
226+
return handle(sealedRequest, e.getCause() != null ? e.getCause() : e);
215227
}
216228
}
217229

@@ -238,9 +250,32 @@ public boolean ping(ClickHouseNode server, int timeout) {
238250
@Override
239251
public CompletableFuture<ClickHouseResponse> execute(ClickHouseRequest<?> request) {
240252
final ClickHouseRequest<?> sealedRequest = request.seal();
253+
final ClickHouseNode server = sealedRequest.getServer();
254+
final ClickHouseProtocol protocol = server.getProtocol();
255+
final ClickHouseClient currentClient = client.get();
256+
if (!currentClient.accept(protocol)) {
257+
ClickHouseClient newClient = null;
258+
try {
259+
newClient = ClickHouseClient.builder().agent(false)
260+
.config(new ClickHouseConfig(currentClient.getConfig(), server.config))
261+
.nodeSelector(ClickHouseNodeSelector.of(protocol)).build();
262+
} catch (IllegalStateException e) {
263+
// let it fail on execution phase
264+
log.debug("Failed to find client for %s", server);
265+
} finally {
266+
if (newClient != null) {
267+
boolean changed = changeClient(currentClient, newClient);
268+
log.debug("Switching client from %s to %s: %s", currentClient, newClient, changed);
269+
if (changed) {
270+
sealedRequest.resetCache();
271+
}
272+
}
273+
}
274+
}
241275
return sealedRequest.getConfig().isAsync()
242276
? getClient().execute(sealedRequest)
243-
.handle((r, t) -> t == null ? r : handle(sealedRequest, t.getCause()))
277+
.handle((r, t) -> t == null ? r
278+
: handle(sealedRequest, t.getCause() != null ? t.getCause() : t))
244279
: CompletableFuture.completedFuture(send(sealedRequest));
245280
}
246281

@@ -349,26 +384,28 @@ public ClickHouseConfig getConfig() {
349384
public ClickHouseClient build() {
350385
ClickHouseClient client = null;
351386

352-
boolean noSelector = nodeSelector == null || nodeSelector == ClickHouseNodeSelector.EMPTY;
353-
int counter = 0;
354387
ClickHouseConfig conf = getConfig();
355-
for (ClickHouseClient c : loadClients()) {
356-
c.init(conf);
388+
int counter = 0;
389+
if (nodeSelector != null) {
390+
for (ClickHouseClient c : loadClients()) {
391+
c.init(conf);
357392

358-
counter++;
359-
if (noSelector || nodeSelector.match(c)) {
360-
client = c;
361-
break;
393+
counter++;
394+
if (nodeSelector == ClickHouseNodeSelector.EMPTY || nodeSelector.match(c)) {
395+
client = c;
396+
break;
397+
}
362398
}
363399
}
364400

365-
if (client == null && !agent) {
401+
if (agent) {
402+
return new Agent(client, conf);
403+
} else if (client == null) {
366404
throw new IllegalStateException(
367405
ClickHouseUtils.format("No suitable ClickHouse client(out of %d) found in classpath for %s.",
368406
counter, nodeSelector));
369407
}
370-
371-
return agent ? new Agent(client) : client;
408+
return client;
372409
}
373410

374411
/**
@@ -485,7 +522,11 @@ public ClickHouseClientBuilder defaultCredentials(ClickHouseCredentials credenti
485522
*/
486523
public ClickHouseClientBuilder nodeSelector(ClickHouseNodeSelector nodeSelector) {
487524
if (!ClickHouseChecker.nonNull(nodeSelector, "nodeSelector").equals(this.nodeSelector)) {
488-
this.nodeSelector = nodeSelector;
525+
this.nodeSelector = (nodeSelector.getPreferredProtocols().isEmpty() || nodeSelector.getPreferredProtocols()
526+
.equals(Collections.singletonList(ClickHouseProtocol.ANY)))
527+
&& nodeSelector.getPreferredTags().isEmpty()
528+
? ClickHouseNodeSelector.EMPTY
529+
: nodeSelector;
489530
resetConfig();
490531
}
491532

clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseCluster.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -136,6 +136,6 @@ public String toString() {
136136
.append(checking.get()).append(", index=").append(index.get()).append(", lock=r")
137137
.append(lock.getReadHoldCount()).append('w').append(lock.getWriteHoldCount()).append(", nodes=")
138138
.append(nodes.size()).append(", faulty=").append(faultyNodes.size()).append(", policy=")
139-
.append(policy.getClass().getSimpleName()).append(']').toString();
139+
.append(policy.getClass().getSimpleName()).append("]@").append(hashCode()).toString();
140140
}
141141
}

0 commit comments

Comments
 (0)