Skip to content

Commit 845e76d

Browse files
committed
Merge branch 'main' into fix_stale_connection_issue
2 parents 41b3e29 + 66f8bce commit 845e76d

File tree

22 files changed

+1778
-137
lines changed

22 files changed

+1778
-137
lines changed

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -55,7 +55,7 @@ public class ClickHouseRequest<SelfT extends ClickHouseRequest<SelfT>> implement
5555

5656
static {
5757
Set<String> set = new HashSet<>();
58-
set.add("query_id");
58+
set.add(ClickHouseClientOption.QUERY_ID.getKey());
5959
set.add(ClickHouseClientOption.SESSION_ID.getKey());
6060
set.add(ClickHouseClientOption.SESSION_CHECK.getKey());
6161
set.add(ClickHouseClientOption.SESSION_TIMEOUT.getKey());

clickhouse-client/src/main/java/com/clickhouse/client/config/ClickHouseClientOption.java

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -420,7 +420,12 @@ public enum ClickHouseClientOption implements ClickHouseOption {
420420
* false.
421421
*/
422422
USE_TIME_ZONE("use_time_zone", "", "Time zone of all DateTime* values. "
423-
+ "Only used when use_server_time_zone is false. Empty value means client time zone.");
423+
+ "Only used when use_server_time_zone is false. Empty value means client time zone."),
424+
425+
/**
426+
* Query ID to be attached to an operation
427+
*/
428+
QUERY_ID("query_id", "", "Query id");
424429

425430
private final String key;
426431
private final Serializable defaultValue;

clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseRequestTest.java

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -11,11 +11,17 @@
1111
import java.util.ArrayList;
1212
import java.util.Collections;
1313
import java.util.HashMap;
14+
import java.util.HashSet;
1415
import java.util.List;
1516
import java.util.Map;
1617
import java.util.Optional;
1718
import java.util.Properties;
19+
import java.util.Set;
1820
import java.util.UUID;
21+
import java.util.concurrent.ExecutorService;
22+
import java.util.concurrent.Executors;
23+
import java.util.concurrent.ScheduledExecutorService;
24+
import java.util.concurrent.TimeUnit;
1925

2026
import org.testng.Assert;
2127
import org.testng.annotations.Test;

clickhouse-http-client/src/main/java/com/clickhouse/client/http/ApacheHttpConnectionImpl.java

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -399,8 +399,7 @@ public HttpConnectionManager(Registry<ConnectionSocketFactory> socketFactory, Cl
399399

400400
ConnectionConfig connConfig = ConnectionConfig.custom()
401401
.setConnectTimeout(Timeout.of(config.getConnectionTimeout(), TimeUnit.MILLISECONDS))
402-
.setTimeToLive(20, TimeUnit.SECONDS)
403-
.setValidateAfterInactivity(10, TimeUnit.SECONDS)
402+
.setValidateAfterInactivity(config.getLongOption(ClickHouseHttpOption.AHC_VALIDATE_AFTER_INACTIVITY), TimeUnit.MILLISECONDS)
404403
.build();
405404
setDefaultConnectionConfig(connConfig);
406405

clickhouse-http-client/src/main/java/com/clickhouse/client/http/config/ClickHouseHttpOption.java

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -67,6 +67,14 @@ public enum ClickHouseHttpOption implements ClickHouseOption {
6767
REMEMBER_LAST_SET_ROLES("remember_last_set_roles", false,
6868
"Whether to remember last set role and send them in every next requests as query parameters."),
6969

70+
/**
71+
* The time in milliseconds after which the connection is validated after inactivity.
72+
* Default value is 5000 ms. If set to negative value, the connection is never validated.
73+
* It is used only for Apache Http Client connection provider.
74+
*/
75+
AHC_VALIDATE_AFTER_INACTIVITY("ahc_validate_after_inactivity", 5000L,
76+
"The time in milliseconds after which the connection is validated after inactivity."),
77+
7078
/**
7179
* Whether to retry on failure with AsyncHttpClient. Failure includes some 'critical' IO exceptions:
7280
* <ul>

clickhouse-http-client/src/main/java11/com/clickhouse/client/http/HttpClientConnectionImpl.java

Lines changed: 4 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -81,7 +81,6 @@ public List<Proxy> select(URI uri) {
8181

8282
private static final String USER_AGENT = ClickHouseClientOption.buildUserAgent(null, "HttpClient");
8383

84-
private final AtomicBoolean busy;
8584
private final HttpClient httpClient;
8685
private final HttpRequest pingRequest;
8786

@@ -197,15 +196,13 @@ protected HttpClientConnectionImpl(ClickHouseNode server, ClickHouseRequest<?> r
197196
builder.sslContext(ClickHouseSslContextProvider.getProvider().getSslContext(SSLContext.class, config)
198197
.orElse(null));
199198
}
200-
201-
busy = new AtomicBoolean(false);
202199
httpClient = builder.build();
203200
pingRequest = newRequest(getBaseUrl() + "ping");
204201
}
205202

206203
@Override
207204
protected boolean isReusable() {
208-
return busy.get();
205+
return true; // httpClient is stateless and can be reused
209206
}
210207

211208
private CompletableFuture<HttpResponse<InputStream>> postRequest(HttpRequest request) {
@@ -218,7 +215,7 @@ private CompletableFuture<HttpResponse<InputStream>> postRequest(HttpRequest req
218215
private ClickHouseHttpResponse postStream(ClickHouseConfig config, HttpRequest.Builder reqBuilder, byte[] boundary,
219216
String sql, ClickHouseInputStream data, List<ClickHouseExternalTable> tables, ClickHouseOutputStream output,
220217
Runnable postAction) throws IOException {
221-
try {
218+
222219
ClickHousePipedOutputStream stream = ClickHouseDataStreamFactory.getInstance()
223220
.createPipedOutputStream(config);
224221
reqBuilder.POST(HttpRequest.BodyPublishers.ofInputStream(stream::getInputStream));
@@ -243,14 +240,11 @@ private ClickHouseHttpResponse postStream(ClickHouseConfig config, HttpRequest.B
243240
}
244241

245242
return buildResponse(config, r, output, postAction);
246-
} finally {
247-
busy.set(false);
248-
}
249243
}
250244

251245
private ClickHouseHttpResponse postString(ClickHouseConfig config, HttpRequest.Builder reqBuilder, String sql,
252246
ClickHouseOutputStream output, Runnable postAction) throws IOException {
253-
try {
247+
254248
reqBuilder.POST(HttpRequest.BodyPublishers.ofString(sql));
255249
HttpResponse<InputStream> r;
256250
try {
@@ -267,9 +261,6 @@ private ClickHouseHttpResponse postString(ClickHouseConfig config, HttpRequest.B
267261
}
268262
}
269263
return buildResponse(config, r, output, postAction);
270-
} finally {
271-
busy.set(false);
272-
}
273264
}
274265

275266
@Override
@@ -281,9 +272,7 @@ protected final String getDefaultUserAgent() {
281272
protected ClickHouseHttpResponse post(ClickHouseConfig config, String sql, ClickHouseInputStream data,
282273
List<ClickHouseExternalTable> tables, ClickHouseOutputStream output, String url,
283274
Map<String, String> headers, Runnable postAction) throws IOException {
284-
if (!busy.compareAndSet(false, true)) {
285-
throw new ConnectException("Connection is busy");
286-
}
275+
287276
ClickHouseConfig c = config == null ? this.config : config;
288277
HttpRequest.Builder reqBuilder = HttpRequest.newBuilder()
289278
.uri(URI.create(ClickHouseChecker.isNullOrEmpty(url) ? this.url : url))

clickhouse-http-client/src/test/java/com/clickhouse/client/http/ApacheHttpConnectionImplTest.java

Lines changed: 63 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -16,6 +16,7 @@
1616

1717
import java.io.IOException;
1818
import java.io.Serializable;
19+
import java.net.ConnectException;
1920
import java.util.Collections;
2021
import java.util.HashMap;
2122
import java.util.List;
@@ -26,10 +27,13 @@
2627
import com.github.tomakehurst.wiremock.WireMockServer;
2728
import com.github.tomakehurst.wiremock.admin.model.ScenarioState;
2829
import com.github.tomakehurst.wiremock.client.WireMock;
30+
import com.github.tomakehurst.wiremock.core.WireMockConfiguration;
2931
import com.github.tomakehurst.wiremock.http.Fault;
3032
import com.github.tomakehurst.wiremock.stubbing.Scenario;
3133
import com.github.tomakehurst.wiremock.stubbing.StubMapping;
34+
import com.github.tomakehurst.wiremock.stubbing.Scenario;
3235
import org.apache.hc.client5.http.socket.PlainConnectionSocketFactory;
36+
import org.apache.hc.core5.http.NoHttpResponseException;
3337
import org.apache.hc.core5.http.HttpStatus;
3438
import org.testng.Assert;
3539
import org.testng.annotations.DataProvider;
@@ -207,4 +211,63 @@ private static StubMapping[] retryOnFailureProvider() {
207211
.build()
208212
};
209213
}
214+
215+
@Test(groups = {"unit"}, dataProvider = "validationTimeoutProvider")
216+
public void testNoHttpResponseExceptionWithValidation(long validationTimeout) {
217+
218+
faultyServer = new WireMockServer(9090);
219+
faultyServer.start();
220+
221+
faultyServer.addStubMapping(WireMock.post(WireMock.anyUrl())
222+
.inScenario("validateOnStaleConnection")
223+
.withRequestBody(WireMock.equalTo("SELECT 100"))
224+
.willReturn(WireMock.aResponse()
225+
.withHeader("X-ClickHouse-Summary",
226+
"{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}"))
227+
.build());
228+
229+
230+
ClickHouseHttpClient httpClient = new ClickHouseHttpClient();
231+
Map<ClickHouseOption, Serializable> options = new HashMap<>();
232+
options.put(ClickHouseHttpOption.AHC_VALIDATE_AFTER_INACTIVITY, validationTimeout);
233+
options.put(ClickHouseHttpOption.MAX_OPEN_CONNECTIONS, 1);
234+
ClickHouseConfig config = new ClickHouseConfig(options);
235+
httpClient.init(config);
236+
ClickHouseRequest request = httpClient.read("http://localhost:9090/").query("SELECT 100");
237+
238+
Runnable powerBlink = () -> {
239+
try {
240+
Thread.sleep(100);
241+
faultyServer.stop();
242+
Thread.sleep(50);
243+
faultyServer.start();
244+
} catch (InterruptedException e) {
245+
Assert.fail("Unexpected exception", e);
246+
}
247+
};
248+
try {
249+
ClickHouseResponse response = httpClient.executeAndWait(request);
250+
Assert.assertEquals(response.getSummary().getReadRows(), 1);
251+
response.close();
252+
new Thread(powerBlink).start();
253+
Thread.sleep(200);
254+
response = httpClient.executeAndWait(request);
255+
Assert.assertEquals(response.getSummary().getReadRows(), 1);
256+
response.close();
257+
} catch (Exception e) {
258+
if (validationTimeout < 0) {
259+
Assert.assertTrue(e instanceof ClickHouseException);
260+
Assert.assertTrue(e.getCause() instanceof ConnectException);
261+
} else {
262+
Assert.fail("Unexpected exception", e);
263+
}
264+
} finally {
265+
faultyServer.stop();
266+
}
267+
}
268+
269+
@DataProvider(name = "validationTimeoutProvider")
270+
public static Object[] validationTimeoutProvider() {
271+
return new Long[] {-1L , 100L };
272+
}
210273
}

clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseStatementImpl.java

Lines changed: 78 additions & 33 deletions
Original file line numberDiff line numberDiff line change
@@ -1,30 +1,15 @@
11
package com.clickhouse.jdbc.internal;
22

3-
import java.io.IOException;
4-
import java.io.InputStream;
5-
import java.io.OutputStream;
6-
import java.io.Serializable;
7-
import java.nio.file.Path;
8-
import java.sql.ResultSet;
9-
import java.sql.SQLException;
10-
import java.sql.SQLFeatureNotSupportedException;
11-
import java.sql.SQLWarning;
12-
import java.sql.Statement;
13-
import java.util.*;
14-
import java.util.Map.Entry;
15-
import java.util.concurrent.TimeUnit;
16-
import java.util.concurrent.TimeoutException;
17-
183
import com.clickhouse.client.ClickHouseClient;
194
import com.clickhouse.client.ClickHouseConfig;
205
import com.clickhouse.client.ClickHouseException;
216
import com.clickhouse.client.ClickHouseNode;
227
import com.clickhouse.client.ClickHouseRequest;
8+
import com.clickhouse.client.ClickHouseRequest.Mutation;
239
import com.clickhouse.client.ClickHouseResponse;
2410
import com.clickhouse.client.ClickHouseResponseSummary;
2511
import com.clickhouse.client.ClickHouseSimpleResponse;
2612
import com.clickhouse.client.ClickHouseTransaction;
27-
import com.clickhouse.client.ClickHouseRequest.Mutation;
2813
import com.clickhouse.client.config.ClickHouseClientOption;
2914
import com.clickhouse.client.config.ClickHouseDefaults;
3015
import com.clickhouse.config.ClickHouseConfigChangeListener;
@@ -42,16 +27,37 @@
4227
import com.clickhouse.data.ClickHouseOutputStream;
4328
import com.clickhouse.data.ClickHouseUtils;
4429
import com.clickhouse.data.ClickHouseValues;
45-
import com.clickhouse.logging.Logger;
46-
import com.clickhouse.logging.LoggerFactory;
4730
import com.clickhouse.jdbc.ClickHouseConnection;
4831
import com.clickhouse.jdbc.ClickHouseResultSet;
4932
import com.clickhouse.jdbc.ClickHouseStatement;
5033
import com.clickhouse.jdbc.JdbcTypeMapping;
51-
import com.clickhouse.jdbc.SqlExceptionUtils;
5234
import com.clickhouse.jdbc.JdbcWrapper;
35+
import com.clickhouse.jdbc.SqlExceptionUtils;
5336
import com.clickhouse.jdbc.parser.ClickHouseSqlStatement;
5437
import com.clickhouse.jdbc.parser.StatementType;
38+
import com.clickhouse.logging.Logger;
39+
import com.clickhouse.logging.LoggerFactory;
40+
41+
import java.io.IOException;
42+
import java.io.InputStream;
43+
import java.io.OutputStream;
44+
import java.io.Serializable;
45+
import java.nio.file.Path;
46+
import java.sql.ResultSet;
47+
import java.sql.SQLException;
48+
import java.sql.SQLFeatureNotSupportedException;
49+
import java.sql.SQLWarning;
50+
import java.sql.Statement;
51+
import java.util.ArrayList;
52+
import java.util.Arrays;
53+
import java.util.HashSet;
54+
import java.util.LinkedList;
55+
import java.util.List;
56+
import java.util.Map;
57+
import java.util.Map.Entry;
58+
import java.util.concurrent.TimeUnit;
59+
import java.util.concurrent.TimeoutException;
60+
import java.util.function.Function;
5561

5662
public class ClickHouseStatementImpl extends JdbcWrapper
5763
implements ClickHouseConfigChangeListener<ClickHouseRequest<?>>, ClickHouseStatement {
@@ -130,11 +136,9 @@ private ClickHouseResponse getLastResponse(Map<ClickHouseOption, Serializable> o
130136
request.set("_set_roles_stmt", requestRoles);
131137
}
132138

133-
request.query(stmt.getSQL(), queryId = connection.newQueryId());
134139
// TODO skip useless queries to reduce network calls and server load
135140
try {
136-
response = autoTx ? request.executeWithinTransaction(connection.isImplicitTransactionSupported())
137-
: request.transaction(connection.getTransaction()).executeAndWait();
141+
response = sendRequest(stmt.getSQL(), r -> r);
138142
} catch (Exception e) {
139143
throw SqlExceptionUtils.handle(e);
140144
} finally {
@@ -272,7 +276,6 @@ protected ClickHouseResponse processSqlStatement(ClickHouseSqlStatement stmt) th
272276

273277
protected ClickHouseResponse executeStatement(String stmt, Map<ClickHouseOption, Serializable> options,
274278
List<ClickHouseExternalTable> tables, Map<String, String> settings) throws SQLException {
275-
boolean autoTx = connection.getAutoCommit() && connection.isTransactionSupported();
276279
try {
277280
if (options != null) {
278281
request.options(options);
@@ -310,9 +313,8 @@ protected ClickHouseResponse executeStatement(String stmt, Map<ClickHouseOption,
310313
}
311314
request.external(list);
312315
}
313-
request.query(stmt, queryId = connection.newQueryId());
314-
return autoTx ? request.executeWithinTransaction(connection.isImplicitTransactionSupported())
315-
: request.transaction(connection.getTransaction()).executeAndWait();
316+
317+
return sendRequest(stmt, r -> r);
316318
} catch (Exception e) {
317319
throw SqlExceptionUtils.handle(e);
318320
}
@@ -328,18 +330,61 @@ protected ClickHouseResponse executeStatement(ClickHouseSqlStatement stmt,
328330
return executeStatement(stmt.getSQL(), options, tables, settings);
329331
}
330332

331-
protected int executeInsert(String sql, InputStream input) throws SQLException {
333+
private ClickHouseResponse sendRequest(String sql, Function<ClickHouseRequest<?>, ClickHouseRequest<?>> preSeal) throws SQLException {
332334
boolean autoTx = connection.getAutoCommit() && connection.isTransactionSupported();
333-
Mutation req = request.write().query(sql, queryId = connection.newQueryId()).data(input);
334-
try (ClickHouseResponse resp = autoTx
335-
? req.executeWithinTransaction(connection.isImplicitTransactionSupported())
336-
: req.transaction(connection.getTransaction()).executeAndWait();
337-
ResultSet rs = updateResult(new ClickHouseSqlStatement(sql, StatementType.INSERT), resp)) {
338-
// ignore
335+
336+
ClickHouseRequest<?> req;
337+
ClickHouseTransaction tx = null;
338+
synchronized (request) {
339+
try {
340+
if (autoTx) {
341+
if (connection.isImplicitTransactionSupported()) {
342+
request.set(ClickHouseTransaction.SETTING_IMPLICIT_TRANSACTION, 1).transaction(null);
343+
} else {
344+
tx = request.getManager().createImplicitTransaction(request);
345+
request.transaction(connection.getTransaction());
346+
}
347+
} else {
348+
try {
349+
request.transaction(connection.getTransaction());
350+
} catch (ClickHouseException e) {
351+
throw SqlExceptionUtils.handle(e);
352+
}
353+
}
354+
355+
req = preSeal.apply(request).query(sql, queryId = connection.newQueryId()).seal();
356+
} catch (Exception e) {
357+
throw SqlExceptionUtils.handle(e);
358+
}
359+
}
360+
361+
try {
362+
return req.executeAndWait();
339363
} catch (Exception e) {
364+
if (tx != null) {
365+
try {
366+
tx.rollback();
367+
} catch (Exception ex) {
368+
log.warn("Failed to rollback transaction", ex);
369+
}
370+
}
340371
throw SqlExceptionUtils.handle(e);
372+
} finally {
373+
try {
374+
request.transaction(null);
375+
} catch (Exception e) {
376+
throw SqlExceptionUtils.handle(ClickHouseException.of(e, req.getServer()));
377+
}
341378
}
379+
}
342380

381+
protected int executeInsert(String sql, InputStream input) throws SQLException {
382+
try (ClickHouseResponse response = sendRequest(sql, r -> r.write().data(input));
383+
ResultSet rs = updateResult(new ClickHouseSqlStatement(sql, StatementType.INSERT), response)) {
384+
// no more actions needed
385+
} catch (Exception e) {
386+
throw SqlExceptionUtils.handle(e);
387+
}
343388
return (int) currentUpdateCount;
344389
}
345390

0 commit comments

Comments
 (0)