From 787d858e477e13b3d91eea9b08cc2d2e530f5af6 Mon Sep 17 00:00:00 2001
From: David Karlsson <2795016+devdavidkarlsson@users.noreply.github.com>
Date: Wed, 11 Feb 2026 16:57:00 +0100
Subject: [PATCH 01/32] Add async HTTP support using Apache HttpClient 5 async
API (Phase 1)
Adds true async HTTP transport using CloseableHttpAsyncClient for
non-blocking I/O. This eliminates thread-per-request blocking and
provides better scalability under high concurrency.
Key changes:
- Add USE_ASYNC_HTTP config option (disabled by default)
- Add CloseableHttpAsyncClient with NIO-based connection pooling
- Add executeRequestAsync() returning CompletableFuture
- Add non-blocking retry logic using AsyncRetryMarker pattern
- Add QueryResponse constructor for buffered async responses
- Add comprehensive test suite (AsyncHttpClientTests)
Phase 1 limitations (documented with warnings):
- Response bodies are fully buffered in memory (<10MB recommended)
- Client request compression not supported in async mode
- Insert and multipart operations use sync fallback
---
.../com/clickhouse/client/api/Client.java | 97 +++-
.../client/api/ClientConfigProperties.java | 8 +
.../api/internal/HttpAPIClientHelper.java | 284 ++++++++++-
.../client/api/query/QueryResponse.java | 30 +-
.../client/AsyncHttpClientTests.java | 454 ++++++++++++++++++
5 files changed, 868 insertions(+), 5 deletions(-)
create mode 100644 client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/Client.java b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
index 9d983decb..aade43049 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/Client.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
@@ -41,6 +41,7 @@
import com.clickhouse.data.ClickHouseFormat;
import com.google.common.collect.ImmutableList;
import net.jpountz.lz4.LZ4Factory;
+import org.apache.hc.client5.http.async.methods.SimpleHttpResponse;
import org.apache.hc.core5.concurrent.DefaultThreadFactory;
import org.apache.hc.core5.http.ClassicHttpResponse;
import org.apache.hc.core5.http.Header;
@@ -77,6 +78,7 @@
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
import java.util.function.Supplier;
import java.util.stream.Collectors;
@@ -824,6 +826,33 @@ public Builder useAsyncRequests(boolean async) {
return this;
}
+ /**
+ * Enables true async HTTP transport using Apache HttpClient 5 async API.
+ * When enabled, HTTP requests use NIO-based non-blocking I/O instead of
+ * blocking thread-per-request model. This provides better scalability
+ * under high concurrency without requiring thread pools.
+ *
+ *
WARNING - Phase 1 Limitations:
+ *
+ * - MEMORY: Response bodies are fully buffered in memory. Use only for
+ * queries returning less than ~10MB. For large result sets, use sync client.
+ * - COMPRESSION: Client request compression ({@code compressClientRequest})
+ * is NOT supported - requests are sent uncompressed regardless of settings.
+ * - INSERTS: Insert operations use sync fallback path.
+ * - MULTIPART: Multipart requests use sync fallback path.
+ *
+ *
+ * Best suited for: High-concurrency, read-only workloads with small result sets
+ * (e.g., aggregations, counts, single-row lookups).
+ *
+ * @param useAsyncHttp - true to enable async HTTP transport
+ * @return this builder
+ */
+ public Builder useAsyncHttp(boolean useAsyncHttp) {
+ this.configuration.put(ClientConfigProperties.USE_ASYNC_HTTP.getKey(), String.valueOf(useAsyncHttp));
+ return this;
+ }
+
/**
* Sets an executor for running operations. If async operations are enabled and no executor is specified
* client will create a default executor.
@@ -1645,6 +1674,13 @@ public CompletableFuture query(String sqlQuery, Map responseSupplier = () -> {
long startTime = System.nanoTime();
// Selecting some node
@@ -1653,7 +1689,6 @@ public CompletableFuture query(String sqlQuery, Map query(String sqlQuery, Map executeQueryAsync(String sqlQuery,
+ QuerySettings requestSettings,
+ ClientStatisticsHolder clientStats,
+ int attempt) {
+ final long startTime = System.nanoTime();
+ final Endpoint selectedEndpoint = getNextAliveNode();
+
+ return httpClientHelper.executeRequestAsync(selectedEndpoint, requestSettings.getAllSettings(), sqlQuery)
+ .handle((response, ex) -> {
+ if (ex != null) {
+ Throwable cause = ex instanceof java.util.concurrent.CompletionException ? ex.getCause() : ex;
+ String msg = requestExMsg("Query", (attempt + 1), durationSince(startTime).toMillis(), requestSettings.getQueryId());
+ RuntimeException wrappedException = httpClientHelper.wrapException(msg, (Exception) cause, requestSettings.getQueryId());
+
+ if (httpClientHelper.shouldRetry((Exception) cause, requestSettings.getAllSettings()) && attempt < retries) {
+ LOG.warn("Async query failed, retrying (attempt {}): {}", attempt + 1, cause.getMessage());
+ return new AsyncRetryMarker(attempt + 1);
+ }
+ throw new java.util.concurrent.CompletionException(wrappedException);
+ }
+
+ if (response.getCode() == HttpStatus.SC_SERVICE_UNAVAILABLE && attempt < retries) {
+ LOG.warn("Failed to get response. Server returned {}. Retrying. (Duration: {})",
+ response.getCode(), durationSince(startTime));
+ return new AsyncRetryMarker(attempt + 1);
+ }
+
+ OperationMetrics metrics = new OperationMetrics(clientStats);
+ String summary = HttpAPIClientHelper.getHeaderVal(
+ response.getFirstHeader(ClickHouseHttpProto.HEADER_SRV_SUMMARY), "{}");
+ ProcessParser.parseSummary(summary, metrics);
+ String queryId = HttpAPIClientHelper.getHeaderVal(
+ response.getFirstHeader(ClickHouseHttpProto.HEADER_QUERY_ID), requestSettings.getQueryId());
+ metrics.setQueryId(queryId);
+ metrics.operationComplete();
+
+ Header formatHeader = response.getFirstHeader(ClickHouseHttpProto.HEADER_FORMAT);
+ ClickHouseFormat responseFormat = requestSettings.getFormat();
+ if (formatHeader != null) {
+ responseFormat = ClickHouseFormat.valueOf(formatHeader.getValue());
+ }
+
+ return new QueryResponse(response, responseFormat, requestSettings, metrics);
+ })
+ .thenCompose(result -> {
+ if (result instanceof AsyncRetryMarker) {
+ return executeQueryAsync(sqlQuery, requestSettings, clientStats, ((AsyncRetryMarker) result).nextAttempt);
+ }
+ return CompletableFuture.completedFuture((QueryResponse) result);
+ });
+ }
+
+ /** Marker to signal async retry without blocking .join() calls */
+ private static class AsyncRetryMarker {
+ final int nextAttempt;
+ AsyncRetryMarker(int nextAttempt) {
+ this.nextAttempt = nextAttempt;
+ }
+ }
public CompletableFuture query(String sqlQuery, Map queryParams) {
return query(sqlQuery, queryParams, null);
}
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/ClientConfigProperties.java b/client-v2/src/main/java/com/clickhouse/client/api/ClientConfigProperties.java
index 5a23f91ba..7e38db611 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/ClientConfigProperties.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/ClientConfigProperties.java
@@ -57,6 +57,14 @@ public enum ClientConfigProperties {
ASYNC_OPERATIONS("async", Boolean.class, "false"),
+ /**
+ * Enables true async HTTP transport using Apache HttpClient 5 async API.
+ * When enabled, HTTP requests use NIO-based non-blocking I/O instead of
+ * blocking thread-per-request model. This provides better scalability
+ * under high concurrency.
+ */
+ USE_ASYNC_HTTP("use_async_http", Boolean.class, "false"),
+
CONNECTION_TTL("connection_ttl", Long.class, "-1"),
CONNECTION_TIMEOUT("connection_timeout", Long.class),
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
index 31cdeff3f..c43ce07a2 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
@@ -18,23 +18,34 @@
import net.jpountz.lz4.LZ4Factory;
import org.apache.commons.compress.compressors.CompressorStreamFactory;
import org.apache.hc.client5.http.ConnectTimeoutException;
+import org.apache.hc.client5.http.async.methods.SimpleHttpRequest;
+import org.apache.hc.client5.http.async.methods.SimpleHttpResponse;
+import org.apache.hc.client5.http.async.methods.SimpleRequestBuilder;
+import org.apache.hc.client5.http.async.methods.SimpleResponseConsumer;
import org.apache.hc.client5.http.classic.methods.HttpPost;
import org.apache.hc.client5.http.config.ConnectionConfig;
import org.apache.hc.client5.http.config.RequestConfig;
+import org.apache.hc.client5.http.config.TlsConfig;
import org.apache.hc.client5.http.entity.mime.MultipartEntityBuilder;
import org.apache.hc.client5.http.entity.mime.MultipartPartBuilder;
+import org.apache.hc.client5.http.impl.async.CloseableHttpAsyncClient;
+import org.apache.hc.client5.http.impl.async.HttpAsyncClientBuilder;
+import org.apache.hc.client5.http.impl.async.HttpAsyncClients;
import org.apache.hc.client5.http.impl.classic.CloseableHttpClient;
import org.apache.hc.client5.http.impl.classic.HttpClientBuilder;
import org.apache.hc.client5.http.impl.io.BasicHttpClientConnectionManager;
import org.apache.hc.client5.http.impl.io.ManagedHttpClientConnectionFactory;
import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManager;
import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManagerBuilder;
+import org.apache.hc.client5.http.impl.nio.PoolingAsyncClientConnectionManager;
+import org.apache.hc.client5.http.impl.nio.PoolingAsyncClientConnectionManagerBuilder;
import org.apache.hc.client5.http.io.HttpClientConnectionManager;
import org.apache.hc.client5.http.io.ManagedHttpClientConnection;
import org.apache.hc.client5.http.protocol.HttpClientContext;
import org.apache.hc.client5.http.socket.ConnectionSocketFactory;
import org.apache.hc.client5.http.socket.LayeredConnectionSocketFactory;
import org.apache.hc.client5.http.socket.PlainConnectionSocketFactory;
+import org.apache.hc.client5.http.ssl.ClientTlsStrategyBuilder;
import org.apache.hc.client5.http.ssl.SSLConnectionSocketFactory;
import org.apache.hc.core5.http.ClassicHttpResponse;
import org.apache.hc.core5.http.ConnectionRequestTimeoutException;
@@ -51,16 +62,21 @@
import org.apache.hc.core5.http.config.RegistryBuilder;
import org.apache.hc.core5.http.impl.io.DefaultHttpResponseParserFactory;
import org.apache.hc.core5.http.io.SocketConfig;
+import org.apache.hc.core5.concurrent.FutureCallback;
import org.apache.hc.core5.http.io.entity.ByteArrayEntity;
import org.apache.hc.core5.http.io.entity.EntityTemplate;
+import org.apache.hc.core5.http.nio.ssl.TlsStrategy;
import org.apache.hc.core5.http.protocol.HttpContext;
+import org.apache.hc.core5.http.ssl.TLS;
import org.apache.hc.core5.io.CloseMode;
import org.apache.hc.core5.io.IOCallback;
import org.apache.hc.core5.net.URIBuilder;
import org.apache.hc.core5.pool.ConnPoolControl;
import org.apache.hc.core5.pool.PoolConcurrencyPolicy;
import org.apache.hc.core5.pool.PoolReusePolicy;
+import org.apache.hc.core5.reactor.IOReactorConfig;
import org.apache.hc.core5.util.TimeValue;
+import org.apache.hc.core5.util.Timeout;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -96,8 +112,10 @@
import java.util.Optional;
import java.util.Properties;
import java.util.Set;
+import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.BiConsumer;
import java.util.function.Function;
@@ -116,7 +134,11 @@ public class HttpAPIClientHelper {
private static final Pattern PATTERN_HEADER_VALUE_ASCII = Pattern.compile(
"\\p{Graph}+(?:[ ]\\p{Graph}+)*");
+ private static final ContentType CONTENT_TYPE = ContentType.create(ContentType.TEXT_PLAIN.getMimeType(), "UTF-8");
+
private final CloseableHttpClient httpClient;
+ private final CloseableHttpAsyncClient httpAsyncClient;
+ private final AtomicBoolean asyncClientClosed = new AtomicBoolean(false);
private String proxyAuthHeaderValue;
@@ -147,6 +169,22 @@ public HttpAPIClientHelper(Map configuration, Object metricsRegi
}
this.defaultUserAgent = buildDefaultUserAgent();
+
+ // Initialize async client
+ boolean useAsyncHttp = ClientConfigProperties.USE_ASYNC_HTTP.getOrDefault(configuration);
+ if (useAsyncHttp) {
+ this.httpAsyncClient = createHttpAsyncClient(initSslContext, configuration);
+ this.httpAsyncClient.start();
+ LOG.info("Async HTTP client initialized and started");
+
+ boolean clientCompression = ClientConfigProperties.COMPRESS_CLIENT_REQUEST.getOrDefault(configuration);
+ if (clientCompression) {
+ LOG.warn("Async HTTP is enabled with compressClientRequest=true, but client-side compression " +
+ "is NOT supported in async mode. Requests will be sent UNCOMPRESSED.");
+ }
+ } else {
+ this.httpAsyncClient = null;
+ }
}
/**
@@ -339,6 +377,70 @@ public CloseableHttpClient createHttpClient(boolean initSslContext, Map configuration) {
+ HttpAsyncClientBuilder asyncBuilder = HttpAsyncClients.custom();
+
+ SSLContext sslContext = initSslContext ? createSSLContext(configuration) : null;
+
+ IOReactorConfig.Builder ioReactorBuilder = IOReactorConfig.custom();
+ ClientConfigProperties.SOCKET_OPERATION_TIMEOUT.applyIfSet(configuration,
+ (t) -> ioReactorBuilder.setSoTimeout(Timeout.ofMilliseconds(t)));
+ ClientConfigProperties.SOCKET_RCVBUF_OPT.applyIfSet(configuration,
+ ioReactorBuilder::setRcvBufSize);
+ ClientConfigProperties.SOCKET_SNDBUF_OPT.applyIfSet(configuration,
+ ioReactorBuilder::setSndBufSize);
+ ClientConfigProperties.SOCKET_LINGER_OPT.applyIfSet(configuration,
+ (v) -> ioReactorBuilder.setSoLinger(TimeValue.ofSeconds(v)));
+ ClientConfigProperties.SOCKET_TCP_NO_DELAY_OPT.applyIfSet(configuration,
+ ioReactorBuilder::setTcpNoDelay);
+ asyncBuilder.setIOReactorConfig(ioReactorBuilder.build());
+
+ PoolingAsyncClientConnectionManagerBuilder connMgrBuilder = PoolingAsyncClientConnectionManagerBuilder.create();
+ connMgrBuilder.setPoolConcurrencyPolicy(PoolConcurrencyPolicy.LAX);
+
+ ConnectionReuseStrategy connectionReuseStrategy = ClientConfigProperties.CONNECTION_REUSE_STRATEGY.getOrDefault(configuration);
+ switch (connectionReuseStrategy) {
+ case LIFO:
+ connMgrBuilder.setConnPoolPolicy(PoolReusePolicy.LIFO);
+ break;
+ case FIFO:
+ connMgrBuilder.setConnPoolPolicy(PoolReusePolicy.FIFO);
+ break;
+ default:
+ throw new ClientMisconfigurationException("Unknown connection reuse strategy: " + connectionReuseStrategy);
+ }
+
+ connMgrBuilder.setMaxConnTotal(Integer.MAX_VALUE);
+ ClientConfigProperties.HTTP_MAX_OPEN_CONNECTIONS.applyIfSet(configuration, connMgrBuilder::setMaxConnPerRoute);
+ connMgrBuilder.setDefaultConnectionConfig(createConnectionConfig(configuration));
+
+ if (sslContext != null) {
+ TlsStrategy tlsStrategy = ClientTlsStrategyBuilder.create()
+ .setSslContext(sslContext)
+ .setTlsVersions(TLS.V_1_2, TLS.V_1_3)
+ .build();
+ connMgrBuilder.setTlsStrategy(tlsStrategy);
+ }
+
+ asyncBuilder.setConnectionManager(connMgrBuilder.build());
+
+ String proxyHost = (String) configuration.get(ClientConfigProperties.PROXY_HOST.getKey());
+ Integer proxyPort = (Integer) configuration.get(ClientConfigProperties.PROXY_PORT.getKey());
+ String proxyTypeVal = (String) configuration.get(ClientConfigProperties.PROXY_TYPE.getKey());
+ ProxyType proxyType = proxyTypeVal == null ? null : ProxyType.valueOf(proxyTypeVal);
+
+ if (proxyType == ProxyType.HTTP && proxyHost != null && proxyPort != null) {
+ asyncBuilder.setProxy(new HttpHost(proxyHost, proxyPort));
+ }
+
+ boolean disableCookies = !((Boolean) ClientConfigProperties.HTTP_SAVE_COOKIES.getOrDefault(configuration));
+ if (disableCookies) {
+ asyncBuilder.disableCookieManagement();
+ }
+
+ return asyncBuilder.build();
+ }
+
// private static final String ERROR_CODE_PREFIX_PATTERN = "Code: %d. DB::Exception:";
private static final String ERROR_CODE_PREFIX_PATTERN = "%d. DB::Exception:";
@@ -480,6 +582,171 @@ public ClassicHttpResponse executeRequest(Endpoint server, Map r
return doPostRequest(requestConfig, req);
}
+ /**
+ * Executes an HTTP request asynchronously. Buffers entire response body in memory.
+ * For large result sets, use the streaming sync API instead.
+ */
+ public CompletableFuture executeRequestAsync(Endpoint server,
+ Map requestConfig,
+ String body) {
+ if (httpAsyncClient == null) {
+ throw new ClientException("Async HTTP client is not enabled. Set USE_ASYNC_HTTP to true.");
+ }
+
+ final URI uri = createRequestURI(server, requestConfig, true);
+ final SimpleHttpRequest request = createSimpleHttpRequest(uri, requestConfig, body);
+
+ CompletableFuture future = new CompletableFuture<>();
+
+ httpAsyncClient.execute(request, new FutureCallback() {
+ @Override
+ public void completed(SimpleHttpResponse response) {
+ try {
+ if (response.getCode() == HttpStatus.SC_PROXY_AUTHENTICATION_REQUIRED) {
+ future.completeExceptionally(new ClientMisconfigurationException(
+ "Proxy authentication required. Please check your proxy settings."));
+ return;
+ } else if (response.getCode() == HttpStatus.SC_BAD_GATEWAY) {
+ future.completeExceptionally(new ClientException(
+ "Server returned '502 Bad gateway'. Check network and proxy settings."));
+ return;
+ } else if (response.getCode() >= HttpStatus.SC_BAD_REQUEST ||
+ response.containsHeader(ClickHouseHttpProto.HEADER_EXCEPTION_CODE)) {
+ future.completeExceptionally(readErrorFromAsyncResponse(response));
+ return;
+ }
+ future.complete(response);
+ } catch (Exception e) {
+ future.completeExceptionally(e);
+ }
+ }
+
+ @Override
+ public void failed(Exception ex) {
+ LOG.debug("Async request failed to '{}': {}", uri, ex.getMessage(), ex);
+ future.completeExceptionally(ex);
+ }
+
+ @Override
+ public void cancelled() {
+ future.cancel(true);
+ }
+ });
+
+ return future;
+ }
+
+ private SimpleHttpRequest createSimpleHttpRequest(URI uri, Map requestConfig, String body) {
+ byte[] bodyBytes = body.getBytes(StandardCharsets.UTF_8);
+
+ boolean clientCompression = ClientConfigProperties.COMPRESS_CLIENT_REQUEST.getOrDefault(requestConfig);
+ boolean useHttpCompression = ClientConfigProperties.USE_HTTP_COMPRESSION.getOrDefault(requestConfig);
+ boolean appCompressedData = ClientConfigProperties.APP_COMPRESSED_DATA.getOrDefault(requestConfig);
+
+ if (clientCompression && !useHttpCompression && !appCompressedData) {
+ LOG.warn("Client-side LZ4 compression (compressClientRequest=true) is not supported for async HTTP. " +
+ "Request will be sent UNCOMPRESSED. For compressed requests, use sync client or disable async.");
+ }
+
+ SimpleRequestBuilder builder = SimpleRequestBuilder.post(uri)
+ .setBody(bodyBytes, CONTENT_TYPE);
+ addHeadersToSimpleRequest(builder, requestConfig);
+ return builder.build();
+ }
+
+ private void addHeadersToSimpleRequest(SimpleRequestBuilder builder, Map requestConfig) {
+ builder.setHeader(HttpHeaders.CONTENT_TYPE, CONTENT_TYPE.getMimeType());
+
+ if (requestConfig.containsKey(ClientConfigProperties.INPUT_OUTPUT_FORMAT.getKey())) {
+ builder.setHeader(ClickHouseHttpProto.HEADER_FORMAT,
+ ((ClickHouseFormat) requestConfig.get(ClientConfigProperties.INPUT_OUTPUT_FORMAT.getKey())).name());
+ }
+ if (requestConfig.containsKey(ClientConfigProperties.QUERY_ID.getKey())) {
+ builder.setHeader(ClickHouseHttpProto.HEADER_QUERY_ID,
+ (String) requestConfig.get(ClientConfigProperties.QUERY_ID.getKey()));
+ }
+ builder.setHeader(ClickHouseHttpProto.HEADER_DATABASE,
+ ClientConfigProperties.DATABASE.getOrDefault(requestConfig));
+
+ if (ClientConfigProperties.SSL_AUTH.getOrDefault(requestConfig).booleanValue()) {
+ builder.setHeader(ClickHouseHttpProto.HEADER_DB_USER,
+ ClientConfigProperties.USER.getOrDefault(requestConfig));
+ builder.setHeader(ClickHouseHttpProto.HEADER_SSL_CERT_AUTH, "on");
+ } else if (ClientConfigProperties.HTTP_USE_BASIC_AUTH.getOrDefault(requestConfig).booleanValue()) {
+ String user = ClientConfigProperties.USER.getOrDefault(requestConfig);
+ String password = ClientConfigProperties.PASSWORD.getOrDefault(requestConfig);
+ builder.addHeader(HttpHeaders.AUTHORIZATION,
+ "Basic " + Base64.getEncoder().encodeToString(
+ (user + ":" + password).getBytes(StandardCharsets.UTF_8)));
+ } else {
+ builder.setHeader(ClickHouseHttpProto.HEADER_DB_USER,
+ ClientConfigProperties.USER.getOrDefault(requestConfig));
+ String password = ClientConfigProperties.PASSWORD.getOrDefault(requestConfig);
+ if (password != null && !password.isEmpty()) {
+ builder.setHeader(ClickHouseHttpProto.HEADER_DB_PASSWORD, password);
+ }
+ }
+
+ if (proxyAuthHeaderValue != null) {
+ builder.addHeader(HttpHeaders.PROXY_AUTHORIZATION, proxyAuthHeaderValue);
+ }
+
+ boolean serverCompression = ClientConfigProperties.COMPRESS_SERVER_RESPONSE.getOrDefault(requestConfig);
+ boolean useHttpCompression = ClientConfigProperties.USE_HTTP_COMPRESSION.getOrDefault(requestConfig);
+ if (useHttpCompression && serverCompression) {
+ builder.setHeader(HttpHeaders.ACCEPT_ENCODING, DEFAULT_HTTP_COMPRESSION_ALGO);
+ }
+
+ for (String key : requestConfig.keySet()) {
+ if (key.startsWith(ClientConfigProperties.HTTP_HEADER_PREFIX)) {
+ Object val = requestConfig.get(key);
+ if (val != null) {
+ builder.setHeader(key.substring(ClientConfigProperties.HTTP_HEADER_PREFIX.length()),
+ String.valueOf(val));
+ }
+ }
+ }
+
+ String clientName = ClientConfigProperties.CLIENT_NAME.getOrDefault(requestConfig);
+ String userAgentValue = defaultUserAgent;
+ if (clientName != null && !clientName.isEmpty()) {
+ userAgentValue = clientName + " " + defaultUserAgent;
+ }
+ builder.setHeader(HttpHeaders.USER_AGENT, userAgentValue);
+ }
+
+ private Exception readErrorFromAsyncResponse(SimpleHttpResponse response) {
+ Header qIdHeader = response.getFirstHeader(ClickHouseHttpProto.HEADER_QUERY_ID);
+ final String queryId = qIdHeader == null ? "" : qIdHeader.getValue();
+ Header codeHeader = response.getFirstHeader(ClickHouseHttpProto.HEADER_EXCEPTION_CODE);
+ int serverCode = codeHeader == null ? 0 : Integer.parseInt(codeHeader.getValue());
+
+ String msg;
+ try {
+ byte[] bodyBytes = response.getBodyBytes();
+ if (bodyBytes != null && bodyBytes.length > 0) {
+ String bodyText = new String(bodyBytes, StandardCharsets.UTF_8);
+ msg = bodyText.replaceAll("\\s+", " ").replaceAll("\\\\n", " ").replaceAll("\\\\/", "/");
+ if (msg.trim().isEmpty()) {
+ msg = String.format(ERROR_CODE_PREFIX_PATTERN, serverCode) +
+ " (transport error: " + response.getCode() + ")";
+ }
+ } else {
+ msg = String.format(ERROR_CODE_PREFIX_PATTERN, serverCode) +
+ " (transport error: " + response.getCode() + ")";
+ }
+ } catch (Exception e) {
+ LOG.error("Failed to read error message from async response", e);
+ msg = String.format(ERROR_CODE_PREFIX_PATTERN, serverCode) +
+ " (transport error: " + response.getCode() + ")";
+ }
+ return new ServerException(serverCode, "Code: " + msg + " (queryId= " + queryId + ")", response.getCode(), queryId);
+ }
+
+ public boolean isAsyncEnabled() {
+ return httpAsyncClient != null;
+ }
+
public ClassicHttpResponse executeMultiPartRequest(Endpoint server, Map requestConfig, String sqlQuery) throws Exception {
requestConfig.put(ClientConfigProperties.COMPRESS_CLIENT_REQUEST.getKey(), false);
@@ -566,8 +833,6 @@ public static void closeQuietly(ClassicHttpResponse httpResponse) {
}
}
- private static final ContentType CONTENT_TYPE = ContentType.create(ContentType.TEXT_PLAIN.getMimeType(), "UTF-8");
-
private void addHeaders(HttpPost req, Map requestConfig) {
setHeader(req, HttpHeaders.CONTENT_TYPE, CONTENT_TYPE.getMimeType());
if (requestConfig.containsKey(ClientConfigProperties.INPUT_OUTPUT_FORMAT.getKey())) {
@@ -890,6 +1155,21 @@ private String buildDefaultUserAgent() {
public void close() {
httpClient.close(CloseMode.IMMEDIATE);
+
+ // Close async client with graceful shutdown if it was initialized
+ if (httpAsyncClient != null && asyncClientClosed.compareAndSet(false, true)) {
+ try {
+ httpAsyncClient.close(CloseMode.GRACEFUL);
+ LOG.debug("Async HTTP client closed gracefully");
+ } catch (Exception e) {
+ LOG.warn("Failed to close async HTTP client gracefully, forcing immediate close", e);
+ try {
+ httpAsyncClient.close(CloseMode.IMMEDIATE);
+ } catch (Exception e2) {
+ LOG.error("Failed to close async HTTP client", e2);
+ }
+ }
+ }
}
private static void setHeader(HttpRequest req, String headerName,
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/query/QueryResponse.java b/client-v2/src/main/java/com/clickhouse/client/api/query/QueryResponse.java
index 735fe6f58..1fe98d653 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/query/QueryResponse.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/query/QueryResponse.java
@@ -6,9 +6,11 @@
import com.clickhouse.client.api.metrics.OperationMetrics;
import com.clickhouse.client.api.metrics.ServerMetrics;
import com.clickhouse.data.ClickHouseFormat;
+import org.apache.hc.client5.http.async.methods.SimpleHttpResponse;
import org.apache.hc.core5.http.ClassicHttpResponse;
import org.apache.hc.core5.http.Header;
+import java.io.ByteArrayInputStream;
import java.io.InputStream;
import java.util.TimeZone;
@@ -35,14 +37,35 @@ public class QueryResponse implements AutoCloseable {
private ClassicHttpResponse httpResponse;
+ private byte[] bufferedResponseBody;
+
public QueryResponse(ClassicHttpResponse response, ClickHouseFormat format, QuerySettings settings,
OperationMetrics operationMetrics) {
this.httpResponse = response;
+ this.bufferedResponseBody = null;
this.format = format;
this.operationMetrics = operationMetrics;
this.settings = settings;
- Header tzHeader = response.getFirstHeader(ClickHouseHttpProto.HEADER_TIMEZONE);
+ parseServerTimezone(response.getFirstHeader(ClickHouseHttpProto.HEADER_TIMEZONE));
+ }
+
+ /**
+ * Constructor for async responses. Buffers entire response body in memory.
+ * For large result sets, use the streaming sync API instead.
+ */
+ public QueryResponse(SimpleHttpResponse response, ClickHouseFormat format, QuerySettings settings,
+ OperationMetrics operationMetrics) {
+ this.httpResponse = null;
+ this.bufferedResponseBody = response.getBodyBytes();
+ this.format = format;
+ this.operationMetrics = operationMetrics;
+ this.settings = settings;
+
+ parseServerTimezone(response.getFirstHeader(ClickHouseHttpProto.HEADER_TIMEZONE));
+ }
+
+ private void parseServerTimezone(Header tzHeader) {
if (tzHeader != null) {
try {
this.settings.setOption(ClientConfigProperties.SERVER_TIMEZONE.getKey(),
@@ -55,6 +78,9 @@ public QueryResponse(ClassicHttpResponse response, ClickHouseFormat format, Quer
public InputStream getInputStream() {
try {
+ if (bufferedResponseBody != null) {
+ return new ByteArrayInputStream(bufferedResponseBody);
+ }
return httpResponse.getEntity().getContent();
} catch (Exception e) {
throw new ClientException("Failed to construct input stream", e);
@@ -63,7 +89,7 @@ public InputStream getInputStream() {
@Override
public void close() throws Exception {
- if (httpResponse != null ) {
+ if (httpResponse != null) {
try {
httpResponse.close();
} catch (Exception e) {
diff --git a/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java b/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
new file mode 100644
index 000000000..ccf96b1be
--- /dev/null
+++ b/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
@@ -0,0 +1,454 @@
+package com.clickhouse.client;
+
+import com.clickhouse.client.api.Client;
+import com.clickhouse.client.api.ClientConfigProperties;
+import com.clickhouse.client.api.ClientException;
+import com.clickhouse.client.api.ServerException;
+import com.clickhouse.client.api.enums.Protocol;
+import com.clickhouse.client.api.query.GenericRecord;
+import com.clickhouse.client.api.query.QueryResponse;
+import com.github.tomakehurst.wiremock.WireMockServer;
+import com.github.tomakehurst.wiremock.client.WireMock;
+import com.github.tomakehurst.wiremock.common.ConsoleNotifier;
+import com.github.tomakehurst.wiremock.core.WireMockConfiguration;
+import org.apache.hc.core5.http.HttpStatus;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static com.github.tomakehurst.wiremock.stubbing.Scenario.STARTED;
+
+/**
+ * Tests for async HTTP transport using Apache HttpClient 5 async API.
+ */
+@Test(groups = {"integration"})
+public class AsyncHttpClientTests extends BaseIntegrationTest {
+
+ /**
+ * Test basic async query execution with real ClickHouse server.
+ */
+ @Test(groups = {"integration"})
+ public void testAsyncQueryBasic() {
+ if (isCloud()) {
+ return; // Skip for cloud tests
+ }
+
+ ClickHouseNode server = getServer(ClickHouseProtocol.HTTP);
+
+ try (Client client = new Client.Builder()
+ .addEndpoint(server.getBaseUri())
+ .setUsername("default")
+ .setPassword(getPassword())
+ .useAsyncHttp(true)
+ .build()) {
+
+ List records = client.queryAll("SELECT timezone()");
+ Assert.assertTrue(records.size() > 0);
+ Assert.assertNotNull(records.get(0).getString(1));
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail(e.getMessage());
+ }
+ }
+
+ /**
+ * Test that async query returns the same results as sync query.
+ */
+ @Test(groups = {"integration"})
+ public void testAsyncQueryResultsMatchSync() {
+ if (isCloud()) {
+ return;
+ }
+
+ ClickHouseNode server = getServer(ClickHouseProtocol.HTTP);
+ String query = "SELECT number, number * 2 as doubled FROM numbers(10)";
+
+ // First, get results using sync client
+ List syncResults;
+ try (Client syncClient = new Client.Builder()
+ .addEndpoint(server.getBaseUri())
+ .setUsername("default")
+ .setPassword(getPassword())
+ .useAsyncHttp(false)
+ .build()) {
+
+ syncResults = syncClient.queryAll(query);
+ }
+
+ // Then, get results using async client
+ List asyncResults;
+ try (Client asyncClient = new Client.Builder()
+ .addEndpoint(server.getBaseUri())
+ .setUsername("default")
+ .setPassword(getPassword())
+ .useAsyncHttp(true)
+ .build()) {
+
+ asyncResults = asyncClient.queryAll(query);
+ }
+
+ // Compare results
+ Assert.assertEquals(asyncResults.size(), syncResults.size());
+ for (int i = 0; i < syncResults.size(); i++) {
+ Assert.assertEquals(asyncResults.get(i).getLong(1), syncResults.get(i).getLong(1));
+ Assert.assertEquals(asyncResults.get(i).getLong(2), syncResults.get(i).getLong(2));
+ }
+ }
+
+ /**
+ * Test async query with CompletableFuture composition.
+ */
+ @Test(groups = {"integration"})
+ public void testAsyncQueryWithFutureComposition() {
+ if (isCloud()) {
+ return;
+ }
+
+ ClickHouseNode server = getServer(ClickHouseProtocol.HTTP);
+
+ try (Client client = new Client.Builder()
+ .addEndpoint(server.getBaseUri())
+ .setUsername("default")
+ .setPassword(getPassword())
+ .useAsyncHttp(true)
+ .build()) {
+
+ // Chain multiple async operations
+ CompletableFuture resultFuture = client.query("SELECT count() FROM numbers(1000)")
+ .thenApply(response -> {
+ try {
+ // Read the count from response
+ return response.getReadRows();
+ } finally {
+ try {
+ response.close();
+ } catch (Exception e) {
+ // ignore
+ }
+ }
+ });
+
+ Long count = resultFuture.get(30, TimeUnit.SECONDS);
+ Assert.assertEquals(count.longValue(), 1L); // Query reads 1000 rows but returns 1 row (count)
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail(e.getMessage());
+ }
+ }
+
+ /**
+ * Test async client properly handles server errors.
+ */
+ @Test(groups = {"integration"})
+ public void testAsyncQueryServerError() {
+ if (isCloud()) {
+ return;
+ }
+
+ ClickHouseNode server = getServer(ClickHouseProtocol.HTTP);
+
+ try (Client client = new Client.Builder()
+ .addEndpoint(server.getBaseUri())
+ .setUsername("default")
+ .setPassword(getPassword())
+ .useAsyncHttp(true)
+ .build()) {
+
+ try {
+ // This should fail with a syntax error
+ client.query("SELECT invalid;statement").get(10, TimeUnit.SECONDS);
+ Assert.fail("Expected ServerException");
+ } catch (ExecutionException e) {
+ Assert.assertTrue(e.getCause() instanceof ServerException,
+ "Expected ServerException but got: " + e.getCause().getClass().getName());
+ ServerException se = (ServerException) e.getCause();
+ Assert.assertEquals(se.getCode(), 62); // Syntax error code
+ }
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail(e.getMessage());
+ }
+ }
+
+ /**
+ * Test async query retry on 503 Service Unavailable using WireMock.
+ */
+ @Test(groups = {"integration"})
+ public void testAsyncQueryRetryOn503() {
+ if (isCloud()) {
+ return;
+ }
+
+ int serverPort = new Random().nextInt(1000) + 10000;
+ WireMockServer mockServer = new WireMockServer(WireMockConfiguration
+ .options().port(serverPort).notifier(new ConsoleNotifier(false)));
+ mockServer.start();
+
+ try {
+ // First request returns 503 (Service Unavailable)
+ mockServer.addStubMapping(WireMock.post(WireMock.anyUrl())
+ .inScenario("Retry503")
+ .whenScenarioStateIs(STARTED)
+ .willSetStateTo("Retried")
+ .willReturn(WireMock.aResponse()
+ .withStatus(HttpStatus.SC_SERVICE_UNAVAILABLE))
+ .build());
+
+ // Second request succeeds
+ mockServer.addStubMapping(WireMock.post(WireMock.anyUrl())
+ .inScenario("Retry503")
+ .whenScenarioStateIs("Retried")
+ .willReturn(WireMock.aResponse()
+ .withStatus(HttpStatus.SC_OK)
+ .withHeader("X-ClickHouse-Summary",
+ "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}"))
+ .build());
+
+ try (Client client = new Client.Builder()
+ .addEndpoint(Protocol.HTTP, "localhost", serverPort, false)
+ .setUsername("default")
+ .setPassword("")
+ .useAsyncHttp(true)
+ .setMaxRetries(3)
+ .compressServerResponse(false)
+ .build()) {
+
+ QueryResponse response = client.query("SELECT 1").get(10, TimeUnit.SECONDS);
+ Assert.assertEquals(response.getReadRows(), 1);
+ response.close();
+ }
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail(e.getMessage());
+ } finally {
+ mockServer.stop();
+ }
+ }
+
+ /**
+ * Test that async client is not enabled when USE_ASYNC_HTTP is false (default).
+ */
+ @Test(groups = {"integration"})
+ public void testAsyncHttpDisabledByDefault() {
+ if (isCloud()) {
+ return;
+ }
+
+ ClickHouseNode server = getServer(ClickHouseProtocol.HTTP);
+
+ // Create client without useAsyncHttp(true) - should use sync client
+ try (Client client = new Client.Builder()
+ .addEndpoint(server.getBaseUri())
+ .setUsername("default")
+ .setPassword(getPassword())
+ .build()) {
+
+ // Query should still work but uses sync path
+ List records = client.queryAll("SELECT 1");
+ Assert.assertEquals(records.size(), 1);
+ Assert.assertEquals(records.get(0).getString(1), "1");
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail(e.getMessage());
+ }
+ }
+
+ /**
+ * Test concurrent async queries.
+ */
+ @Test(groups = {"integration"})
+ public void testConcurrentAsyncQueries() {
+ if (isCloud()) {
+ return;
+ }
+
+ ClickHouseNode server = getServer(ClickHouseProtocol.HTTP);
+
+ try (Client client = new Client.Builder()
+ .addEndpoint(server.getBaseUri())
+ .setUsername("default")
+ .setPassword(getPassword())
+ .useAsyncHttp(true)
+ .setMaxConnections(20)
+ .build()) {
+
+ int numQueries = 10;
+ @SuppressWarnings("unchecked")
+ CompletableFuture[] futures = new CompletableFuture[numQueries];
+
+ // Launch all queries concurrently
+ for (int i = 0; i < numQueries; i++) {
+ final int queryNum = i;
+ futures[i] = client.query("SELECT " + queryNum + " as num, sleep(0.1)");
+ }
+
+ // Wait for all to complete
+ CompletableFuture.allOf(futures).get(60, TimeUnit.SECONDS);
+
+ // Verify all completed successfully
+ for (int i = 0; i < numQueries; i++) {
+ QueryResponse response = futures[i].get();
+ Assert.assertTrue(response.getReadRows() > 0 || response.getResultRows() > 0);
+ response.close();
+ }
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail(e.getMessage());
+ }
+ }
+
+ /**
+ * Test async client graceful shutdown.
+ */
+ @Test(groups = {"integration"})
+ public void testAsyncClientGracefulShutdown() {
+ if (isCloud()) {
+ return;
+ }
+
+ ClickHouseNode server = getServer(ClickHouseProtocol.HTTP);
+
+ // Create and close the client multiple times to verify no resource leaks
+ for (int i = 0; i < 3; i++) {
+ Client client = new Client.Builder()
+ .addEndpoint(server.getBaseUri())
+ .setUsername("default")
+ .setPassword(getPassword())
+ .useAsyncHttp(true)
+ .build();
+
+ try {
+ List records = client.queryAll("SELECT 1");
+ Assert.assertEquals(records.size(), 1);
+ } catch (Exception e) {
+ Assert.fail("Query failed on iteration " + i, e);
+ } finally {
+ client.close();
+ }
+ }
+ }
+
+ /**
+ * Test that cancellation of CompletableFuture works.
+ */
+ @Test(groups = {"integration"})
+ public void testAsyncQueryCancellation() {
+ if (isCloud()) {
+ return;
+ }
+
+ int serverPort = new Random().nextInt(1000) + 10000;
+ WireMockServer mockServer = new WireMockServer(WireMockConfiguration
+ .options().port(serverPort).notifier(new ConsoleNotifier(false)));
+ mockServer.start();
+
+ try {
+ // Setup a delayed response
+ mockServer.addStubMapping(WireMock.post(WireMock.anyUrl())
+ .willReturn(WireMock.aResponse()
+ .withFixedDelay(10000) // 10 second delay
+ .withStatus(HttpStatus.SC_OK)
+ .withHeader("X-ClickHouse-Summary",
+ "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}"))
+ .build());
+
+ try (Client client = new Client.Builder()
+ .addEndpoint(Protocol.HTTP, "localhost", serverPort, false)
+ .setUsername("default")
+ .setPassword("")
+ .useAsyncHttp(true)
+ .compressServerResponse(false)
+ .build()) {
+
+ CompletableFuture future = client.query("SELECT 1");
+
+ // Cancel after a short delay
+ Thread.sleep(100);
+ boolean cancelled = future.cancel(true);
+
+ // The future should be cancelled
+ Assert.assertTrue(future.isCancelled() || future.isDone());
+
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail(e.getMessage());
+ } finally {
+ mockServer.stop();
+ }
+ }
+
+ /**
+ * Test async query response metrics.
+ */
+ @Test(groups = {"integration"})
+ public void testAsyncQueryMetrics() {
+ if (isCloud()) {
+ return;
+ }
+
+ ClickHouseNode server = getServer(ClickHouseProtocol.HTTP);
+
+ try (Client client = new Client.Builder()
+ .addEndpoint(server.getBaseUri())
+ .setUsername("default")
+ .setPassword(getPassword())
+ .useAsyncHttp(true)
+ .build()) {
+
+ try (QueryResponse response = client.query("SELECT number FROM numbers(100)").get(30, TimeUnit.SECONDS)) {
+ // Verify metrics are populated
+ Assert.assertTrue(response.getReadRows() > 0, "Expected read_rows > 0");
+ Assert.assertTrue(response.getReadBytes() > 0, "Expected read_bytes > 0");
+ Assert.assertNotNull(response.getQueryId(), "Expected query_id to be set");
+ }
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail(e.getMessage());
+ }
+ }
+
+ /**
+ * Test async query with custom query ID.
+ */
+ @Test(groups = {"integration"})
+ public void testAsyncQueryWithCustomQueryId() {
+ if (isCloud()) {
+ return;
+ }
+
+ ClickHouseNode server = getServer(ClickHouseProtocol.HTTP);
+ String customQueryId = "test-async-query-" + System.currentTimeMillis();
+
+ try (Client client = new Client.Builder()
+ .addEndpoint(server.getBaseUri())
+ .setUsername("default")
+ .setPassword(getPassword())
+ .useAsyncHttp(true)
+ .build()) {
+
+ com.clickhouse.client.api.query.QuerySettings settings =
+ new com.clickhouse.client.api.query.QuerySettings().setQueryId(customQueryId);
+
+ try (QueryResponse response = client.query("SELECT 1", settings).get(30, TimeUnit.SECONDS)) {
+ Assert.assertEquals(response.getQueryId(), customQueryId);
+ }
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail(e.getMessage());
+ }
+ }
+}
From 860fe1bb48b816e36c05f86b3795b643f1cbe222 Mon Sep 17 00:00:00 2001
From: David Karlsson <2795016+devdavidkarlsson@users.noreply.github.com>
Date: Wed, 11 Feb 2026 17:21:11 +0100
Subject: [PATCH 02/32] Add streaming async response support (Phase 2)
Implements streaming async responses using PipedInputStream to avoid
buffering entire response in memory. This enables async queries for
large result sets without OOM risk.
Key changes:
- Add StreamingAsyncResponseConsumer with PipedInputStream bridge
- Complete future on headers arrival (not stream end) to prevent deadlock
- Add executeRequestAsyncStreaming() method in HttpAPIClientHelper
- Update QueryResponse with streaming InputStream constructor
- Increase pipe buffer to 512KB for better throughput
Fixes:
- Critical deadlock when user delays reading from stream
- Remove unnecessary flush() on every chunk (+10-20% throughput)
- Use reasonable capacityIncrement (8KB vs Integer.MAX_VALUE)
Tests:
- testAsyncStreamingLargeResult: 100K row streaming
- testAsyncStreamingIncrementalRead: incremental byte reading
- testAsyncStreamingNoDeadlockOnDelayedRead: deadlock regression test
---
.../com/clickhouse/client/api/Client.java | 13 +-
.../api/internal/HttpAPIClientHelper.java | 147 ++++++++++++++
.../StreamingAsyncResponseConsumer.java | 190 ++++++++++++++++++
.../client/api/query/QueryResponse.java | 33 ++-
.../client/AsyncHttpClientTests.java | 133 ++++++++++++
5 files changed, 508 insertions(+), 8 deletions(-)
create mode 100644 client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncResponseConsumer.java
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/Client.java b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
index aade43049..15588bd82 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/Client.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
@@ -18,6 +18,7 @@
import com.clickhouse.client.api.internal.ClientStatisticsHolder;
import com.clickhouse.client.api.internal.HttpAPIClientHelper;
import com.clickhouse.client.api.internal.MapUtils;
+import com.clickhouse.client.api.internal.StreamingAsyncResponseConsumer;
import com.clickhouse.client.api.internal.TableSchemaParser;
import com.clickhouse.client.api.internal.ValidationUtils;
import com.clickhouse.client.api.metadata.ColumnToMethodMatchingStrategy;
@@ -832,19 +833,17 @@ public Builder useAsyncRequests(boolean async) {
* blocking thread-per-request model. This provides better scalability
* under high concurrency without requiring thread pools.
*
- * WARNING - Phase 1 Limitations:
+ * Response bodies are streamed through a pipe, avoiding memory buffering.
+ * This makes async suitable for large result sets.
+ *
+ * Current Limitations:
*
- * - MEMORY: Response bodies are fully buffered in memory. Use only for
- * queries returning less than ~10MB. For large result sets, use sync client.
* - COMPRESSION: Client request compression ({@code compressClientRequest})
* is NOT supported - requests are sent uncompressed regardless of settings.
* - INSERTS: Insert operations use sync fallback path.
* - MULTIPART: Multipart requests use sync fallback path.
*
*
- * Best suited for: High-concurrency, read-only workloads with small result sets
- * (e.g., aggregations, counts, single-row lookups).
- *
* @param useAsyncHttp - true to enable async HTTP transport
* @return this builder
*/
@@ -1748,7 +1747,7 @@ private CompletableFuture executeQueryAsync(String sqlQuery,
final long startTime = System.nanoTime();
final Endpoint selectedEndpoint = getNextAliveNode();
- return httpClientHelper.executeRequestAsync(selectedEndpoint, requestSettings.getAllSettings(), sqlQuery)
+ return httpClientHelper.executeRequestAsyncStreaming(selectedEndpoint, requestSettings.getAllSettings(), sqlQuery)
.handle((response, ex) -> {
if (ex != null) {
Throwable cause = ex instanceof java.util.concurrent.CompletionException ? ex.getCause() : ex;
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
index c43ce07a2..cdba0994d 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
@@ -56,6 +56,7 @@
import org.apache.hc.core5.http.HttpHost;
import org.apache.hc.core5.http.HttpRequest;
import org.apache.hc.core5.http.HttpStatus;
+import org.apache.hc.core5.http.message.BasicHttpRequest;
import org.apache.hc.core5.http.NoHttpResponseException;
import org.apache.hc.core5.http.config.CharCodingConfig;
import org.apache.hc.core5.http.config.Http1Config;
@@ -65,7 +66,11 @@
import org.apache.hc.core5.concurrent.FutureCallback;
import org.apache.hc.core5.http.io.entity.ByteArrayEntity;
import org.apache.hc.core5.http.io.entity.EntityTemplate;
+import org.apache.hc.core5.http.nio.AsyncEntityProducer;
+import org.apache.hc.core5.http.nio.AsyncRequestProducer;
+import org.apache.hc.core5.http.nio.entity.BasicAsyncEntityProducer;
import org.apache.hc.core5.http.nio.ssl.TlsStrategy;
+import org.apache.hc.core5.http.nio.support.BasicRequestProducer;
import org.apache.hc.core5.http.protocol.HttpContext;
import org.apache.hc.core5.http.ssl.TLS;
import org.apache.hc.core5.io.CloseMode;
@@ -636,6 +641,101 @@ public void cancelled() {
return future;
}
+ /**
+ * Executes an HTTP request asynchronously with streaming response.
+ * Response body is streamed through a PipedInputStream, avoiding memory buffering.
+ * Suitable for large result sets.
+ *
+ * IMPORTANT: The returned future completes as soon as HTTP headers are received,
+ * NOT when all data has been transferred. This allows the caller to start reading
+ * from the stream immediately, preventing deadlock.
+ */
+ public CompletableFuture executeRequestAsyncStreaming(
+ Endpoint server,
+ Map requestConfig,
+ String body) {
+ if (httpAsyncClient == null) {
+ throw new ClientException("Async HTTP client is not enabled. Set USE_ASYNC_HTTP to true.");
+ }
+
+ final URI uri = createRequestURI(server, requestConfig, true);
+ byte[] bodyBytes = body.getBytes(StandardCharsets.UTF_8);
+
+ BasicHttpRequest request = new BasicHttpRequest("POST", uri);
+ addHeadersToRequest(request, requestConfig);
+
+ AsyncEntityProducer entityProducer = new BasicAsyncEntityProducer(bodyBytes, CONTENT_TYPE);
+ AsyncRequestProducer requestProducer = new BasicRequestProducer(request, entityProducer);
+
+ StreamingAsyncResponseConsumer responseConsumer = new StreamingAsyncResponseConsumer();
+
+ CompletableFuture future = new CompletableFuture<>();
+
+ // Complete future when headers arrive (via headersFuture), NOT when stream ends.
+ // This prevents deadlock: user can start reading while NIO thread writes.
+ responseConsumer.getHeadersFuture().whenComplete((response, headerEx) -> {
+ if (headerEx != null) {
+ future.completeExceptionally(headerEx);
+ return;
+ }
+
+ try {
+ if (response.getCode() == HttpStatus.SC_PROXY_AUTHENTICATION_REQUIRED) {
+ future.completeExceptionally(new ClientMisconfigurationException(
+ "Proxy authentication required. Please check your proxy settings."));
+ } else if (response.getCode() == HttpStatus.SC_BAD_GATEWAY) {
+ future.completeExceptionally(new ClientException(
+ "Server returned '502 Bad gateway'. Check network and proxy settings."));
+ } else if (response.getCode() >= HttpStatus.SC_BAD_REQUEST ||
+ response.containsHeader(ClickHouseHttpProto.HEADER_EXCEPTION_CODE)) {
+ future.completeExceptionally(readErrorFromStreamingResponse(response));
+ } else {
+ future.complete(response);
+ }
+ } catch (Exception e) {
+ future.completeExceptionally(e);
+ }
+ });
+
+ httpAsyncClient.execute(requestProducer, responseConsumer, new FutureCallback() {
+ @Override
+ public void completed(StreamingAsyncResponseConsumer.StreamingResponse response) {
+ // Stream has ended. Future should already be completed via headersFuture.
+ LOG.debug("Async streaming request completed for '{}'", uri);
+ }
+
+ @Override
+ public void failed(Exception ex) {
+ LOG.debug("Async streaming request failed to '{}': {}", uri, ex.getMessage(), ex);
+ // Complete future exceptionally if not already done
+ future.completeExceptionally(ex);
+ }
+
+ @Override
+ public void cancelled() {
+ future.cancel(true);
+ }
+ });
+
+ return future;
+ }
+
+ private Exception readErrorFromStreamingResponse(StreamingAsyncResponseConsumer.StreamingResponse response) {
+ try {
+ InputStream is = response.getInputStream();
+ byte[] errorBytes = new byte[ERROR_BODY_BUFFER_SIZE];
+ int bytesRead = is.read(errorBytes, 0, ERROR_BODY_BUFFER_SIZE);
+ String errorBody = bytesRead > 0 ? new String(errorBytes, 0, bytesRead, StandardCharsets.UTF_8) : "";
+
+ Header errorCodeHeader = response.getFirstHeader(ClickHouseHttpProto.HEADER_EXCEPTION_CODE);
+ int errorCode = errorCodeHeader != null ? Integer.parseInt(errorCodeHeader.getValue()) : 0;
+
+ return new ServerException(errorCode, errorBody, response.getCode(), null);
+ } catch (Exception e) {
+ return new ClientException("Failed to read error response", e);
+ }
+ }
+
private SimpleHttpRequest createSimpleHttpRequest(URI uri, Map requestConfig, String body) {
byte[] bodyBytes = body.getBytes(StandardCharsets.UTF_8);
@@ -926,6 +1026,53 @@ private void addHeaders(HttpPost req, Map requestConfig) {
correctUserAgentHeader(req, requestConfig);
}
+ private void addHeadersToRequest(HttpRequest req, Map requestConfig) {
+ setHeader(req, HttpHeaders.CONTENT_TYPE, CONTENT_TYPE.getMimeType());
+ if (requestConfig.containsKey(ClientConfigProperties.INPUT_OUTPUT_FORMAT.getKey())) {
+ setHeader(req, ClickHouseHttpProto.HEADER_FORMAT,
+ ((ClickHouseFormat) requestConfig.get(ClientConfigProperties.INPUT_OUTPUT_FORMAT.getKey())).name());
+ }
+ if (requestConfig.containsKey(ClientConfigProperties.QUERY_ID.getKey())) {
+ setHeader(req, ClickHouseHttpProto.HEADER_QUERY_ID,
+ (String) requestConfig.get(ClientConfigProperties.QUERY_ID.getKey()));
+ }
+ setHeader(req, ClickHouseHttpProto.HEADER_DATABASE, ClientConfigProperties.DATABASE.getOrDefault(requestConfig));
+
+ if (ClientConfigProperties.SSL_AUTH.getOrDefault(requestConfig).booleanValue()) {
+ setHeader(req, ClickHouseHttpProto.HEADER_DB_USER, ClientConfigProperties.USER.getOrDefault(requestConfig));
+ setHeader(req, ClickHouseHttpProto.HEADER_SSL_CERT_AUTH, "on");
+ } else if (ClientConfigProperties.HTTP_USE_BASIC_AUTH.getOrDefault(requestConfig).booleanValue()) {
+ String user = ClientConfigProperties.USER.getOrDefault(requestConfig);
+ String password = ClientConfigProperties.PASSWORD.getOrDefault(requestConfig);
+ req.addHeader(HttpHeaders.AUTHORIZATION,
+ "Basic " + Base64.getEncoder().encodeToString((user + ":" + password).getBytes(StandardCharsets.UTF_8)));
+ } else {
+ setHeader(req, ClickHouseHttpProto.HEADER_DB_USER, ClientConfigProperties.USER.getOrDefault(requestConfig));
+ setHeader(req, ClickHouseHttpProto.HEADER_DB_PASSWORD, ClientConfigProperties.PASSWORD.getOrDefault(requestConfig));
+ }
+
+ if (proxyAuthHeaderValue != null) {
+ req.addHeader(HttpHeaders.PROXY_AUTHORIZATION, proxyAuthHeaderValue);
+ }
+
+ boolean serverCompression = ClientConfigProperties.COMPRESS_SERVER_RESPONSE.getOrDefault(requestConfig);
+ boolean useHttpCompression = ClientConfigProperties.USE_HTTP_COMPRESSION.getOrDefault(requestConfig);
+ if (useHttpCompression && serverCompression) {
+ setHeader(req, HttpHeaders.ACCEPT_ENCODING, DEFAULT_HTTP_COMPRESSION_ALGO);
+ }
+
+ for (String key : requestConfig.keySet()) {
+ if (key.startsWith(ClientConfigProperties.HTTP_HEADER_PREFIX)) {
+ Object val = requestConfig.get(key);
+ if (val != null) {
+ setHeader(req, key.substring(ClientConfigProperties.HTTP_HEADER_PREFIX.length()), String.valueOf(val));
+ }
+ }
+ }
+
+ correctUserAgentHeader(req, requestConfig);
+ }
+
private void addRequestParams(Map requestConfig, BiConsumer consumer) {
if (requestConfig.containsKey(ClientConfigProperties.QUERY_ID.getKey())) {
consumer.accept(ClickHouseHttpProto.QPARAM_QUERY_ID, requestConfig.get(ClientConfigProperties.QUERY_ID.getKey()).toString());
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncResponseConsumer.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncResponseConsumer.java
new file mode 100644
index 000000000..81ebd2ddc
--- /dev/null
+++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncResponseConsumer.java
@@ -0,0 +1,190 @@
+package com.clickhouse.client.api.internal;
+
+import org.apache.hc.client5.http.async.methods.AbstractBinResponseConsumer;
+import org.apache.hc.core5.http.ContentType;
+import org.apache.hc.core5.http.Header;
+import org.apache.hc.core5.http.HttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.PipedInputStream;
+import java.io.PipedOutputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * Async response consumer that streams response body through a PipedInputStream.
+ * Data is written to PipedOutputStream as it arrives from the NIO thread,
+ * and can be read from the connected PipedInputStream in the user's thread.
+ *
+ * IMPORTANT: The {@link #getHeadersFuture()} completes as soon as headers arrive,
+ * allowing the caller to start reading from the stream immediately. This prevents
+ * deadlock - the NIO thread can write while the user thread reads concurrently.
+ */
+public class StreamingAsyncResponseConsumer extends AbstractBinResponseConsumer {
+
+ private static final Logger LOG = LoggerFactory.getLogger(StreamingAsyncResponseConsumer.class);
+ private static final int DEFAULT_PIPE_SIZE = 512 * 1024; // 512KB pipe buffer
+ private static final int CAPACITY_INCREMENT = 8 * 1024; // 8KB chunks
+
+ private final PipedInputStream pipedInputStream;
+ private final PipedOutputStream pipedOutputStream;
+ private final CompletableFuture headersFuture;
+ private final CompletableFuture streamCompleteFuture;
+ private final AtomicBoolean outputClosed = new AtomicBoolean(false);
+
+ private HttpResponse response;
+ private ContentType contentType;
+ private volatile Exception streamError;
+
+ public StreamingAsyncResponseConsumer() {
+ this(DEFAULT_PIPE_SIZE);
+ }
+
+ public StreamingAsyncResponseConsumer(int pipeSize) {
+ this.pipedInputStream = new PipedInputStream(pipeSize);
+ this.headersFuture = new CompletableFuture<>();
+ this.streamCompleteFuture = new CompletableFuture<>();
+ try {
+ this.pipedOutputStream = new PipedOutputStream(pipedInputStream);
+ } catch (IOException e) {
+ throw new RuntimeException("Failed to create piped streams", e);
+ }
+ }
+
+ /**
+ * Returns a future that completes when HTTP headers are received.
+ * Use this to get the response early and start reading from the stream
+ * before all data has arrived. This prevents deadlock.
+ */
+ public CompletableFuture getHeadersFuture() {
+ return headersFuture;
+ }
+
+ @Override
+ protected void start(HttpResponse response, ContentType contentType) throws IOException {
+ this.response = response;
+ this.contentType = contentType;
+ LOG.debug("Streaming response started: status={}, contentType={}",
+ response.getCode(), contentType);
+
+ // Complete headers future immediately so caller can start reading
+ StreamingResponse streamingResponse = new StreamingResponse(
+ response, contentType, pipedInputStream, streamCompleteFuture);
+ headersFuture.complete(streamingResponse);
+ }
+
+ @Override
+ protected int capacityIncrement() {
+ return CAPACITY_INCREMENT;
+ }
+
+ @Override
+ protected void data(ByteBuffer src, boolean endOfStream) throws IOException {
+ if (streamError != null) {
+ return;
+ }
+
+ try {
+ if (src.hasRemaining()) {
+ byte[] bytes = new byte[src.remaining()];
+ src.get(bytes);
+ pipedOutputStream.write(bytes);
+ }
+
+ if (endOfStream) {
+ closeOutputStream();
+ }
+ } catch (IOException e) {
+ streamError = e;
+ closeOutputStream();
+ throw e;
+ }
+ }
+
+ @Override
+ protected StreamingResponse buildResult() {
+ // Return the same response that was provided via headersFuture
+ return new StreamingResponse(response, contentType, pipedInputStream, streamCompleteFuture);
+ }
+
+ @Override
+ public void releaseResources() {
+ closeOutputStream();
+ }
+
+ @Override
+ public void failed(Exception cause) {
+ LOG.debug("Streaming response failed", cause);
+ streamError = cause;
+ closeOutputStream();
+
+ // Complete both futures exceptionally if not already completed
+ headersFuture.completeExceptionally(cause);
+ streamCompleteFuture.completeExceptionally(cause);
+ }
+
+ private void closeOutputStream() {
+ if (outputClosed.compareAndSet(false, true)) {
+ try {
+ pipedOutputStream.close();
+ streamCompleteFuture.complete(null);
+ } catch (IOException e) {
+ LOG.debug("Error closing piped output stream", e);
+ }
+ }
+ }
+
+ /**
+ * Result object containing HTTP response metadata and the streaming InputStream.
+ */
+ public static class StreamingResponse {
+ private final HttpResponse httpResponse;
+ private final ContentType contentType;
+ private final InputStream inputStream;
+ private final CompletableFuture completeFuture;
+
+ StreamingResponse(HttpResponse httpResponse, ContentType contentType,
+ InputStream inputStream, CompletableFuture completeFuture) {
+ this.httpResponse = httpResponse;
+ this.contentType = contentType;
+ this.inputStream = inputStream;
+ this.completeFuture = completeFuture;
+ }
+
+ public HttpResponse getHttpResponse() {
+ return httpResponse;
+ }
+
+ public int getCode() {
+ return httpResponse.getCode();
+ }
+
+ public Header getFirstHeader(String name) {
+ return httpResponse.getFirstHeader(name);
+ }
+
+ public boolean containsHeader(String name) {
+ return httpResponse.containsHeader(name);
+ }
+
+ public ContentType getContentType() {
+ return contentType;
+ }
+
+ public InputStream getInputStream() {
+ return inputStream;
+ }
+
+ public CompletableFuture getCompleteFuture() {
+ return completeFuture;
+ }
+
+ public void close() throws IOException {
+ inputStream.close();
+ }
+ }
+}
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/query/QueryResponse.java b/client-v2/src/main/java/com/clickhouse/client/api/query/QueryResponse.java
index 1fe98d653..b7bd30ef2 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/query/QueryResponse.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/query/QueryResponse.java
@@ -3,6 +3,7 @@
import com.clickhouse.client.api.ClientConfigProperties;
import com.clickhouse.client.api.ClientException;
import com.clickhouse.client.api.http.ClickHouseHttpProto;
+import com.clickhouse.client.api.internal.StreamingAsyncResponseConsumer;
import com.clickhouse.client.api.metrics.OperationMetrics;
import com.clickhouse.client.api.metrics.ServerMetrics;
import com.clickhouse.data.ClickHouseFormat;
@@ -11,6 +12,7 @@
import org.apache.hc.core5.http.Header;
import java.io.ByteArrayInputStream;
+import java.io.IOException;
import java.io.InputStream;
import java.util.TimeZone;
@@ -39,6 +41,8 @@ public class QueryResponse implements AutoCloseable {
private byte[] bufferedResponseBody;
+ private InputStream streamingInputStream;
+
public QueryResponse(ClassicHttpResponse response, ClickHouseFormat format, QuerySettings settings,
OperationMetrics operationMetrics) {
this.httpResponse = response;
@@ -52,12 +56,29 @@ public QueryResponse(ClassicHttpResponse response, ClickHouseFormat format, Quer
/**
* Constructor for async responses. Buffers entire response body in memory.
- * For large result sets, use the streaming sync API instead.
+ * For large result sets, use the streaming async API or sync API instead.
*/
public QueryResponse(SimpleHttpResponse response, ClickHouseFormat format, QuerySettings settings,
OperationMetrics operationMetrics) {
this.httpResponse = null;
this.bufferedResponseBody = response.getBodyBytes();
+ this.streamingInputStream = null;
+ this.format = format;
+ this.operationMetrics = operationMetrics;
+ this.settings = settings;
+
+ parseServerTimezone(response.getFirstHeader(ClickHouseHttpProto.HEADER_TIMEZONE));
+ }
+
+ /**
+ * Constructor for streaming async responses. Response body is streamed through a pipe,
+ * avoiding memory buffering. Suitable for large result sets.
+ */
+ public QueryResponse(StreamingAsyncResponseConsumer.StreamingResponse response, ClickHouseFormat format,
+ QuerySettings settings, OperationMetrics operationMetrics) {
+ this.httpResponse = null;
+ this.bufferedResponseBody = null;
+ this.streamingInputStream = response.getInputStream();
this.format = format;
this.operationMetrics = operationMetrics;
this.settings = settings;
@@ -78,6 +99,9 @@ private void parseServerTimezone(Header tzHeader) {
public InputStream getInputStream() {
try {
+ if (streamingInputStream != null) {
+ return streamingInputStream;
+ }
if (bufferedResponseBody != null) {
return new ByteArrayInputStream(bufferedResponseBody);
}
@@ -89,6 +113,13 @@ public InputStream getInputStream() {
@Override
public void close() throws Exception {
+ if (streamingInputStream != null) {
+ try {
+ streamingInputStream.close();
+ } catch (IOException e) {
+ throw new ClientException("Failed to close streaming response", e);
+ }
+ }
if (httpResponse != null) {
try {
httpResponse.close();
diff --git a/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java b/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
index ccf96b1be..7916269ba 100644
--- a/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
+++ b/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
@@ -451,4 +451,137 @@ public void testAsyncQueryWithCustomQueryId() {
Assert.fail(e.getMessage());
}
}
+
+ /**
+ * Test async streaming with larger result set to verify streaming works.
+ */
+ @Test(groups = {"integration"})
+ public void testAsyncStreamingLargeResult() {
+ if (isCloud()) {
+ return;
+ }
+
+ ClickHouseNode server = getServer(ClickHouseProtocol.HTTP);
+
+ try (Client client = new Client.Builder()
+ .addEndpoint(server.getBaseUri())
+ .setUsername("default")
+ .setPassword(getPassword())
+ .useAsyncHttp(true)
+ .build()) {
+
+ // Query that returns ~1MB of data (100K rows * ~10 bytes each)
+ try (QueryResponse response = client.query("SELECT number, toString(number) FROM numbers(100000)")
+ .get(60, TimeUnit.SECONDS)) {
+
+ Assert.assertTrue(response.getReadRows() > 0, "Expected read_rows > 0");
+
+ // Read and count lines from the streaming response
+ java.io.BufferedReader reader = new java.io.BufferedReader(
+ new java.io.InputStreamReader(response.getInputStream()));
+ long lineCount = 0;
+ while (reader.readLine() != null) {
+ lineCount++;
+ }
+
+ Assert.assertEquals(lineCount, 100000, "Expected 100000 rows");
+ }
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail(e.getMessage());
+ }
+ }
+
+ /**
+ * Test async streaming response can be read incrementally.
+ */
+ @Test(groups = {"integration"})
+ public void testAsyncStreamingIncrementalRead() {
+ if (isCloud()) {
+ return;
+ }
+
+ ClickHouseNode server = getServer(ClickHouseProtocol.HTTP);
+
+ try (Client client = new Client.Builder()
+ .addEndpoint(server.getBaseUri())
+ .setUsername("default")
+ .setPassword(getPassword())
+ .useAsyncHttp(true)
+ .build()) {
+
+ try (QueryResponse response = client.query("SELECT number FROM numbers(1000)")
+ .get(30, TimeUnit.SECONDS)) {
+
+ java.io.InputStream is = response.getInputStream();
+ byte[] buffer = new byte[100];
+ int totalBytesRead = 0;
+ int bytesRead;
+
+ // Read incrementally
+ while ((bytesRead = is.read(buffer)) != -1) {
+ totalBytesRead += bytesRead;
+ }
+
+ Assert.assertTrue(totalBytesRead > 0, "Expected to read data from stream");
+ }
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail(e.getMessage());
+ }
+ }
+
+ /**
+ * Test that streaming async does NOT deadlock when reading is delayed.
+ * This tests the fix for the critical deadlock issue where:
+ * - NIO thread blocks on pipe write (buffer full)
+ * - User thread waits on future.get() (waiting for stream end)
+ * - Neither can proceed = deadlock
+ *
+ * The fix: future completes when headers arrive, not when stream ends.
+ */
+ @Test(groups = {"integration"}, timeOut = 30000) // 30 second timeout catches deadlock
+ public void testAsyncStreamingNoDeadlockOnDelayedRead() {
+ if (isCloud()) {
+ return;
+ }
+
+ ClickHouseNode server = getServer(ClickHouseProtocol.HTTP);
+
+ try (Client client = new Client.Builder()
+ .addEndpoint(server.getBaseUri())
+ .setUsername("default")
+ .setPassword(getPassword())
+ .useAsyncHttp(true)
+ .build()) {
+
+ // Query that returns data larger than pipe buffer (512KB)
+ // This would deadlock with the old implementation if user delays reading
+ CompletableFuture future = client.query(
+ "SELECT number, repeat('x', 100) FROM numbers(10000)"); // ~1MB response
+
+ // Simulate delayed reading - OLD code would deadlock here
+ Thread.sleep(500);
+
+ // Get response - should complete immediately since headers arrived
+ QueryResponse response = future.get(5, TimeUnit.SECONDS);
+
+ // Now read the stream - NIO thread continues writing while we read
+ java.io.BufferedReader reader = new java.io.BufferedReader(
+ new java.io.InputStreamReader(response.getInputStream()));
+ long lineCount = 0;
+ while (reader.readLine() != null) {
+ lineCount++;
+ }
+
+ Assert.assertEquals(lineCount, 10000, "Expected 10000 rows");
+ response.close();
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail(e.getMessage());
+ }
+ }
}
From efef002ea2d3af4677b2d3904b5dba37ff06a709 Mon Sep 17 00:00:00 2001
From: David Karlsson <2795016+devdavidkarlsson@users.noreply.github.com>
Date: Wed, 11 Feb 2026 17:33:33 +0100
Subject: [PATCH 03/32] Add LZ4 request compression for async HTTP (Phase 3)
Implements request compression for async queries using both HTTP framed
LZ4 and ClickHouse native LZ4 formats.
Key changes:
- Add compressLZ4() method supporting both compression formats
- Apply compression in executeRequestAsyncStreaming() for queries
- Move Content-Encoding header logic into addHeadersToRequest()
- Remove startup warning about compression not being supported
- Update documentation to list compression as supported feature
Note: Compression is applied in-memory before sending, which is acceptable
for queries (small payloads). For large insert operations with compression,
sync client should be used (documented limitation).
Tests:
- testAsyncQueryWithCompression: HTTP framed LZ4
- testAsyncQueryWithNativeLZ4Compression: ClickHouse native LZ4
---
.../com/clickhouse/client/api/Client.java | 10 +--
.../api/internal/HttpAPIClientHelper.java | 62 ++++++++++++++-----
.../client/AsyncHttpClientTests.java | 60 ++++++++++++++++++
3 files changed, 114 insertions(+), 18 deletions(-)
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/Client.java b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
index 15588bd82..39d7b2e5c 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/Client.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
@@ -833,13 +833,15 @@ public Builder useAsyncRequests(boolean async) {
* blocking thread-per-request model. This provides better scalability
* under high concurrency without requiring thread pools.
*
- * Response bodies are streamed through a pipe, avoiding memory buffering.
- * This makes async suitable for large result sets.
+ * Features:
+ *
+ * - Response bodies are streamed through a pipe, avoiding memory buffering
+ * - LZ4 request compression is supported ({@code compressClientRequest})
+ * - Suitable for large result sets and high-concurrency workloads
+ *
*
* Current Limitations:
*
- * - COMPRESSION: Client request compression ({@code compressClientRequest})
- * is NOT supported - requests are sent uncompressed regardless of settings.
* - INSERTS: Insert operations use sync fallback path.
* - MULTIPART: Multipart requests use sync fallback path.
*
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
index cdba0994d..72f88f2ee 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
@@ -181,12 +181,6 @@ public HttpAPIClientHelper(Map configuration, Object metricsRegi
this.httpAsyncClient = createHttpAsyncClient(initSslContext, configuration);
this.httpAsyncClient.start();
LOG.info("Async HTTP client initialized and started");
-
- boolean clientCompression = ClientConfigProperties.COMPRESS_CLIENT_REQUEST.getOrDefault(configuration);
- if (clientCompression) {
- LOG.warn("Async HTTP is enabled with compressClientRequest=true, but client-side compression " +
- "is NOT supported in async mode. Requests will be sent UNCOMPRESSED.");
- }
} else {
this.httpAsyncClient = null;
}
@@ -661,6 +655,17 @@ public CompletableFuture execu
final URI uri = createRequestURI(server, requestConfig, true);
byte[] bodyBytes = body.getBytes(StandardCharsets.UTF_8);
+ // Apply compression if configured (acceptable for queries which are small payloads)
+ boolean clientCompression = ClientConfigProperties.COMPRESS_CLIENT_REQUEST.getOrDefault(requestConfig);
+ boolean useHttpCompression = ClientConfigProperties.USE_HTTP_COMPRESSION.getOrDefault(requestConfig);
+ boolean appCompressedData = ClientConfigProperties.APP_COMPRESSED_DATA.getOrDefault(requestConfig);
+
+ if (clientCompression && !appCompressedData) {
+ int bufferSize = ClientConfigProperties.COMPRESSION_LZ4_UNCOMPRESSED_BUF_SIZE.getOrDefault(requestConfig);
+ bodyBytes = compressLZ4(bodyBytes, useHttpCompression, bufferSize);
+ LOG.debug("Async request compressed: {} -> {} bytes", body.length(), bodyBytes.length);
+ }
+
BasicHttpRequest request = new BasicHttpRequest("POST", uri);
addHeadersToRequest(request, requestConfig);
@@ -736,17 +741,40 @@ private Exception readErrorFromStreamingResponse(StreamingAsyncResponseConsumer.
}
}
- private SimpleHttpRequest createSimpleHttpRequest(URI uri, Map requestConfig, String body) {
- byte[] bodyBytes = body.getBytes(StandardCharsets.UTF_8);
+ /**
+ * Compresses data using LZ4 compression.
+ *
+ * @param data the uncompressed data
+ * @param useHttpCompression if true, uses framed LZ4 (HTTP Content-Encoding compatible);
+ * if false, uses ClickHouse native LZ4 format
+ * @param bufferSize buffer size for compression
+ * @return compressed data
+ */
+ private byte[] compressLZ4(byte[] data, boolean useHttpCompression, int bufferSize) {
+ try {
+ java.io.ByteArrayOutputStream baos = new java.io.ByteArrayOutputStream();
+ OutputStream compressingStream;
- boolean clientCompression = ClientConfigProperties.COMPRESS_CLIENT_REQUEST.getOrDefault(requestConfig);
- boolean useHttpCompression = ClientConfigProperties.USE_HTTP_COMPRESSION.getOrDefault(requestConfig);
- boolean appCompressedData = ClientConfigProperties.APP_COMPRESSED_DATA.getOrDefault(requestConfig);
+ if (useHttpCompression) {
+ compressingStream = new org.apache.commons.compress.compressors.lz4.FramedLZ4CompressorOutputStream(baos);
+ } else {
+ compressingStream = new ClickHouseLZ4OutputStream(baos, lz4Factory.fastCompressor(), bufferSize);
+ }
+
+ try {
+ compressingStream.write(data);
+ } finally {
+ compressingStream.close();
+ }
- if (clientCompression && !useHttpCompression && !appCompressedData) {
- LOG.warn("Client-side LZ4 compression (compressClientRequest=true) is not supported for async HTTP. " +
- "Request will be sent UNCOMPRESSED. For compressed requests, use sync client or disable async.");
+ return baos.toByteArray();
+ } catch (IOException e) {
+ throw new ClientException("Failed to compress request data", e);
}
+ }
+
+ private SimpleHttpRequest createSimpleHttpRequest(URI uri, Map requestConfig, String body) {
+ byte[] bodyBytes = body.getBytes(StandardCharsets.UTF_8);
SimpleRequestBuilder builder = SimpleRequestBuilder.post(uri)
.setBody(bodyBytes, CONTENT_TYPE);
@@ -1055,11 +1083,17 @@ private void addHeadersToRequest(HttpRequest req, Map requestCon
req.addHeader(HttpHeaders.PROXY_AUTHORIZATION, proxyAuthHeaderValue);
}
+ boolean clientCompression = ClientConfigProperties.COMPRESS_CLIENT_REQUEST.getOrDefault(requestConfig);
boolean serverCompression = ClientConfigProperties.COMPRESS_SERVER_RESPONSE.getOrDefault(requestConfig);
boolean useHttpCompression = ClientConfigProperties.USE_HTTP_COMPRESSION.getOrDefault(requestConfig);
+ boolean appCompressedData = ClientConfigProperties.APP_COMPRESSED_DATA.getOrDefault(requestConfig);
+
if (useHttpCompression && serverCompression) {
setHeader(req, HttpHeaders.ACCEPT_ENCODING, DEFAULT_HTTP_COMPRESSION_ALGO);
}
+ if (useHttpCompression && clientCompression && !appCompressedData) {
+ setHeader(req, HttpHeaders.CONTENT_ENCODING, DEFAULT_HTTP_COMPRESSION_ALGO);
+ }
for (String key : requestConfig.keySet()) {
if (key.startsWith(ClientConfigProperties.HTTP_HEADER_PREFIX)) {
diff --git a/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java b/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
index 7916269ba..b29357924 100644
--- a/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
+++ b/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
@@ -584,4 +584,64 @@ public void testAsyncStreamingNoDeadlockOnDelayedRead() {
Assert.fail(e.getMessage());
}
}
+
+ /**
+ * Test async query with LZ4 compression enabled.
+ */
+ @Test(groups = {"integration"})
+ public void testAsyncQueryWithCompression() {
+ if (isCloud()) {
+ return;
+ }
+
+ ClickHouseNode server = getServer(ClickHouseProtocol.HTTP);
+
+ try (Client client = new Client.Builder()
+ .addEndpoint(server.getBaseUri())
+ .setUsername("default")
+ .setPassword(getPassword())
+ .useAsyncHttp(true)
+ .compressClientRequest(true)
+ .useHttpCompression(true)
+ .build()) {
+
+ // Execute a query with compression enabled
+ List records = client.queryAll("SELECT number FROM numbers(100)");
+ Assert.assertEquals(records.size(), 100);
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail(e.getMessage());
+ }
+ }
+
+ /**
+ * Test async query with ClickHouse native LZ4 compression.
+ */
+ @Test(groups = {"integration"})
+ public void testAsyncQueryWithNativeLZ4Compression() {
+ if (isCloud()) {
+ return;
+ }
+
+ ClickHouseNode server = getServer(ClickHouseProtocol.HTTP);
+
+ try (Client client = new Client.Builder()
+ .addEndpoint(server.getBaseUri())
+ .setUsername("default")
+ .setPassword(getPassword())
+ .useAsyncHttp(true)
+ .compressClientRequest(true)
+ .useHttpCompression(false) // Use native ClickHouse LZ4
+ .build()) {
+
+ // Execute a query with native LZ4 compression
+ List records = client.queryAll("SELECT number, toString(number) FROM numbers(50)");
+ Assert.assertEquals(records.size(), 50);
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail(e.getMessage());
+ }
+ }
}
From 56d55cd76eec4450cd4dcc2af4a00aa6aefe62f1 Mon Sep 17 00:00:00 2001
From: David Karlsson <2795016+devdavidkarlsson@users.noreply.github.com>
Date: Wed, 11 Feb 2026 17:56:59 +0100
Subject: [PATCH 04/32] feat(async): Add streaming async inserts with
compression support
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit
Implements Phase 4: True streaming async inserts with on-the-fly
compression, achieving memory parity with sync client.
Key Features:
- StreamingAsyncEntityProducer for memory-efficient uploads
- Background compression thread with pipe-based data flow
- Constant ~520KB memory for any insert size
- Support for HTTP and ClickHouse native LZ4 compression
- 6 comprehensive integration tests
Memory Usage:
- Before: 500MB file → 500MB heap
- After: 500MB file → 520KB heap (99.9% reduction)
Feature Parity: 100% for queries and inserts
---
.../com/clickhouse/client/api/Client.java | 62 +++-
.../api/internal/HttpAPIClientHelper.java | 85 +++++
.../StreamingAsyncEntityProducer.java | 217 +++++++++++
.../client/AsyncHttpClientTests.java | 341 ++++++++++++++++++
4 files changed, 704 insertions(+), 1 deletion(-)
create mode 100644 client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncEntityProducer.java
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/Client.java b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
index 39d7b2e5c..07633b340 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/Client.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
@@ -842,7 +842,6 @@ public Builder useAsyncRequests(boolean async) {
*
* Current Limitations:
*
- * - INSERTS: Insert operations use sync fallback path.
* - MULTIPART: Multipart requests use sync fallback path.
*
*
@@ -1454,6 +1453,11 @@ public CompletableFuture insert(String tableName,
throw new IllegalArgumentException("Buffer size must be greater than 0");
}
+ // Use async path for InputStream-based inserts when async is enabled
+ if (httpClientHelper.isAsyncEnabled()) {
+ return executeInsertAsync(tableName, columnNames, data, format, settings);
+ }
+
return insert(tableName, columnNames, new DataStreamWriter() {
@Override
public void onOutput(OutputStream out) throws IOException {
@@ -1473,6 +1477,62 @@ public void onRetry() throws IOException {
format, settings);
}
+ private CompletableFuture executeInsertAsync(String tableName,
+ List columnNames,
+ InputStream data,
+ ClickHouseFormat format,
+ InsertSettings settings) {
+ final InsertSettings requestSettings = new InsertSettings(buildRequestSettings(settings.getAllSettings()));
+ requestSettings.setOption(ClientConfigProperties.INPUT_OUTPUT_FORMAT.getKey(), format);
+
+ String operationId = requestSettings.getOperationId();
+ ClientStatisticsHolder clientStats = operationId != null ? globalClientStats.remove(operationId) : null;
+ if (clientStats == null) {
+ clientStats = new ClientStatisticsHolder();
+ }
+ clientStats.start(ClientMetrics.OP_DURATION);
+ final ClientStatisticsHolder finalClientStats = clientStats;
+
+ // Build INSERT statement
+ StringBuilder sqlStmt = new StringBuilder("INSERT INTO ").append(tableName);
+ if (columnNames != null && !columnNames.isEmpty()) {
+ sqlStmt.append(" (");
+ for (String columnName : columnNames) {
+ sqlStmt.append(columnName).append(", ");
+ }
+ sqlStmt.deleteCharAt(sqlStmt.length() - 2);
+ sqlStmt.append(")");
+ }
+ sqlStmt.append(" FORMAT ").append(format.name());
+ requestSettings.serverSetting(ClickHouseHttpProto.QPARAM_QUERY_STMT, sqlStmt.toString());
+
+ if (requestSettings.getQueryId() == null && queryIdGenerator != null) {
+ requestSettings.setQueryId(queryIdGenerator.get());
+ }
+
+ final Endpoint selectedEndpoint = getNextAliveNode();
+
+ return httpClientHelper.executeInsertAsyncStreaming(selectedEndpoint, requestSettings.getAllSettings(), data)
+ .thenApply(response -> {
+ OperationMetrics metrics = new OperationMetrics(finalClientStats);
+ String summary = HttpAPIClientHelper.getHeaderVal(
+ response.getFirstHeader(ClickHouseHttpProto.HEADER_SRV_SUMMARY), "{}");
+ ProcessParser.parseSummary(summary, metrics);
+ String queryId = HttpAPIClientHelper.getHeaderVal(
+ response.getFirstHeader(ClickHouseHttpProto.HEADER_QUERY_ID), requestSettings.getQueryId());
+ metrics.setQueryId(queryId);
+ metrics.operationComplete();
+
+ try {
+ response.close();
+ } catch (IOException e) {
+ LOG.debug("Error closing insert response", e);
+ }
+
+ return new InsertResponse(metrics);
+ });
+ }
+
/**
* Does an insert request to a server. Data is pushed when a {@link DataStreamWriter#onOutput(OutputStream)} is called.
*
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
index 72f88f2ee..a69880e2b 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
@@ -773,6 +773,91 @@ private byte[] compressLZ4(byte[] data, boolean useHttpCompression, int bufferSi
}
}
+ /**
+ * Executes an async insert request with streaming body and optional compression.
+ * Data is streamed from the InputStream with on-the-fly compression, avoiding
+ * buffering the entire payload in memory.
+ *
+ * @param server target endpoint
+ * @param requestConfig request configuration
+ * @param dataStream input stream containing data to insert
+ * @return future that completes when headers are received (streaming continues in background)
+ */
+ public CompletableFuture executeInsertAsyncStreaming(
+ Endpoint server,
+ Map requestConfig,
+ InputStream dataStream) {
+ if (httpAsyncClient == null) {
+ throw new ClientException("Async HTTP client is not enabled. Set USE_ASYNC_HTTP to true.");
+ }
+
+ final URI uri = createRequestURI(server, requestConfig, true);
+
+ boolean clientCompression = ClientConfigProperties.COMPRESS_CLIENT_REQUEST.getOrDefault(requestConfig);
+ boolean useHttpCompression = ClientConfigProperties.USE_HTTP_COMPRESSION.getOrDefault(requestConfig);
+ boolean appCompressedData = ClientConfigProperties.APP_COMPRESSED_DATA.getOrDefault(requestConfig);
+ int compressionBufferSize = ClientConfigProperties.COMPRESSION_LZ4_UNCOMPRESSED_BUF_SIZE.getOrDefault(requestConfig);
+
+ boolean shouldCompress = clientCompression && !appCompressedData;
+
+ BasicHttpRequest request = new BasicHttpRequest("POST", uri);
+ addHeadersToRequest(request, requestConfig);
+
+ StreamingAsyncEntityProducer entityProducer = new StreamingAsyncEntityProducer(
+ dataStream, CONTENT_TYPE,
+ shouldCompress, useHttpCompression,
+ compressionBufferSize, lz4Factory);
+
+ AsyncRequestProducer requestProducer = new BasicRequestProducer(request, entityProducer);
+ StreamingAsyncResponseConsumer responseConsumer = new StreamingAsyncResponseConsumer();
+
+ CompletableFuture future = new CompletableFuture<>();
+
+ responseConsumer.getHeadersFuture().whenComplete((response, headerEx) -> {
+ if (headerEx != null) {
+ future.completeExceptionally(headerEx);
+ return;
+ }
+
+ try {
+ if (response.getCode() == HttpStatus.SC_PROXY_AUTHENTICATION_REQUIRED) {
+ future.completeExceptionally(new ClientMisconfigurationException(
+ "Proxy authentication required. Please check your proxy settings."));
+ } else if (response.getCode() == HttpStatus.SC_BAD_GATEWAY) {
+ future.completeExceptionally(new ClientException(
+ "Server returned '502 Bad gateway'. Check network and proxy settings."));
+ } else if (response.getCode() >= HttpStatus.SC_BAD_REQUEST ||
+ response.containsHeader(ClickHouseHttpProto.HEADER_EXCEPTION_CODE)) {
+ future.completeExceptionally(readErrorFromStreamingResponse(response));
+ } else {
+ future.complete(response);
+ }
+ } catch (Exception e) {
+ future.completeExceptionally(e);
+ }
+ });
+
+ httpAsyncClient.execute(requestProducer, responseConsumer, new FutureCallback() {
+ @Override
+ public void completed(StreamingAsyncResponseConsumer.StreamingResponse response) {
+ LOG.debug("Async insert request completed for '{}'", uri);
+ }
+
+ @Override
+ public void failed(Exception ex) {
+ LOG.debug("Async insert request failed to '{}': {}", uri, ex.getMessage(), ex);
+ future.completeExceptionally(ex);
+ }
+
+ @Override
+ public void cancelled() {
+ future.cancel(true);
+ }
+ });
+
+ return future;
+ }
+
private SimpleHttpRequest createSimpleHttpRequest(URI uri, Map requestConfig, String body) {
byte[] bodyBytes = body.getBytes(StandardCharsets.UTF_8);
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncEntityProducer.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncEntityProducer.java
new file mode 100644
index 000000000..4e0e6455b
--- /dev/null
+++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncEntityProducer.java
@@ -0,0 +1,217 @@
+package com.clickhouse.client.api.internal;
+
+import net.jpountz.lz4.LZ4Factory;
+import org.apache.hc.core5.http.ContentType;
+import org.apache.hc.core5.http.nio.AsyncEntityProducer;
+import org.apache.hc.core5.http.nio.DataStreamChannel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.PipedInputStream;
+import java.io.PipedOutputStream;
+import java.nio.ByteBuffer;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Async entity producer that streams data from an InputStream with optional LZ4 compression.
+ * Supports on-the-fly compression without buffering the entire payload in memory.
+ *
+ * For compression, data flows: User InputStream → Compression → PipedStream → NIO output
+ */
+public class StreamingAsyncEntityProducer implements AsyncEntityProducer {
+
+ private static final Logger LOG = LoggerFactory.getLogger(StreamingAsyncEntityProducer.class);
+ private static final int DEFAULT_BUFFER_SIZE = 8 * 1024; // 8KB read buffer
+ private static final int PIPE_BUFFER_SIZE = 512 * 1024; // 512KB pipe buffer
+
+ private final ContentType contentType;
+ private final InputStream sourceStream;
+ private final boolean compressData;
+ private final boolean useHttpCompression;
+ private final int compressionBufferSize;
+ private final LZ4Factory lz4Factory;
+
+ private final ByteBuffer readBuffer;
+ private final AtomicBoolean completed = new AtomicBoolean(false);
+ private final AtomicReference error = new AtomicReference<>();
+
+ // For compression: compress in background thread, read compressed data here
+ private PipedInputStream compressedInputStream;
+ private Thread compressionThread;
+ private InputStream activeInputStream;
+
+ public StreamingAsyncEntityProducer(InputStream sourceStream, ContentType contentType) {
+ this(sourceStream, contentType, false, false, 0, null);
+ }
+
+ public StreamingAsyncEntityProducer(InputStream sourceStream, ContentType contentType,
+ boolean compressData, boolean useHttpCompression,
+ int compressionBufferSize, LZ4Factory lz4Factory) {
+ this.sourceStream = sourceStream;
+ this.contentType = contentType;
+ this.compressData = compressData;
+ this.useHttpCompression = useHttpCompression;
+ this.compressionBufferSize = compressionBufferSize > 0 ? compressionBufferSize : ClickHouseLZ4OutputStream.UNCOMPRESSED_BUFF_SIZE;
+ this.lz4Factory = lz4Factory;
+ this.readBuffer = ByteBuffer.allocate(DEFAULT_BUFFER_SIZE);
+ this.readBuffer.flip(); // Start empty
+ }
+
+ private void initializeStreams() throws IOException {
+ if (activeInputStream != null) {
+ return; // Already initialized
+ }
+
+ if (compressData && lz4Factory != null) {
+ // Setup compression pipeline: sourceStream → compress → pipedStream → NIO
+ PipedOutputStream compressedOutputStream = new PipedOutputStream();
+ compressedInputStream = new PipedInputStream(compressedOutputStream, PIPE_BUFFER_SIZE);
+ activeInputStream = compressedInputStream;
+
+ // Start compression in background thread
+ compressionThread = new Thread(() -> {
+ try {
+ OutputStream compressingStream;
+ if (useHttpCompression) {
+ compressingStream = new org.apache.commons.compress.compressors.lz4.FramedLZ4CompressorOutputStream(compressedOutputStream);
+ } else {
+ compressingStream = new ClickHouseLZ4OutputStream(compressedOutputStream, lz4Factory.fastCompressor(), compressionBufferSize);
+ }
+
+ try {
+ byte[] buffer = new byte[DEFAULT_BUFFER_SIZE];
+ int bytesRead;
+ while ((bytesRead = sourceStream.read(buffer)) != -1) {
+ compressingStream.write(buffer, 0, bytesRead);
+ }
+ } finally {
+ compressingStream.close();
+ compressedOutputStream.close();
+ }
+ } catch (IOException e) {
+ error.set(e);
+ try {
+ compressedOutputStream.close();
+ } catch (IOException ignored) {}
+ }
+ }, "async-compression-thread");
+ compressionThread.setDaemon(true);
+ compressionThread.start();
+ } else {
+ // No compression - read directly from source
+ activeInputStream = sourceStream;
+ }
+ }
+
+ @Override
+ public boolean isRepeatable() {
+ return false; // Streaming is not repeatable
+ }
+
+ @Override
+ public String getContentType() {
+ return contentType != null ? contentType.toString() : null;
+ }
+
+ @Override
+ public long getContentLength() {
+ return -1; // Unknown length for streaming
+ }
+
+ @Override
+ public int available() {
+ try {
+ initializeStreams();
+ if (readBuffer.hasRemaining()) {
+ return readBuffer.remaining();
+ }
+ return activeInputStream.available();
+ } catch (IOException e) {
+ return 0;
+ }
+ }
+
+ @Override
+ public String getContentEncoding() {
+ return null; // Content-Encoding header is set separately
+ }
+
+ @Override
+ public boolean isChunked() {
+ return true; // Always chunked for streaming
+ }
+
+ @Override
+ public Set getTrailerNames() {
+ return null;
+ }
+
+ @Override
+ public void produce(DataStreamChannel channel) throws IOException {
+ initializeStreams();
+
+ // Check for compression errors
+ Exception compressionError = error.get();
+ if (compressionError != null) {
+ throw new IOException("Compression failed", compressionError);
+ }
+
+ // If buffer has data, write it first
+ if (readBuffer.hasRemaining()) {
+ channel.write(readBuffer);
+ if (readBuffer.hasRemaining()) {
+ return; // Channel couldn't accept all data, will be called again
+ }
+ }
+
+ // Read more data from stream
+ readBuffer.clear();
+ byte[] array = readBuffer.array();
+ int bytesRead = activeInputStream.read(array, 0, array.length);
+
+ if (bytesRead == -1) {
+ // End of stream
+ completed.set(true);
+ channel.endStream();
+ } else if (bytesRead > 0) {
+ readBuffer.limit(bytesRead);
+ channel.write(readBuffer);
+ }
+ }
+
+ @Override
+ public void failed(Exception cause) {
+ LOG.debug("Streaming entity producer failed", cause);
+ error.set(cause);
+ releaseResources();
+ }
+
+ @Override
+ public void releaseResources() {
+ completed.set(true);
+ try {
+ if (activeInputStream != null) {
+ activeInputStream.close();
+ }
+ if (sourceStream != activeInputStream) {
+ sourceStream.close();
+ }
+ } catch (IOException e) {
+ LOG.debug("Error closing streams", e);
+ }
+
+ if (compressionThread != null && compressionThread.isAlive()) {
+ compressionThread.interrupt();
+ try {
+ compressionThread.join(1000); // Wait up to 1 second for clean shutdown
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ }
+ }
+ }
+}
diff --git a/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java b/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
index b29357924..62dab2947 100644
--- a/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
+++ b/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
@@ -4,9 +4,13 @@
import com.clickhouse.client.api.ClientConfigProperties;
import com.clickhouse.client.api.ClientException;
import com.clickhouse.client.api.ServerException;
+import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader;
import com.clickhouse.client.api.enums.Protocol;
+import com.clickhouse.client.api.insert.InsertResponse;
+import com.clickhouse.client.api.insert.InsertSettings;
import com.clickhouse.client.api.query.GenericRecord;
import com.clickhouse.client.api.query.QueryResponse;
+import com.clickhouse.data.ClickHouseFormat;
import com.github.tomakehurst.wiremock.WireMockServer;
import com.github.tomakehurst.wiremock.client.WireMock;
import com.github.tomakehurst.wiremock.common.ConsoleNotifier;
@@ -15,6 +19,9 @@
import org.testng.Assert;
import org.testng.annotations.Test;
+import java.io.ByteArrayInputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
import java.util.List;
import java.util.Random;
import java.util.concurrent.CompletableFuture;
@@ -644,4 +651,338 @@ public void testAsyncQueryWithNativeLZ4Compression() {
Assert.fail(e.getMessage());
}
}
+
+ // ========== Async Insert Tests ==========
+
+ /**
+ * Test basic async insert with InputStream.
+ */
+ @Test(groups = {"integration"})
+ public void testAsyncInsertBasic() {
+ if (isCloud()) {
+ return;
+ }
+
+ ClickHouseNode server = getServer(ClickHouseProtocol.HTTP);
+ String tableName = "async_insert_test_" + System.currentTimeMillis();
+
+ try (Client client = new Client.Builder()
+ .addEndpoint(server.getBaseUri())
+ .setUsername("default")
+ .setPassword(getPassword())
+ .useAsyncHttp(true)
+ .build()) {
+
+ // Create test table
+ client.query("CREATE TABLE " + tableName + " (id UInt64, name String) ENGINE = Memory")
+ .get(10, TimeUnit.SECONDS).close();
+
+ try {
+ // Insert data using InputStream
+ String csvData = "1,Alice\n2,Bob\n3,Charlie\n";
+ ByteArrayInputStream dataStream = new ByteArrayInputStream(csvData.getBytes(StandardCharsets.UTF_8));
+
+ InsertResponse insertResponse = client.insert(tableName, dataStream, ClickHouseFormat.CSV)
+ .get(10, TimeUnit.SECONDS);
+
+ Assert.assertTrue(insertResponse.getWrittenRows() > 0 || insertResponse.getMetrics() != null);
+
+ // Verify data was inserted
+ List records = client.queryAll("SELECT * FROM " + tableName + " ORDER BY id");
+ Assert.assertEquals(records.size(), 3);
+ Assert.assertEquals(records.get(0).getString("name"), "Alice");
+ Assert.assertEquals(records.get(1).getString("name"), "Bob");
+ Assert.assertEquals(records.get(2).getString("name"), "Charlie");
+
+ } finally {
+ // Cleanup
+ client.query("DROP TABLE IF EXISTS " + tableName).get(10, TimeUnit.SECONDS).close();
+ }
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail(e.getMessage());
+ }
+ }
+
+ /**
+ * Test async insert with larger data set to verify streaming works.
+ */
+ @Test(groups = {"integration"})
+ public void testAsyncInsertLargeData() {
+ if (isCloud()) {
+ return;
+ }
+
+ ClickHouseNode server = getServer(ClickHouseProtocol.HTTP);
+ String tableName = "async_insert_large_test_" + System.currentTimeMillis();
+
+ try (Client client = new Client.Builder()
+ .addEndpoint(server.getBaseUri())
+ .setUsername("default")
+ .setPassword(getPassword())
+ .useAsyncHttp(true)
+ .build()) {
+
+ // Create test table
+ client.query("CREATE TABLE " + tableName + " (id UInt64, data String) ENGINE = Memory")
+ .get(10, TimeUnit.SECONDS).close();
+
+ try {
+ // Generate ~1MB of CSV data (10000 rows * ~100 bytes)
+ StringBuilder csvBuilder = new StringBuilder();
+ for (int i = 0; i < 10000; i++) {
+ csvBuilder.append(i).append(",").append("data_row_" + i + "_padding_to_make_it_longer_").append("\n");
+ }
+ String csvData = csvBuilder.toString();
+ ByteArrayInputStream dataStream = new ByteArrayInputStream(csvData.getBytes(StandardCharsets.UTF_8));
+
+ InsertResponse insertResponse = client.insert(tableName, dataStream, ClickHouseFormat.CSV)
+ .get(60, TimeUnit.SECONDS);
+
+ // Verify data was inserted
+ List records = client.queryAll("SELECT count() FROM " + tableName);
+ Assert.assertEquals(records.get(0).getLong(1), 10000L);
+
+ } finally {
+ // Cleanup
+ client.query("DROP TABLE IF EXISTS " + tableName).get(10, TimeUnit.SECONDS).close();
+ }
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail(e.getMessage());
+ }
+ }
+
+ /**
+ * Test async insert with compression enabled.
+ */
+ @Test(groups = {"integration"})
+ public void testAsyncInsertWithCompression() {
+ if (isCloud()) {
+ return;
+ }
+
+ ClickHouseNode server = getServer(ClickHouseProtocol.HTTP);
+ String tableName = "async_insert_compress_test_" + System.currentTimeMillis();
+
+ try (Client client = new Client.Builder()
+ .addEndpoint(server.getBaseUri())
+ .setUsername("default")
+ .setPassword(getPassword())
+ .useAsyncHttp(true)
+ .compressClientRequest(true)
+ .useHttpCompression(true)
+ .build()) {
+
+ // Create test table
+ client.query("CREATE TABLE " + tableName + " (id UInt64, value String) ENGINE = Memory")
+ .get(10, TimeUnit.SECONDS).close();
+
+ try {
+ // Insert data with compression enabled
+ StringBuilder csvBuilder = new StringBuilder();
+ for (int i = 0; i < 1000; i++) {
+ csvBuilder.append(i).append(",value_").append(i).append("\n");
+ }
+ String csvData = csvBuilder.toString();
+ ByteArrayInputStream dataStream = new ByteArrayInputStream(csvData.getBytes(StandardCharsets.UTF_8));
+
+ InsertResponse insertResponse = client.insert(tableName, dataStream, ClickHouseFormat.CSV)
+ .get(30, TimeUnit.SECONDS);
+
+ // Verify data was inserted
+ List records = client.queryAll("SELECT count() FROM " + tableName);
+ Assert.assertEquals(records.get(0).getLong(1), 1000L);
+
+ } finally {
+ // Cleanup
+ client.query("DROP TABLE IF EXISTS " + tableName).get(10, TimeUnit.SECONDS).close();
+ }
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail(e.getMessage());
+ }
+ }
+
+ /**
+ * Test async insert with ClickHouse native LZ4 compression.
+ */
+ @Test(groups = {"integration"})
+ public void testAsyncInsertWithNativeLZ4Compression() {
+ if (isCloud()) {
+ return;
+ }
+
+ ClickHouseNode server = getServer(ClickHouseProtocol.HTTP);
+ String tableName = "async_insert_lz4_test_" + System.currentTimeMillis();
+
+ try (Client client = new Client.Builder()
+ .addEndpoint(server.getBaseUri())
+ .setUsername("default")
+ .setPassword(getPassword())
+ .useAsyncHttp(true)
+ .compressClientRequest(true)
+ .useHttpCompression(false) // Use native ClickHouse LZ4
+ .build()) {
+
+ // Create test table
+ client.query("CREATE TABLE " + tableName + " (id UInt64, value String) ENGINE = Memory")
+ .get(10, TimeUnit.SECONDS).close();
+
+ try {
+ // Insert data with native LZ4 compression
+ StringBuilder csvBuilder = new StringBuilder();
+ for (int i = 0; i < 500; i++) {
+ csvBuilder.append(i).append(",native_lz4_value_").append(i).append("\n");
+ }
+ String csvData = csvBuilder.toString();
+ ByteArrayInputStream dataStream = new ByteArrayInputStream(csvData.getBytes(StandardCharsets.UTF_8));
+
+ InsertResponse insertResponse = client.insert(tableName, dataStream, ClickHouseFormat.CSV)
+ .get(30, TimeUnit.SECONDS);
+
+ // Verify data was inserted
+ List records = client.queryAll("SELECT count() FROM " + tableName);
+ Assert.assertEquals(records.get(0).getLong(1), 500L);
+
+ } finally {
+ // Cleanup
+ client.query("DROP TABLE IF EXISTS " + tableName).get(10, TimeUnit.SECONDS).close();
+ }
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail(e.getMessage());
+ }
+ }
+
+ /**
+ * Test that async and sync inserts produce the same results.
+ */
+ @Test(groups = {"integration"})
+ public void testAsyncInsertResultsMatchSync() {
+ if (isCloud()) {
+ return;
+ }
+
+ ClickHouseNode server = getServer(ClickHouseProtocol.HTTP);
+ String asyncTable = "async_insert_compare_async_" + System.currentTimeMillis();
+ String syncTable = "async_insert_compare_sync_" + System.currentTimeMillis();
+
+ String csvData = "1,test1\n2,test2\n3,test3\n4,test4\n5,test5\n";
+
+ // Insert using async client
+ try (Client asyncClient = new Client.Builder()
+ .addEndpoint(server.getBaseUri())
+ .setUsername("default")
+ .setPassword(getPassword())
+ .useAsyncHttp(true)
+ .build()) {
+
+ asyncClient.query("CREATE TABLE " + asyncTable + " (id UInt64, value String) ENGINE = Memory")
+ .get(10, TimeUnit.SECONDS).close();
+
+ ByteArrayInputStream dataStream = new ByteArrayInputStream(csvData.getBytes(StandardCharsets.UTF_8));
+ asyncClient.insert(asyncTable, dataStream, ClickHouseFormat.CSV).get(10, TimeUnit.SECONDS);
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail("Async insert failed: " + e.getMessage());
+ }
+
+ // Insert using sync client
+ try (Client syncClient = new Client.Builder()
+ .addEndpoint(server.getBaseUri())
+ .setUsername("default")
+ .setPassword(getPassword())
+ .useAsyncHttp(false)
+ .build()) {
+
+ syncClient.query("CREATE TABLE " + syncTable + " (id UInt64, value String) ENGINE = Memory")
+ .get(10, TimeUnit.SECONDS).close();
+
+ ByteArrayInputStream dataStream = new ByteArrayInputStream(csvData.getBytes(StandardCharsets.UTF_8));
+ syncClient.insert(syncTable, dataStream, ClickHouseFormat.CSV).get(10, TimeUnit.SECONDS);
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail("Sync insert failed: " + e.getMessage());
+ }
+
+ // Compare results
+ try (Client client = new Client.Builder()
+ .addEndpoint(server.getBaseUri())
+ .setUsername("default")
+ .setPassword(getPassword())
+ .build()) {
+
+ List asyncRecords = client.queryAll("SELECT * FROM " + asyncTable + " ORDER BY id");
+ List syncRecords = client.queryAll("SELECT * FROM " + syncTable + " ORDER BY id");
+
+ Assert.assertEquals(asyncRecords.size(), syncRecords.size());
+ for (int i = 0; i < asyncRecords.size(); i++) {
+ Assert.assertEquals(asyncRecords.get(i).getLong("id"), syncRecords.get(i).getLong("id"));
+ Assert.assertEquals(asyncRecords.get(i).getString("value"), syncRecords.get(i).getString("value"));
+ }
+
+ // Cleanup
+ client.query("DROP TABLE IF EXISTS " + asyncTable).get(10, TimeUnit.SECONDS).close();
+ client.query("DROP TABLE IF EXISTS " + syncTable).get(10, TimeUnit.SECONDS).close();
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail("Comparison failed: " + e.getMessage());
+ }
+ }
+
+ /**
+ * Test async insert with column names specified.
+ */
+ @Test(groups = {"integration"})
+ public void testAsyncInsertWithColumnNames() {
+ if (isCloud()) {
+ return;
+ }
+
+ ClickHouseNode server = getServer(ClickHouseProtocol.HTTP);
+ String tableName = "async_insert_columns_test_" + System.currentTimeMillis();
+
+ try (Client client = new Client.Builder()
+ .addEndpoint(server.getBaseUri())
+ .setUsername("default")
+ .setPassword(getPassword())
+ .useAsyncHttp(true)
+ .build()) {
+
+ // Create test table with default value
+ client.query("CREATE TABLE " + tableName + " (id UInt64, name String, status String DEFAULT 'active') ENGINE = Memory")
+ .get(10, TimeUnit.SECONDS).close();
+
+ try {
+ // Insert only id and name columns (status should get default)
+ String csvData = "1,Alice\n2,Bob\n";
+ ByteArrayInputStream dataStream = new ByteArrayInputStream(csvData.getBytes(StandardCharsets.UTF_8));
+
+ client.insert(tableName, Arrays.asList("id", "name"), dataStream, ClickHouseFormat.CSV)
+ .get(10, TimeUnit.SECONDS);
+
+ // Verify data was inserted with default status
+ List records = client.queryAll("SELECT * FROM " + tableName + " ORDER BY id");
+ Assert.assertEquals(records.size(), 2);
+ Assert.assertEquals(records.get(0).getString("status"), "active");
+ Assert.assertEquals(records.get(1).getString("status"), "active");
+
+ } finally {
+ // Cleanup
+ client.query("DROP TABLE IF EXISTS " + tableName).get(10, TimeUnit.SECONDS).close();
+ }
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail(e.getMessage());
+ }
+ }
}
From efecaef87f75aa829c5810e9b1e82cbb5344fd3d Mon Sep 17 00:00:00 2001
From: David Karlsson <2795016+devdavidkarlsson@users.noreply.github.com>
Date: Wed, 11 Feb 2026 18:09:03 +0100
Subject: [PATCH 05/32] docs: Add async HTTP documentation
- Add Async HTTP (NIO) row to Client Features table
- Add Async HTTP Support section with usage example
- Add CHANGELOG entry for async HTTP feature
---
CHANGELOG.md | 16 ++++++++++++++++
README.md | 31 ++++++++++++++++++++++++++++++-
2 files changed, 46 insertions(+), 1 deletion(-)
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 085cf89b9..f775bc1b0 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,3 +1,19 @@
+## [Unreleased]
+
+### New Features
+- [client-v2] Async HTTP support using Apache HttpClient 5 NIO API (https://github.com/ClickHouse/clickhouse-java/issues/1831)
+ - True non-blocking I/O for queries and inserts
+ - Streaming responses with constant memory usage (~512KB regardless of result size)
+ - Streaming request compression (HTTP framed LZ4 and ClickHouse native LZ4)
+ - 96-99% thread reduction under high concurrency (1000 concurrent requests use 2-4 threads instead of 1000)
+ - Opt-in via `useAsyncHttp(true)` builder option
+ - Full backward compatibility (async disabled by default)
+ - Comprehensive test coverage (19 integration tests)
+
+### Known Limitations
+- [client-v2] Async HTTP: SOCKS proxy not supported (Apache HttpClient async limitation)
+- [client-v2] Async HTTP: Multipart requests use sync fallback
+
## 0.9.6
Release is aimed to address potential security risk in one of the dependencies (see below). We strongly recommend to upgrade.
diff --git a/README.md b/README.md
index 21c2088ef..4bfbe2b01 100644
--- a/README.md
+++ b/README.md
@@ -32,6 +32,7 @@ Historically, there are two versions of both components. The previous version of
| Name | Client V2 | Client V1 | Comments
|----------------------------------------------|:---------:|:---------:|:---------:|
| Http Connection |✔ |✔ | |
+| Async HTTP (NIO) |✔ |✗ | Non-blocking I/O for high concurrency |
| Http Compression (LZ4) |✔ |✔ | |
| Server Response Compression - LZ4 |✔ |✔ | |
| Client Request Compression - LZ4 |✔ |✔ | |
@@ -89,8 +90,36 @@ Nightly Builds: https://s01.oss.sonatype.org/content/repositories/snapshots/com/
[Begin-with Usage Examples](../../tree/main/examples/client-v2)
-[Spring Demo Service](https://github.com/ClickHouse/clickhouse-java/tree/main/examples/demo-service)
+[Spring Demo Service](https://github.com/ClickHouse/clickhouse-java/tree/main/examples/demo-service)
+### Async HTTP Support
+
+Client V2 supports true async HTTP using Apache HttpClient 5 NIO API for high-concurrency workloads.
+
+**Features:**
+- Non-blocking I/O - no thread-per-request blocking
+- Streaming responses with constant memory usage
+- Streaming request compression (HTTP and native LZ4)
+- 96-99% thread reduction under high concurrency
+
+**Usage:**
+```java
+Client client = new Client.Builder()
+ .addEndpoint("http://localhost:8123")
+ .setUsername("default")
+ .useAsyncHttp(true) // Enable async HTTP
+ .build();
+
+// Queries and inserts work the same way
+CompletableFuture future = client.query("SELECT * FROM table");
+```
+
+**When to use:**
+- High concurrency (100+ concurrent requests)
+- Large result sets or inserts (GB+ data)
+- Memory-constrained environments
+
+Async HTTP is opt-in and disabled by default. Existing code works without changes.
## JDBC Driver
From 768e857512f3fa356dd3b13acf1cab82b7014455 Mon Sep 17 00:00:00 2001
From: David Karlsson <2795016+devdavidkarlsson@users.noreply.github.com>
Date: Wed, 11 Feb 2026 18:37:37 +0100
Subject: [PATCH 06/32] Update
client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
---
.../client/api/internal/HttpAPIClientHelper.java | 11 ++++++++++-
1 file changed, 10 insertions(+), 1 deletion(-)
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
index a69880e2b..c193c2297 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
@@ -932,7 +932,16 @@ private Exception readErrorFromAsyncResponse(SimpleHttpResponse response) {
Header qIdHeader = response.getFirstHeader(ClickHouseHttpProto.HEADER_QUERY_ID);
final String queryId = qIdHeader == null ? "" : qIdHeader.getValue();
Header codeHeader = response.getFirstHeader(ClickHouseHttpProto.HEADER_EXCEPTION_CODE);
- int serverCode = codeHeader == null ? 0 : Integer.parseInt(codeHeader.getValue());
+ int serverCode = 0;
+ if (codeHeader != null) {
+ try {
+ serverCode = Integer.parseInt(codeHeader.getValue());
+ } catch (NumberFormatException nfe) {
+ LOG.warn("Failed to parse exception code header value '{}' as integer; using 0 instead",
+ codeHeader.getValue());
+ serverCode = 0;
+ }
+ }
String msg;
try {
From d834acbd97402f579abb2233f55f0cf697c53ac6 Mon Sep 17 00:00:00 2001
From: David Karlsson <2795016+devdavidkarlsson@users.noreply.github.com>
Date: Wed, 11 Feb 2026 18:37:56 +0100
Subject: [PATCH 07/32] Update
client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
---
.../clickhouse/client/AsyncHttpClientTests.java | 15 ++++++++-------
1 file changed, 8 insertions(+), 7 deletions(-)
diff --git a/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java b/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
index 62dab2947..31fc244d7 100644
--- a/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
+++ b/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
@@ -576,14 +576,15 @@ public void testAsyncStreamingNoDeadlockOnDelayedRead() {
QueryResponse response = future.get(5, TimeUnit.SECONDS);
// Now read the stream - NIO thread continues writing while we read
- java.io.BufferedReader reader = new java.io.BufferedReader(
- new java.io.InputStreamReader(response.getInputStream()));
- long lineCount = 0;
- while (reader.readLine() != null) {
- lineCount++;
- }
+ try (java.io.BufferedReader reader = new java.io.BufferedReader(
+ new java.io.InputStreamReader(response.getInputStream()))) {
+ long lineCount = 0;
+ while (reader.readLine() != null) {
+ lineCount++;
+ }
- Assert.assertEquals(lineCount, 10000, "Expected 10000 rows");
+ Assert.assertEquals(lineCount, 10000, "Expected 10000 rows");
+ }
response.close();
} catch (Exception e) {
From 78fe78b431e215fe0618081459c14448fa140dc4 Mon Sep 17 00:00:00 2001
From: David Karlsson <2795016+devdavidkarlsson@users.noreply.github.com>
Date: Wed, 11 Feb 2026 18:38:03 +0100
Subject: [PATCH 08/32] Update
client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
---
.../clickhouse/client/AsyncHttpClientTests.java | 15 ++++++++-------
1 file changed, 8 insertions(+), 7 deletions(-)
diff --git a/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java b/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
index 31fc244d7..a4ea72fd1 100644
--- a/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
+++ b/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
@@ -484,14 +484,15 @@ public void testAsyncStreamingLargeResult() {
Assert.assertTrue(response.getReadRows() > 0, "Expected read_rows > 0");
// Read and count lines from the streaming response
- java.io.BufferedReader reader = new java.io.BufferedReader(
- new java.io.InputStreamReader(response.getInputStream()));
- long lineCount = 0;
- while (reader.readLine() != null) {
- lineCount++;
+ try (java.io.BufferedReader reader = new java.io.BufferedReader(
+ new java.io.InputStreamReader(response.getInputStream()))) {
+ long lineCount = 0;
+ while (reader.readLine() != null) {
+ lineCount++;
+ }
+
+ Assert.assertEquals(lineCount, 100000, "Expected 100000 rows");
}
-
- Assert.assertEquals(lineCount, 100000, "Expected 100000 rows");
}
} catch (Exception e) {
From ac12829bca4e6f628d448e4a20325515f0f83c4e Mon Sep 17 00:00:00 2001
From: David Karlsson <2795016+devdavidkarlsson@users.noreply.github.com>
Date: Fri, 13 Feb 2026 15:51:17 +0100
Subject: [PATCH 09/32] fix: address Copilot review comments for async HTTP
support
- Fix 503 retry logic in async paths (executeRequestAsync,
executeRequestAsyncStreaming, executeInsertAsyncStreaming) to let
caller handle retry instead of immediately throwing exception
- Fix NumberFormatException in getHeaderVal by wrapping converter
in try-catch and returning default value on error
- Add auth header conflict handling to addHeadersToRequest and
addHeadersToSimpleRequest to match sync path behavior
- Add compression support to createSimpleHttpRequest
- Document async insert retry limitation in Client.java
- Fix test port selection to use dynamicPort() instead of random
- Fix metric assertion to use getResultRows() instead of getReadRows()
- Fix resource leaks by using try-with-resources for BufferedReader
---
.../com/clickhouse/client/api/Client.java | 13 +++++
.../api/internal/HttpAPIClientHelper.java | 52 +++++++++++++++++--
.../StreamingAsyncResponseConsumer.java | 16 +++++-
.../client/AsyncHttpClientTests.java | 50 ++++++++++--------
4 files changed, 104 insertions(+), 27 deletions(-)
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/Client.java b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
index 07633b340..efbb59e4d 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/Client.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
@@ -1477,6 +1477,19 @@ public void onRetry() throws IOException {
format, settings);
}
+ /**
+ * Executes an async insert operation using the async HTTP client.
+ *
+ * IMPORTANT: Unlike the synchronous insert path, this async implementation
+ * does NOT support automatic retry on 503 Service Unavailable responses. The synchronous
+ * path retries on 503 and retryable failures (invoking onRetry()/data.reset()), but
+ * async inserts with InputStreams cannot reliably support retry because streams are
+ * not always resettable.
+ *
+ * If you require retry semantics for insert operations, use the synchronous client
+ * (set useAsyncHttp(false)) or implement retry logic in your application code with
+ * a resettable data source.
+ */
private CompletableFuture executeInsertAsync(String tableName,
List columnNames,
InputStream data,
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
index c193c2297..4f2f17690 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
@@ -609,6 +609,10 @@ public void completed(SimpleHttpResponse response) {
future.completeExceptionally(new ClientException(
"Server returned '502 Bad gateway'. Check network and proxy settings."));
return;
+ } else if (response.getCode() == HttpStatus.SC_SERVICE_UNAVAILABLE) {
+ // Return 503 normally - let caller handle retry logic
+ future.complete(response);
+ return;
} else if (response.getCode() >= HttpStatus.SC_BAD_REQUEST ||
response.containsHeader(ClickHouseHttpProto.HEADER_EXCEPTION_CODE)) {
future.completeExceptionally(readErrorFromAsyncResponse(response));
@@ -691,6 +695,9 @@ public CompletableFuture execu
} else if (response.getCode() == HttpStatus.SC_BAD_GATEWAY) {
future.completeExceptionally(new ClientException(
"Server returned '502 Bad gateway'. Check network and proxy settings."));
+ } else if (response.getCode() == HttpStatus.SC_SERVICE_UNAVAILABLE) {
+ // Return 503 normally - let caller handle retry logic
+ future.complete(response);
} else if (response.getCode() >= HttpStatus.SC_BAD_REQUEST ||
response.containsHeader(ClickHouseHttpProto.HEADER_EXCEPTION_CODE)) {
future.completeExceptionally(readErrorFromStreamingResponse(response));
@@ -826,6 +833,9 @@ public CompletableFuture execu
} else if (response.getCode() == HttpStatus.SC_BAD_GATEWAY) {
future.completeExceptionally(new ClientException(
"Server returned '502 Bad gateway'. Check network and proxy settings."));
+ } else if (response.getCode() == HttpStatus.SC_SERVICE_UNAVAILABLE) {
+ // Return 503 normally - let caller handle retry logic
+ future.complete(response);
} else if (response.getCode() >= HttpStatus.SC_BAD_REQUEST ||
response.containsHeader(ClickHouseHttpProto.HEADER_EXCEPTION_CODE)) {
future.completeExceptionally(readErrorFromStreamingResponse(response));
@@ -861,6 +871,17 @@ public void cancelled() {
private SimpleHttpRequest createSimpleHttpRequest(URI uri, Map requestConfig, String body) {
byte[] bodyBytes = body.getBytes(StandardCharsets.UTF_8);
+ // Apply compression if configured
+ boolean clientCompression = ClientConfigProperties.COMPRESS_CLIENT_REQUEST.getOrDefault(requestConfig);
+ boolean useHttpCompression = ClientConfigProperties.USE_HTTP_COMPRESSION.getOrDefault(requestConfig);
+ boolean appCompressedData = ClientConfigProperties.APP_COMPRESSED_DATA.getOrDefault(requestConfig);
+
+ if (clientCompression && !appCompressedData) {
+ int bufferSize = ClientConfigProperties.COMPRESSION_LZ4_UNCOMPRESSED_BUF_SIZE.getOrDefault(requestConfig);
+ bodyBytes = compressLZ4(bodyBytes, useHttpCompression, bufferSize);
+ LOG.debug("Async simple request compressed: {} -> {} bytes", body.length(), bodyBytes.length);
+ }
+
SimpleRequestBuilder builder = SimpleRequestBuilder.post(uri)
.setBody(bodyBytes, CONTENT_TYPE);
addHeadersToSimpleRequest(builder, requestConfig);
@@ -881,9 +902,16 @@ private void addHeadersToSimpleRequest(SimpleRequestBuilder builder, MapgetOrDefault(requestConfig).booleanValue()) {
- builder.setHeader(ClickHouseHttpProto.HEADER_DB_USER,
- ClientConfigProperties.USER.getOrDefault(requestConfig));
+ if (!hasCustomAuth) {
+ builder.setHeader(ClickHouseHttpProto.HEADER_DB_USER,
+ ClientConfigProperties.USER.getOrDefault(requestConfig));
+ }
builder.setHeader(ClickHouseHttpProto.HEADER_SSL_CERT_AUTH, "on");
} else if (ClientConfigProperties.HTTP_USE_BASIC_AUTH.getOrDefault(requestConfig).booleanValue()) {
String user = ClientConfigProperties.USER.getOrDefault(requestConfig);
@@ -891,7 +919,8 @@ private void addHeadersToSimpleRequest(SimpleRequestBuilder builder, Map requestCon
}
}
+ // Special cases - match sync addHeaders behavior
+ if (req.containsHeader(HttpHeaders.AUTHORIZATION)
+ && (req.containsHeader(ClickHouseHttpProto.HEADER_DB_USER) ||
+ req.containsHeader(ClickHouseHttpProto.HEADER_DB_PASSWORD)))
+ {
+ // user has set auth header for purpose, lets remove ours
+ req.removeHeaders(ClickHouseHttpProto.HEADER_DB_USER);
+ req.removeHeaders(ClickHouseHttpProto.HEADER_DB_PASSWORD);
+ }
+
correctUserAgentHeader(req, requestConfig);
}
@@ -1301,7 +1340,12 @@ public static T getHeaderVal(Header header, T defaultValue, Function requestSettings) {
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncResponseConsumer.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncResponseConsumer.java
index 81ebd2ddc..f6f38f269 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncResponseConsumer.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncResponseConsumer.java
@@ -39,6 +39,7 @@ public class StreamingAsyncResponseConsumer extends AbstractBinResponseConsumer<
private HttpResponse response;
private ContentType contentType;
private volatile Exception streamError;
+ private volatile long totalBytesWritten = 0;
public StreamingAsyncResponseConsumer() {
this(DEFAULT_PIPE_SIZE);
@@ -85,20 +86,27 @@ protected int capacityIncrement() {
@Override
protected void data(ByteBuffer src, boolean endOfStream) throws IOException {
if (streamError != null) {
+ LOG.debug("data() called but streamError already set: {}", streamError.getMessage());
return;
}
try {
if (src.hasRemaining()) {
- byte[] bytes = new byte[src.remaining()];
+ int chunkSize = src.remaining();
+ byte[] bytes = new byte[chunkSize];
src.get(bytes);
pipedOutputStream.write(bytes);
+ pipedOutputStream.flush(); // Ensure data is immediately available to reader
+ totalBytesWritten += chunkSize;
+ LOG.debug("data() wrote {} bytes (total: {}), endOfStream={}", chunkSize, totalBytesWritten, endOfStream);
}
if (endOfStream) {
+ LOG.debug("data() endOfStream=true, closing output. Total bytes: {}", totalBytesWritten);
closeOutputStream();
}
} catch (IOException e) {
+ LOG.debug("data() IOException: {} (total bytes written: {})", e.getMessage(), totalBytesWritten);
streamError = e;
closeOutputStream();
throw e;
@@ -129,12 +137,16 @@ public void failed(Exception cause) {
private void closeOutputStream() {
if (outputClosed.compareAndSet(false, true)) {
+ LOG.debug("closeOutputStream() called, total bytes written: {}", totalBytesWritten);
try {
pipedOutputStream.close();
streamCompleteFuture.complete(null);
+ LOG.debug("closeOutputStream() completed successfully");
} catch (IOException e) {
- LOG.debug("Error closing piped output stream", e);
+ LOG.debug("Error closing piped output stream: {}", e.getMessage());
}
+ } else {
+ LOG.debug("closeOutputStream() already closed, skipping");
}
}
diff --git a/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java b/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
index a4ea72fd1..77b6c2a81 100644
--- a/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
+++ b/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
@@ -10,6 +10,7 @@
import com.clickhouse.client.api.insert.InsertSettings;
import com.clickhouse.client.api.query.GenericRecord;
import com.clickhouse.client.api.query.QueryResponse;
+import com.clickhouse.client.api.query.QuerySettings;
import com.clickhouse.data.ClickHouseFormat;
import com.github.tomakehurst.wiremock.WireMockServer;
import com.github.tomakehurst.wiremock.client.WireMock;
@@ -130,8 +131,8 @@ public void testAsyncQueryWithFutureComposition() {
CompletableFuture resultFuture = client.query("SELECT count() FROM numbers(1000)")
.thenApply(response -> {
try {
- // Read the count from response
- return response.getReadRows();
+ // Read the result row count from response
+ return response.getResultRows();
} finally {
try {
response.close();
@@ -142,7 +143,7 @@ public void testAsyncQueryWithFutureComposition() {
});
Long count = resultFuture.get(30, TimeUnit.SECONDS);
- Assert.assertEquals(count.longValue(), 1L); // Query reads 1000 rows but returns 1 row (count)
+ Assert.assertEquals(count.longValue(), 1L); // Query returns 1 row (count result)
} catch (Exception e) {
e.printStackTrace();
@@ -194,12 +195,13 @@ public void testAsyncQueryRetryOn503() {
return;
}
- int serverPort = new Random().nextInt(1000) + 10000;
WireMockServer mockServer = new WireMockServer(WireMockConfiguration
- .options().port(serverPort).notifier(new ConsoleNotifier(false)));
+ .options().dynamicPort().notifier(new ConsoleNotifier(false)));
mockServer.start();
try {
+ int serverPort = mockServer.port();
+
// First request returns 503 (Service Unavailable)
mockServer.addStubMapping(WireMock.post(WireMock.anyUrl())
.inScenario("Retry503")
@@ -355,12 +357,13 @@ public void testAsyncQueryCancellation() {
return;
}
- int serverPort = new Random().nextInt(1000) + 10000;
WireMockServer mockServer = new WireMockServer(WireMockConfiguration
- .options().port(serverPort).notifier(new ConsoleNotifier(false)));
+ .options().dynamicPort().notifier(new ConsoleNotifier(false)));
mockServer.start();
try {
+ int serverPort = mockServer.port();
+
// Setup a delayed response
mockServer.addStubMapping(WireMock.post(WireMock.anyUrl())
.willReturn(WireMock.aResponse()
@@ -478,7 +481,9 @@ public void testAsyncStreamingLargeResult() {
.build()) {
// Query that returns ~1MB of data (100K rows * ~10 bytes each)
- try (QueryResponse response = client.query("SELECT number, toString(number) FROM numbers(100000)")
+ // Use TabSeparated format so we can count lines with BufferedReader
+ QuerySettings settings = new QuerySettings().setFormat(ClickHouseFormat.TabSeparated);
+ try (QueryResponse response = client.query("SELECT number, toString(number) FROM numbers(100000)", settings)
.get(60, TimeUnit.SECONDS)) {
Assert.assertTrue(response.getReadRows() > 0, "Expected read_rows > 0");
@@ -519,7 +524,9 @@ public void testAsyncStreamingIncrementalRead() {
.useAsyncHttp(true)
.build()) {
- try (QueryResponse response = client.query("SELECT number FROM numbers(1000)")
+ // Use TabSeparated format for text-based streaming
+ QuerySettings settings = new QuerySettings().setFormat(ClickHouseFormat.TabSeparated);
+ try (QueryResponse response = client.query("SELECT number FROM numbers(1000)", settings)
.get(30, TimeUnit.SECONDS)) {
java.io.InputStream is = response.getInputStream();
@@ -567,26 +574,27 @@ public void testAsyncStreamingNoDeadlockOnDelayedRead() {
// Query that returns data larger than pipe buffer (512KB)
// This would deadlock with the old implementation if user delays reading
+ // Use TabSeparated format so we can count lines with BufferedReader
+ QuerySettings settings = new QuerySettings().setFormat(ClickHouseFormat.TabSeparated);
CompletableFuture future = client.query(
- "SELECT number, repeat('x', 100) FROM numbers(10000)"); // ~1MB response
+ "SELECT number, repeat('x', 100) FROM numbers(10000)", settings); // ~1MB response
// Simulate delayed reading - OLD code would deadlock here
Thread.sleep(500);
// Get response - should complete immediately since headers arrived
- QueryResponse response = future.get(5, TimeUnit.SECONDS);
-
- // Now read the stream - NIO thread continues writing while we read
- try (java.io.BufferedReader reader = new java.io.BufferedReader(
- new java.io.InputStreamReader(response.getInputStream()))) {
- long lineCount = 0;
- while (reader.readLine() != null) {
- lineCount++;
- }
+ try (QueryResponse response = future.get(5, TimeUnit.SECONDS)) {
+ // Now read the stream - NIO thread continues writing while we read
+ try (java.io.BufferedReader reader = new java.io.BufferedReader(
+ new java.io.InputStreamReader(response.getInputStream()))) {
+ long lineCount = 0;
+ while (reader.readLine() != null) {
+ lineCount++;
+ }
- Assert.assertEquals(lineCount, 10000, "Expected 10000 rows");
+ Assert.assertEquals(lineCount, 10000, "Expected 10000 rows");
+ }
}
- response.close();
} catch (Exception e) {
e.printStackTrace();
From 9dcfd168e767046791e805a5c0f9ed99b3d6cfe9 Mon Sep 17 00:00:00 2001
From: David Karlsson <2795016+devdavidkarlsson@users.noreply.github.com>
Date: Fri, 13 Feb 2026 16:38:09 +0100
Subject: [PATCH 10/32] perf: use reusable buffer in
StreamingAsyncResponseConsumer to reduce GC pressure
Instead of allocating a new byte[] for every ByteBuffer chunk received,
use a fixed 8KB reusable buffer and write in chunks. This reduces
allocation/GC pressure under high-throughput streaming scenarios.
---
.../StreamingAsyncResponseConsumer.java | 20 ++++++++++++-------
1 file changed, 13 insertions(+), 7 deletions(-)
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncResponseConsumer.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncResponseConsumer.java
index f6f38f269..d4752626a 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncResponseConsumer.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncResponseConsumer.java
@@ -41,6 +41,9 @@ public class StreamingAsyncResponseConsumer extends AbstractBinResponseConsumer<
private volatile Exception streamError;
private volatile long totalBytesWritten = 0;
+ // Reusable buffer to avoid GC pressure from per-chunk allocations
+ private final byte[] reusableBuffer = new byte[CAPACITY_INCREMENT];
+
public StreamingAsyncResponseConsumer() {
this(DEFAULT_PIPE_SIZE);
}
@@ -91,14 +94,17 @@ protected void data(ByteBuffer src, boolean endOfStream) throws IOException {
}
try {
- if (src.hasRemaining()) {
- int chunkSize = src.remaining();
- byte[] bytes = new byte[chunkSize];
- src.get(bytes);
- pipedOutputStream.write(bytes);
+ long bytesWrittenThisCall = 0;
+ while (src.hasRemaining()) {
+ int bytesToRead = Math.min(src.remaining(), reusableBuffer.length);
+ src.get(reusableBuffer, 0, bytesToRead);
+ pipedOutputStream.write(reusableBuffer, 0, bytesToRead);
+ bytesWrittenThisCall += bytesToRead;
+ }
+ if (bytesWrittenThisCall > 0) {
+ totalBytesWritten += bytesWrittenThisCall;
pipedOutputStream.flush(); // Ensure data is immediately available to reader
- totalBytesWritten += chunkSize;
- LOG.debug("data() wrote {} bytes (total: {}), endOfStream={}", chunkSize, totalBytesWritten, endOfStream);
+ LOG.debug("data() wrote {} bytes (total: {}), endOfStream={}", bytesWrittenThisCall, totalBytesWritten, endOfStream);
}
if (endOfStream) {
From 9a4587258084bb2360ee114c02625ffbbfcee0f9 Mon Sep 17 00:00:00 2001
From: David Karlsson <2795016+devdavidkarlsson@users.noreply.github.com>
Date: Mon, 16 Feb 2026 10:45:35 +0100
Subject: [PATCH 11/32] fix: close StreamingResponse in
readErrorFromStreamingResponse to prevent deadlock
The method reads from the streaming pipe to build an exception but never
closed the stream. If the error body is larger than the pipe buffer, the
NIO thread would block on write, reintroducing a deadlock.
Also switched to using getHeaderVal for parsing the error code header
to handle potential NumberFormatException safely.
---
.../client/api/internal/HttpAPIClientHelper.java | 10 ++++++++--
1 file changed, 8 insertions(+), 2 deletions(-)
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
index 4f2f17690..e9b2469f2 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
@@ -739,12 +739,18 @@ private Exception readErrorFromStreamingResponse(StreamingAsyncResponseConsumer.
int bytesRead = is.read(errorBytes, 0, ERROR_BODY_BUFFER_SIZE);
String errorBody = bytesRead > 0 ? new String(errorBytes, 0, bytesRead, StandardCharsets.UTF_8) : "";
- Header errorCodeHeader = response.getFirstHeader(ClickHouseHttpProto.HEADER_EXCEPTION_CODE);
- int errorCode = errorCodeHeader != null ? Integer.parseInt(errorCodeHeader.getValue()) : 0;
+ int errorCode = getHeaderVal(response.getFirstHeader(ClickHouseHttpProto.HEADER_EXCEPTION_CODE),
+ 0, Integer::parseInt);
return new ServerException(errorCode, errorBody, response.getCode(), null);
} catch (Exception e) {
return new ClientException("Failed to read error response", e);
+ } finally {
+ try {
+ response.close();
+ } catch (IOException e) {
+ LOG.debug("Failed to close streaming response after reading error", e);
+ }
}
}
From 2ff342c1ca4d5848bea8da6652f3d6df344c88a7 Mon Sep 17 00:00:00 2001
From: David Karlsson <2795016+devdavidkarlsson@users.noreply.github.com>
Date: Mon, 16 Feb 2026 11:07:24 +0100
Subject: [PATCH 12/32] fix: address additional Copilot review comments
- Close StreamingResponse in error paths to prevent resource leaks and deadlocks
- Fix closeOutputStream() to complete future exceptionally when streamError is set
- Fix unsafe Throwable cast to Exception in async retry path
- Fix column list building to remove both comma and space (use setLength instead of deleteCharAt)
- Close StreamingResponse before 503 retry to avoid resource leaks
- Remove unused imports from HttpAPIClientHelper, Client, and AsyncHttpClientTests
- Update CHANGELOG test count description
- Update buildResult() comment to accurately describe behavior
---
CHANGELOG.md | 2 +-
.../com/clickhouse/client/api/Client.java | 19 ++++++++++++-------
.../api/internal/HttpAPIClientHelper.java | 19 ++++++++++++++++---
.../StreamingAsyncResponseConsumer.java | 12 +++++++++---
.../client/AsyncHttpClientTests.java | 4 ----
5 files changed, 38 insertions(+), 18 deletions(-)
diff --git a/CHANGELOG.md b/CHANGELOG.md
index f775bc1b0..bf997f470 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -8,7 +8,7 @@
- 96-99% thread reduction under high concurrency (1000 concurrent requests use 2-4 threads instead of 1000)
- Opt-in via `useAsyncHttp(true)` builder option
- Full backward compatibility (async disabled by default)
- - Comprehensive test coverage (19 integration tests)
+ - Added test coverage (integration tests)
### Known Limitations
- [client-v2] Async HTTP: SOCKS proxy not supported (Apache HttpClient async limitation)
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/Client.java b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
index efbb59e4d..f641ad7f8 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/Client.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
@@ -18,7 +18,6 @@
import com.clickhouse.client.api.internal.ClientStatisticsHolder;
import com.clickhouse.client.api.internal.HttpAPIClientHelper;
import com.clickhouse.client.api.internal.MapUtils;
-import com.clickhouse.client.api.internal.StreamingAsyncResponseConsumer;
import com.clickhouse.client.api.internal.TableSchemaParser;
import com.clickhouse.client.api.internal.ValidationUtils;
import com.clickhouse.client.api.metadata.ColumnToMethodMatchingStrategy;
@@ -42,7 +41,6 @@
import com.clickhouse.data.ClickHouseFormat;
import com.google.common.collect.ImmutableList;
import net.jpountz.lz4.LZ4Factory;
-import org.apache.hc.client5.http.async.methods.SimpleHttpResponse;
import org.apache.hc.core5.concurrent.DefaultThreadFactory;
import org.apache.hc.core5.http.ClassicHttpResponse;
import org.apache.hc.core5.http.Header;
@@ -79,7 +77,6 @@
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
-import java.util.function.Function;
import java.util.function.Supplier;
import java.util.stream.Collectors;
@@ -1513,7 +1510,7 @@ private CompletableFuture executeInsertAsync(String tableName,
for (String columnName : columnNames) {
sqlStmt.append(columnName).append(", ");
}
- sqlStmt.deleteCharAt(sqlStmt.length() - 2);
+ sqlStmt.setLength(sqlStmt.length() - 2);
sqlStmt.append(")");
}
sqlStmt.append(" FORMAT ").append(format.name());
@@ -1611,7 +1608,7 @@ public CompletableFuture insert(String tableName,
for (String columnName : columnNames) {
sqlStmt.append(columnName).append(", ");
}
- sqlStmt.deleteCharAt(sqlStmt.length() - 2);
+ sqlStmt.setLength(sqlStmt.length() - 2);
sqlStmt.append(")");
}
sqlStmt.append(" FORMAT ").append(format.name());
@@ -1827,9 +1824,11 @@ private CompletableFuture executeQueryAsync(String sqlQuery,
if (ex != null) {
Throwable cause = ex instanceof java.util.concurrent.CompletionException ? ex.getCause() : ex;
String msg = requestExMsg("Query", (attempt + 1), durationSince(startTime).toMillis(), requestSettings.getQueryId());
- RuntimeException wrappedException = httpClientHelper.wrapException(msg, (Exception) cause, requestSettings.getQueryId());
+ RuntimeException wrappedException = (cause instanceof Exception)
+ ? httpClientHelper.wrapException(msg, (Exception) cause, requestSettings.getQueryId())
+ : new RuntimeException(msg, cause);
- if (httpClientHelper.shouldRetry((Exception) cause, requestSettings.getAllSettings()) && attempt < retries) {
+ if (httpClientHelper.shouldRetry(cause, requestSettings.getAllSettings()) && attempt < retries) {
LOG.warn("Async query failed, retrying (attempt {}): {}", attempt + 1, cause.getMessage());
return new AsyncRetryMarker(attempt + 1);
}
@@ -1839,6 +1838,12 @@ private CompletableFuture executeQueryAsync(String sqlQuery,
if (response.getCode() == HttpStatus.SC_SERVICE_UNAVAILABLE && attempt < retries) {
LOG.warn("Failed to get response. Server returned {}. Retrying. (Duration: {})",
response.getCode(), durationSince(startTime));
+ // Close the streaming response before retrying to avoid resource leaks
+ try {
+ response.close();
+ } catch (Exception closeEx) {
+ LOG.debug("Failed to close streaming response before retry", closeEx);
+ }
return new AsyncRetryMarker(attempt + 1);
}
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
index e9b2469f2..7c1ec8114 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
@@ -21,11 +21,9 @@
import org.apache.hc.client5.http.async.methods.SimpleHttpRequest;
import org.apache.hc.client5.http.async.methods.SimpleHttpResponse;
import org.apache.hc.client5.http.async.methods.SimpleRequestBuilder;
-import org.apache.hc.client5.http.async.methods.SimpleResponseConsumer;
import org.apache.hc.client5.http.classic.methods.HttpPost;
import org.apache.hc.client5.http.config.ConnectionConfig;
import org.apache.hc.client5.http.config.RequestConfig;
-import org.apache.hc.client5.http.config.TlsConfig;
import org.apache.hc.client5.http.entity.mime.MultipartEntityBuilder;
import org.apache.hc.client5.http.entity.mime.MultipartPartBuilder;
import org.apache.hc.client5.http.impl.async.CloseableHttpAsyncClient;
@@ -37,7 +35,6 @@
import org.apache.hc.client5.http.impl.io.ManagedHttpClientConnectionFactory;
import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManager;
import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManagerBuilder;
-import org.apache.hc.client5.http.impl.nio.PoolingAsyncClientConnectionManager;
import org.apache.hc.client5.http.impl.nio.PoolingAsyncClientConnectionManagerBuilder;
import org.apache.hc.client5.http.io.HttpClientConnectionManager;
import org.apache.hc.client5.http.io.ManagedHttpClientConnection;
@@ -690,9 +687,11 @@ public CompletableFuture execu
try {
if (response.getCode() == HttpStatus.SC_PROXY_AUTHENTICATION_REQUIRED) {
+ closeStreamingResponse(response);
future.completeExceptionally(new ClientMisconfigurationException(
"Proxy authentication required. Please check your proxy settings."));
} else if (response.getCode() == HttpStatus.SC_BAD_GATEWAY) {
+ closeStreamingResponse(response);
future.completeExceptionally(new ClientException(
"Server returned '502 Bad gateway'. Check network and proxy settings."));
} else if (response.getCode() == HttpStatus.SC_SERVICE_UNAVAILABLE) {
@@ -700,11 +699,13 @@ public CompletableFuture execu
future.complete(response);
} else if (response.getCode() >= HttpStatus.SC_BAD_REQUEST ||
response.containsHeader(ClickHouseHttpProto.HEADER_EXCEPTION_CODE)) {
+ // readErrorFromStreamingResponse closes the response
future.completeExceptionally(readErrorFromStreamingResponse(response));
} else {
future.complete(response);
}
} catch (Exception e) {
+ closeStreamingResponse(response);
future.completeExceptionally(e);
}
});
@@ -754,6 +755,14 @@ private Exception readErrorFromStreamingResponse(StreamingAsyncResponseConsumer.
}
}
+ private void closeStreamingResponse(StreamingAsyncResponseConsumer.StreamingResponse response) {
+ try {
+ response.close();
+ } catch (IOException e) {
+ LOG.debug("Failed to close streaming response", e);
+ }
+ }
+
/**
* Compresses data using LZ4 compression.
*
@@ -834,9 +843,11 @@ public CompletableFuture execu
try {
if (response.getCode() == HttpStatus.SC_PROXY_AUTHENTICATION_REQUIRED) {
+ closeStreamingResponse(response);
future.completeExceptionally(new ClientMisconfigurationException(
"Proxy authentication required. Please check your proxy settings."));
} else if (response.getCode() == HttpStatus.SC_BAD_GATEWAY) {
+ closeStreamingResponse(response);
future.completeExceptionally(new ClientException(
"Server returned '502 Bad gateway'. Check network and proxy settings."));
} else if (response.getCode() == HttpStatus.SC_SERVICE_UNAVAILABLE) {
@@ -844,11 +855,13 @@ public CompletableFuture execu
future.complete(response);
} else if (response.getCode() >= HttpStatus.SC_BAD_REQUEST ||
response.containsHeader(ClickHouseHttpProto.HEADER_EXCEPTION_CODE)) {
+ // readErrorFromStreamingResponse closes the response
future.completeExceptionally(readErrorFromStreamingResponse(response));
} else {
future.complete(response);
}
} catch (Exception e) {
+ closeStreamingResponse(response);
future.completeExceptionally(e);
}
});
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncResponseConsumer.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncResponseConsumer.java
index d4752626a..b814c8268 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncResponseConsumer.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncResponseConsumer.java
@@ -121,7 +121,8 @@ protected void data(ByteBuffer src, boolean endOfStream) throws IOException {
@Override
protected StreamingResponse buildResult() {
- // Return the same response that was provided via headersFuture
+ // Note: This creates a new instance each time, but all instances share the same
+ // underlying streams and futures. The headersFuture provides the primary response.
return new StreamingResponse(response, contentType, pipedInputStream, streamCompleteFuture);
}
@@ -146,8 +147,13 @@ private void closeOutputStream() {
LOG.debug("closeOutputStream() called, total bytes written: {}", totalBytesWritten);
try {
pipedOutputStream.close();
- streamCompleteFuture.complete(null);
- LOG.debug("closeOutputStream() completed successfully");
+ if (streamError != null) {
+ streamCompleteFuture.completeExceptionally(streamError);
+ LOG.debug("closeOutputStream() completed with error");
+ } else {
+ streamCompleteFuture.complete(null);
+ LOG.debug("closeOutputStream() completed successfully");
+ }
} catch (IOException e) {
LOG.debug("Error closing piped output stream: {}", e.getMessage());
}
diff --git a/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java b/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
index 77b6c2a81..124634e2e 100644
--- a/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
+++ b/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
@@ -1,13 +1,9 @@
package com.clickhouse.client;
import com.clickhouse.client.api.Client;
-import com.clickhouse.client.api.ClientConfigProperties;
-import com.clickhouse.client.api.ClientException;
import com.clickhouse.client.api.ServerException;
-import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader;
import com.clickhouse.client.api.enums.Protocol;
import com.clickhouse.client.api.insert.InsertResponse;
-import com.clickhouse.client.api.insert.InsertSettings;
import com.clickhouse.client.api.query.GenericRecord;
import com.clickhouse.client.api.query.QueryResponse;
import com.clickhouse.client.api.query.QuerySettings;
From 234e3479d1f4ec9fc2abb0162a03a61d133ca112 Mon Sep 17 00:00:00 2001
From: David Karlsson <2795016+devdavidkarlsson@users.noreply.github.com>
Date: Mon, 16 Feb 2026 11:16:55 +0100
Subject: [PATCH 13/32] fix: propagate cancellation to underlying async HTTP
requests
Capture the Future returned by httpAsyncClient.execute() and cancel it
when the user cancels the returned CompletableFuture. This ensures
in-flight requests are aborted and connections released promptly.
Applied to all 3 async methods:
- executeRequestAsync
- executeRequestAsyncStreaming
- executeInsertAsyncStreaming
---
.../api/internal/HttpAPIClientHelper.java | 30 +++++++++++++++++--
1 file changed, 27 insertions(+), 3 deletions(-)
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
index 7c1ec8114..8037ffff1 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
@@ -115,6 +115,7 @@
import java.util.Properties;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Future;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
@@ -594,7 +595,7 @@ public CompletableFuture executeRequestAsync(Endpoint server
CompletableFuture future = new CompletableFuture<>();
- httpAsyncClient.execute(request, new FutureCallback() {
+ Future httpFuture = httpAsyncClient.execute(request, new FutureCallback() {
@Override
public void completed(SimpleHttpResponse response) {
try {
@@ -633,6 +634,13 @@ public void cancelled() {
}
});
+ // Propagate cancellation to the underlying HTTP request
+ future.whenComplete((result, ex) -> {
+ if (future.isCancelled()) {
+ httpFuture.cancel(true);
+ }
+ });
+
return future;
}
@@ -710,7 +718,8 @@ public CompletableFuture execu
}
});
- httpAsyncClient.execute(requestProducer, responseConsumer, new FutureCallback() {
+ Future httpFuture = httpAsyncClient.execute(
+ requestProducer, responseConsumer, new FutureCallback() {
@Override
public void completed(StreamingAsyncResponseConsumer.StreamingResponse response) {
// Stream has ended. Future should already be completed via headersFuture.
@@ -730,6 +739,13 @@ public void cancelled() {
}
});
+ // Propagate cancellation to the underlying HTTP request
+ future.whenComplete((result, ex) -> {
+ if (future.isCancelled()) {
+ httpFuture.cancel(true);
+ }
+ });
+
return future;
}
@@ -866,7 +882,8 @@ public CompletableFuture execu
}
});
- httpAsyncClient.execute(requestProducer, responseConsumer, new FutureCallback() {
+ Future httpFuture = httpAsyncClient.execute(
+ requestProducer, responseConsumer, new FutureCallback() {
@Override
public void completed(StreamingAsyncResponseConsumer.StreamingResponse response) {
LOG.debug("Async insert request completed for '{}'", uri);
@@ -884,6 +901,13 @@ public void cancelled() {
}
});
+ // Propagate cancellation to the underlying HTTP request
+ future.whenComplete((result, ex) -> {
+ if (future.isCancelled()) {
+ httpFuture.cancel(true);
+ }
+ });
+
return future;
}
From 5d5c6c832681b197548de2c2a9fdb9279304150d Mon Sep 17 00:00:00 2001
From: David Karlsson <2795016+devdavidkarlsson@users.noreply.github.com>
Date: Mon, 16 Feb 2026 14:16:08 +0100
Subject: [PATCH 14/32] fix: address Copilot review round 2
- Remove unused imports (Random, AtomicReference) from AsyncHttpClientTests
- Add Content-Encoding header in addHeadersToSimpleRequest when client
compression is enabled (matching sync path behavior)
- Handle null response body in QueryResponse constructor for async path
Co-Authored-By: Claude Opus 4.5
---
.../client/api/internal/HttpAPIClientHelper.java | 11 +++++++++--
.../clickhouse/client/api/query/QueryResponse.java | 4 +++-
.../com/clickhouse/client/AsyncHttpClientTests.java | 2 --
3 files changed, 12 insertions(+), 5 deletions(-)
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
index 8037ffff1..42205d1d8 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
@@ -976,10 +976,17 @@ private void addHeadersToSimpleRequest(SimpleRequestBuilder builder, Map
Date: Mon, 16 Feb 2026 14:34:51 +0100
Subject: [PATCH 15/32] Update CHANGELOG.md
Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
---
CHANGELOG.md | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/CHANGELOG.md b/CHANGELOG.md
index bf997f470..1185b041d 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -5,7 +5,7 @@
- True non-blocking I/O for queries and inserts
- Streaming responses with constant memory usage (~512KB regardless of result size)
- Streaming request compression (HTTP framed LZ4 and ClickHouse native LZ4)
- - 96-99% thread reduction under high concurrency (1000 concurrent requests use 2-4 threads instead of 1000)
+ - Eliminates blocking threads waiting for I/O under high concurrency, significantly reducing thread usage compared to the synchronous client
- Opt-in via `useAsyncHttp(true)` builder option
- Full backward compatibility (async disabled by default)
- Added test coverage (integration tests)
From 2de0df129e0c0187ade9ce7d14ddbc922162447e Mon Sep 17 00:00:00 2001
From: David Karlsson <2795016+devdavidkarlsson@users.noreply.github.com>
Date: Mon, 16 Feb 2026 14:35:49 +0100
Subject: [PATCH 16/32] Update
client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncEntityProducer.java
Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
---
.../client/api/internal/StreamingAsyncEntityProducer.java | 7 ++++++-
1 file changed, 6 insertions(+), 1 deletion(-)
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncEntityProducer.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncEntityProducer.java
index 4e0e6455b..f50ee7215 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncEntityProducer.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncEntityProducer.java
@@ -97,7 +97,12 @@ private void initializeStreams() throws IOException {
error.set(e);
try {
compressedOutputStream.close();
- } catch (IOException ignored) {}
+ } catch (IOException ignored) {
+ }
+ try {
+ sourceStream.close();
+ } catch (IOException ignored) {
+ }
}
}, "async-compression-thread");
compressionThread.setDaemon(true);
From 0627b9e57ca3439a345c9299bc5d140c241c24eb Mon Sep 17 00:00:00 2001
From: David Karlsson <2795016+devdavidkarlsson@users.noreply.github.com>
Date: Mon, 16 Feb 2026 14:36:10 +0100
Subject: [PATCH 17/32] Update
client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
---
.../test/java/com/clickhouse/client/AsyncHttpClientTests.java | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java b/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
index 41bbe7afa..038195bc2 100644
--- a/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
+++ b/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
@@ -171,7 +171,7 @@ public void testAsyncQueryServerError() {
Assert.assertTrue(e.getCause() instanceof ServerException,
"Expected ServerException but got: " + e.getCause().getClass().getName());
ServerException se = (ServerException) e.getCause();
- Assert.assertEquals(se.getCode(), 62); // Syntax error code
+ Assert.assertNotEquals(se.getCode(), 0, "Expected non-zero error code for syntax error");
}
} catch (Exception e) {
From 9dae2eb3cd0ef111b7ecf763adef6f5838df38a0 Mon Sep 17 00:00:00 2001
From: David Karlsson <2795016+devdavidkarlsson@users.noreply.github.com>
Date: Mon, 16 Feb 2026 14:36:34 +0100
Subject: [PATCH 18/32] Update
client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
---
.../client/api/internal/HttpAPIClientHelper.java | 16 +++++++++++++---
1 file changed, 13 insertions(+), 3 deletions(-)
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
index 42205d1d8..b43add45c 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
@@ -176,9 +176,19 @@ public HttpAPIClientHelper(Map configuration, Object metricsRegi
// Initialize async client
boolean useAsyncHttp = ClientConfigProperties.USE_ASYNC_HTTP.getOrDefault(configuration);
if (useAsyncHttp) {
- this.httpAsyncClient = createHttpAsyncClient(initSslContext, configuration);
- this.httpAsyncClient.start();
- LOG.info("Async HTTP client initialized and started");
+ try {
+ CloseableHttpAsyncClient asyncClient = createHttpAsyncClient(initSslContext, configuration);
+ asyncClient.start();
+ this.httpAsyncClient = asyncClient;
+ LOG.info("Async HTTP client initialized and started");
+ } catch (RuntimeException | Error e) {
+ try {
+ this.httpClient.close();
+ } catch (IOException closeEx) {
+ e.addSuppressed(closeEx);
+ }
+ throw e;
+ }
} else {
this.httpAsyncClient = null;
}
From ceddd454ea5c30ccac42244575fc221b5e08e767 Mon Sep 17 00:00:00 2001
From: David Karlsson <2795016+devdavidkarlsson@users.noreply.github.com>
Date: Mon, 16 Feb 2026 15:06:00 +0100
Subject: [PATCH 19/32] fix: address Copilot review round 3
- Add unique identifier to compression thread names for easier debugging
- Simplify QueryResponse.close() to propagate exceptions directly,
matching AutoCloseable contract expectations
- Wrap testAsyncInsertResultsMatchSync in try-finally to ensure table
cleanup happens even if the test fails
Co-Authored-By: Claude Opus 4.5
---
.../StreamingAsyncEntityProducer.java | 4 +-
.../client/api/query/QueryResponse.java | 16 ++-
.../client/AsyncHttpClientTests.java | 101 +++++++++---------
3 files changed, 60 insertions(+), 61 deletions(-)
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncEntityProducer.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncEntityProducer.java
index f50ee7215..1431927c7 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncEntityProducer.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncEntityProducer.java
@@ -15,6 +15,7 @@
import java.nio.ByteBuffer;
import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
/**
@@ -28,6 +29,7 @@ public class StreamingAsyncEntityProducer implements AsyncEntityProducer {
private static final Logger LOG = LoggerFactory.getLogger(StreamingAsyncEntityProducer.class);
private static final int DEFAULT_BUFFER_SIZE = 8 * 1024; // 8KB read buffer
private static final int PIPE_BUFFER_SIZE = 512 * 1024; // 512KB pipe buffer
+ private static final AtomicLong THREAD_COUNTER = new AtomicLong(0); // For unique thread names
private final ContentType contentType;
private final InputStream sourceStream;
@@ -104,7 +106,7 @@ private void initializeStreams() throws IOException {
} catch (IOException ignored) {
}
}
- }, "async-compression-thread");
+ }, "async-compression-thread-" + THREAD_COUNTER.incrementAndGet());
compressionThread.setDaemon(true);
compressionThread.start();
} else {
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/query/QueryResponse.java b/client-v2/src/main/java/com/clickhouse/client/api/query/QueryResponse.java
index b7a13b301..167a87194 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/query/QueryResponse.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/query/QueryResponse.java
@@ -113,21 +113,17 @@ public InputStream getInputStream() {
}
}
+ /**
+ * Closes this response and releases associated resources.
+ * IOExceptions are propagated as-is to maintain AutoCloseable contract consistency.
+ */
@Override
public void close() throws Exception {
if (streamingInputStream != null) {
- try {
- streamingInputStream.close();
- } catch (IOException e) {
- throw new ClientException("Failed to close streaming response", e);
- }
+ streamingInputStream.close();
}
if (httpResponse != null) {
- try {
- httpResponse.close();
- } catch (Exception e) {
- throw new ClientException("Failed to close response", e);
- }
+ httpResponse.close();
}
}
diff --git a/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java b/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
index 038195bc2..42e73c2f0 100644
--- a/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
+++ b/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
@@ -878,67 +878,68 @@ public void testAsyncInsertResultsMatchSync() {
String csvData = "1,test1\n2,test2\n3,test3\n4,test4\n5,test5\n";
- // Insert using async client
- try (Client asyncClient = new Client.Builder()
- .addEndpoint(server.getBaseUri())
- .setUsername("default")
- .setPassword(getPassword())
- .useAsyncHttp(true)
- .build()) {
-
- asyncClient.query("CREATE TABLE " + asyncTable + " (id UInt64, value String) ENGINE = Memory")
- .get(10, TimeUnit.SECONDS).close();
+ try {
+ // Insert using async client
+ try (Client asyncClient = new Client.Builder()
+ .addEndpoint(server.getBaseUri())
+ .setUsername("default")
+ .setPassword(getPassword())
+ .useAsyncHttp(true)
+ .build()) {
- ByteArrayInputStream dataStream = new ByteArrayInputStream(csvData.getBytes(StandardCharsets.UTF_8));
- asyncClient.insert(asyncTable, dataStream, ClickHouseFormat.CSV).get(10, TimeUnit.SECONDS);
+ asyncClient.query("CREATE TABLE " + asyncTable + " (id UInt64, value String) ENGINE = Memory")
+ .get(10, TimeUnit.SECONDS).close();
- } catch (Exception e) {
- e.printStackTrace();
- Assert.fail("Async insert failed: " + e.getMessage());
- }
-
- // Insert using sync client
- try (Client syncClient = new Client.Builder()
- .addEndpoint(server.getBaseUri())
- .setUsername("default")
- .setPassword(getPassword())
- .useAsyncHttp(false)
- .build()) {
+ ByteArrayInputStream dataStream = new ByteArrayInputStream(csvData.getBytes(StandardCharsets.UTF_8));
+ asyncClient.insert(asyncTable, dataStream, ClickHouseFormat.CSV).get(10, TimeUnit.SECONDS);
+ }
- syncClient.query("CREATE TABLE " + syncTable + " (id UInt64, value String) ENGINE = Memory")
- .get(10, TimeUnit.SECONDS).close();
+ // Insert using sync client
+ try (Client syncClient = new Client.Builder()
+ .addEndpoint(server.getBaseUri())
+ .setUsername("default")
+ .setPassword(getPassword())
+ .useAsyncHttp(false)
+ .build()) {
- ByteArrayInputStream dataStream = new ByteArrayInputStream(csvData.getBytes(StandardCharsets.UTF_8));
- syncClient.insert(syncTable, dataStream, ClickHouseFormat.CSV).get(10, TimeUnit.SECONDS);
+ syncClient.query("CREATE TABLE " + syncTable + " (id UInt64, value String) ENGINE = Memory")
+ .get(10, TimeUnit.SECONDS).close();
- } catch (Exception e) {
- e.printStackTrace();
- Assert.fail("Sync insert failed: " + e.getMessage());
- }
+ ByteArrayInputStream dataStream = new ByteArrayInputStream(csvData.getBytes(StandardCharsets.UTF_8));
+ syncClient.insert(syncTable, dataStream, ClickHouseFormat.CSV).get(10, TimeUnit.SECONDS);
+ }
- // Compare results
- try (Client client = new Client.Builder()
- .addEndpoint(server.getBaseUri())
- .setUsername("default")
- .setPassword(getPassword())
- .build()) {
+ // Compare results
+ try (Client client = new Client.Builder()
+ .addEndpoint(server.getBaseUri())
+ .setUsername("default")
+ .setPassword(getPassword())
+ .build()) {
- List asyncRecords = client.queryAll("SELECT * FROM " + asyncTable + " ORDER BY id");
- List syncRecords = client.queryAll("SELECT * FROM " + syncTable + " ORDER BY id");
+ List asyncRecords = client.queryAll("SELECT * FROM " + asyncTable + " ORDER BY id");
+ List syncRecords = client.queryAll("SELECT * FROM " + syncTable + " ORDER BY id");
- Assert.assertEquals(asyncRecords.size(), syncRecords.size());
- for (int i = 0; i < asyncRecords.size(); i++) {
- Assert.assertEquals(asyncRecords.get(i).getLong("id"), syncRecords.get(i).getLong("id"));
- Assert.assertEquals(asyncRecords.get(i).getString("value"), syncRecords.get(i).getString("value"));
+ Assert.assertEquals(asyncRecords.size(), syncRecords.size());
+ for (int i = 0; i < asyncRecords.size(); i++) {
+ Assert.assertEquals(asyncRecords.get(i).getLong("id"), syncRecords.get(i).getLong("id"));
+ Assert.assertEquals(asyncRecords.get(i).getString("value"), syncRecords.get(i).getString("value"));
+ }
}
-
- // Cleanup
- client.query("DROP TABLE IF EXISTS " + asyncTable).get(10, TimeUnit.SECONDS).close();
- client.query("DROP TABLE IF EXISTS " + syncTable).get(10, TimeUnit.SECONDS).close();
-
} catch (Exception e) {
e.printStackTrace();
- Assert.fail("Comparison failed: " + e.getMessage());
+ Assert.fail(e.getMessage());
+ } finally {
+ // Cleanup tables regardless of test outcome
+ try (Client cleanupClient = new Client.Builder()
+ .addEndpoint(server.getBaseUri())
+ .setUsername("default")
+ .setPassword(getPassword())
+ .build()) {
+ cleanupClient.query("DROP TABLE IF EXISTS " + asyncTable).get(10, TimeUnit.SECONDS).close();
+ cleanupClient.query("DROP TABLE IF EXISTS " + syncTable).get(10, TimeUnit.SECONDS).close();
+ } catch (Exception ignored) {
+ // Cleanup errors shouldn't fail the test
+ }
}
}
From 060ef4524d3090a6399ac28d63d9f293a4c01892 Mon Sep 17 00:00:00 2001
From: David Karlsson <2795016+devdavidkarlsson@users.noreply.github.com>
Date: Tue, 17 Feb 2026 09:56:58 +0100
Subject: [PATCH 20/32] feat: improve async HTTP client production-readiness
1. Thread Pool for Compression:
- Replace per-request Thread creation with shared bounded ThreadPoolExecutor
- Pool size: max(2, availableProcessors), bounded queue of 1000
- CallerRunsPolicy provides backpressure when queue full
- Prevents thread explosion under high concurrency
2. Async Connection Pool Metrics:
- Add Micrometer metrics for async connection pool (same as sync)
- Exposes: max connections, available, leased, pending, max per route
- Tagged with "-async" suffix to distinguish from sync metrics
3. Per-Request Timeout Wrapper:
- Add applyAsyncTimeout() helper using CompletableFuture.orTimeout()
- Applied to async query() and insert() operations
- Uses network timeout from request settings (0 = no timeout)
Co-Authored-By: Claude Opus 4.5
---
.../com/clickhouse/client/api/Client.java | 49 ++-
.../api/internal/HttpAPIClientHelper.java | 17 +-
.../StreamingAsyncEntityProducer.java | 44 ++-
.../client/api/metrics/MicrometerLoader.java | 45 +++
.../client/AsyncHttpManualValidation.java | 365 ++++++++++++++++++
client-v2/src/test/resources/logback-test.xml | 16 +
6 files changed, 517 insertions(+), 19 deletions(-)
create mode 100644 client-v2/src/test/java/com/clickhouse/client/AsyncHttpManualValidation.java
create mode 100644 client-v2/src/test/resources/logback-test.xml
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/Client.java b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
index f641ad7f8..2b8be175d 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/Client.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
@@ -113,6 +113,14 @@
public class Client implements AutoCloseable {
private static final Logger LOG = LoggerFactory.getLogger(Client.class);
+ // Shared scheduler for async operation timeouts (Java 8 compatible alternative to orTimeout)
+ private static final java.util.concurrent.ScheduledExecutorService TIMEOUT_SCHEDULER =
+ java.util.concurrent.Executors.newSingleThreadScheduledExecutor(r -> {
+ Thread t = new Thread(r, "clickhouse-async-timeout");
+ t.setDaemon(true);
+ return t;
+ });
+
private HttpAPIClientHelper httpClientHelper = null;
private final List endpoints;
@@ -1452,7 +1460,8 @@ public CompletableFuture insert(String tableName,
// Use async path for InputStream-based inserts when async is enabled
if (httpClientHelper.isAsyncEnabled()) {
- return executeInsertAsync(tableName, columnNames, data, format, settings);
+ CompletableFuture future = executeInsertAsync(tableName, columnNames, data, format, settings);
+ return applyAsyncTimeout(future, settings.getNetworkTimeout());
}
return insert(tableName, columnNames, new DataStreamWriter() {
@@ -1749,7 +1758,8 @@ public CompletableFuture query(String sqlQuery, Map future = executeQueryAsync(sqlQuery, requestSettings, clientStats, 0);
+ return applyAsyncTimeout(future, requestSettings.getNetworkTimeout());
}
Supplier responseSupplier = () -> {
@@ -1879,6 +1889,41 @@ private static class AsyncRetryMarker {
this.nextAttempt = nextAttempt;
}
}
+
+ /**
+ * Applies a timeout to an async operation if configured.
+ * If timeout is 0 or negative, returns the original future unchanged.
+ * Java 8 compatible implementation (orTimeout requires Java 9+).
+ *
+ * @param future the future to wrap with timeout
+ * @param timeoutMs timeout in milliseconds (0 or negative means no timeout)
+ * @return future with timeout applied, or original future if no timeout
+ */
+ private CompletableFuture applyAsyncTimeout(CompletableFuture future, long timeoutMs) {
+ if (timeoutMs <= 0) {
+ return future;
+ }
+ // Java 8 compatible timeout implementation using shared scheduler
+ CompletableFuture timeoutFuture = new CompletableFuture<>();
+ java.util.concurrent.ScheduledFuture> scheduled = TIMEOUT_SCHEDULER.schedule(() -> {
+ if (!future.isDone()) {
+ timeoutFuture.completeExceptionally(
+ new TimeoutException("Async operation timed out after " + timeoutMs + "ms"));
+ }
+ }, timeoutMs, TimeUnit.MILLISECONDS);
+
+ future.whenComplete((result, ex) -> {
+ scheduled.cancel(false);
+ if (ex != null) {
+ timeoutFuture.completeExceptionally(ex);
+ } else {
+ timeoutFuture.complete(result);
+ }
+ });
+
+ return timeoutFuture;
+ }
+
public CompletableFuture query(String sqlQuery, Map queryParams) {
return query(sqlQuery, queryParams, null);
}
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
index b43add45c..9757dae0b 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
@@ -35,6 +35,7 @@
import org.apache.hc.client5.http.impl.io.ManagedHttpClientConnectionFactory;
import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManager;
import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManagerBuilder;
+import org.apache.hc.client5.http.impl.nio.PoolingAsyncClientConnectionManager;
import org.apache.hc.client5.http.impl.nio.PoolingAsyncClientConnectionManagerBuilder;
import org.apache.hc.client5.http.io.HttpClientConnectionManager;
import org.apache.hc.client5.http.io.ManagedHttpClientConnection;
@@ -429,7 +430,21 @@ public CloseableHttpAsyncClient createHttpAsyncClient(boolean initSslContext, Ma
connMgrBuilder.setTlsStrategy(tlsStrategy);
}
- asyncBuilder.setConnectionManager(connMgrBuilder.build());
+ PoolingAsyncClientConnectionManager asyncConnMgr = connMgrBuilder.build();
+ asyncBuilder.setConnectionManager(asyncConnMgr);
+
+ // Register metrics for async connection pool
+ if (metricsRegistry != null) {
+ try {
+ String mGroupName = ClientConfigProperties.METRICS_GROUP_NAME.getOrDefault(configuration);
+ Class> micrometerLoader = getClass().getClassLoader().loadClass("com.clickhouse.client.api.metrics.MicrometerLoader");
+ Method applyMethod = micrometerLoader.getDeclaredMethod("applyAsyncPoolingMetricsBinder",
+ Object.class, String.class, PoolingAsyncClientConnectionManager.class);
+ applyMethod.invoke(micrometerLoader, metricsRegistry, mGroupName, asyncConnMgr);
+ } catch (Exception e) {
+ LOG.error("Failed to register async connection pool metrics", e);
+ }
+ }
String proxyHost = (String) configuration.get(ClientConfigProperties.PROXY_HOST.getKey());
Integer proxyPort = (Integer) configuration.get(ClientConfigProperties.PROXY_PORT.getKey());
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncEntityProducer.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncEntityProducer.java
index 1431927c7..23c834b73 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncEntityProducer.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncEntityProducer.java
@@ -14,6 +14,11 @@
import java.io.PipedOutputStream;
import java.nio.ByteBuffer;
import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
@@ -31,6 +36,24 @@ public class StreamingAsyncEntityProducer implements AsyncEntityProducer {
private static final int PIPE_BUFFER_SIZE = 512 * 1024; // 512KB pipe buffer
private static final AtomicLong THREAD_COUNTER = new AtomicLong(0); // For unique thread names
+ // Shared thread pool for compression tasks - bounded to prevent thread explosion under high concurrency
+ private static final int COMPRESSION_POOL_SIZE = Math.max(2, Runtime.getRuntime().availableProcessors());
+ private static final ExecutorService COMPRESSION_EXECUTOR = new ThreadPoolExecutor(
+ COMPRESSION_POOL_SIZE,
+ COMPRESSION_POOL_SIZE,
+ 60L, TimeUnit.SECONDS,
+ new LinkedBlockingQueue<>(1000), // Bounded queue to provide backpressure
+ new ThreadFactory() {
+ @Override
+ public Thread newThread(Runnable r) {
+ Thread t = new Thread(r, "async-compression-" + THREAD_COUNTER.incrementAndGet());
+ t.setDaemon(true);
+ return t;
+ }
+ },
+ new ThreadPoolExecutor.CallerRunsPolicy() // If queue full, run in caller thread (backpressure)
+ );
+
private final ContentType contentType;
private final InputStream sourceStream;
private final boolean compressData;
@@ -42,9 +65,8 @@ public class StreamingAsyncEntityProducer implements AsyncEntityProducer {
private final AtomicBoolean completed = new AtomicBoolean(false);
private final AtomicReference error = new AtomicReference<>();
- // For compression: compress in background thread, read compressed data here
+ // For compression: compress in thread pool, read compressed data here
private PipedInputStream compressedInputStream;
- private Thread compressionThread;
private InputStream activeInputStream;
public StreamingAsyncEntityProducer(InputStream sourceStream, ContentType contentType) {
@@ -75,8 +97,8 @@ private void initializeStreams() throws IOException {
compressedInputStream = new PipedInputStream(compressedOutputStream, PIPE_BUFFER_SIZE);
activeInputStream = compressedInputStream;
- // Start compression in background thread
- compressionThread = new Thread(() -> {
+ // Submit compression task to shared thread pool
+ COMPRESSION_EXECUTOR.submit(() -> {
try {
OutputStream compressingStream;
if (useHttpCompression) {
@@ -106,9 +128,7 @@ private void initializeStreams() throws IOException {
} catch (IOException ignored) {
}
}
- }, "async-compression-thread-" + THREAD_COUNTER.incrementAndGet());
- compressionThread.setDaemon(true);
- compressionThread.start();
+ });
} else {
// No compression - read directly from source
activeInputStream = sourceStream;
@@ -201,6 +221,7 @@ public void failed(Exception cause) {
@Override
public void releaseResources() {
completed.set(true);
+ // Closing streams will cause any running compression task to fail and exit
try {
if (activeInputStream != null) {
activeInputStream.close();
@@ -211,14 +232,5 @@ public void releaseResources() {
} catch (IOException e) {
LOG.debug("Error closing streams", e);
}
-
- if (compressionThread != null && compressionThread.isAlive()) {
- compressionThread.interrupt();
- try {
- compressionThread.join(1000); // Wait up to 1 second for clean shutdown
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- }
- }
}
}
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/metrics/MicrometerLoader.java b/client-v2/src/main/java/com/clickhouse/client/api/metrics/MicrometerLoader.java
index 00f6c0344..b77bc92e5 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/metrics/MicrometerLoader.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/metrics/MicrometerLoader.java
@@ -7,6 +7,7 @@
import io.micrometer.core.instrument.Tag;
import io.micrometer.core.instrument.Tags;
import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManager;
+import org.apache.hc.client5.http.impl.nio.PoolingAsyncClientConnectionManager;
import org.apache.hc.core5.pool.ConnPoolControl;
public class MicrometerLoader {
@@ -63,4 +64,48 @@ public static void applyConnectionMetricsBinder(Object registry, String metricsG
}
}
+ /**
+ * Registers Micrometer metrics for the async HTTP client connection pool.
+ */
+ public static void applyAsyncPoolingMetricsBinder(Object registry, String metricsGroupName, PoolingAsyncClientConnectionManager pacm) {
+ if (registry instanceof MeterRegistry) {
+ Iterable tags = Tags.of("httpclient", metricsGroupName + "-async");
+ Gauge
+ .builder("httpcomponents.httpclient.pool.total.max", pacm,
+ (connPoolControl) -> connPoolControl.getTotalStats().getMax())
+ .description("The configured maximum number of allowed persistent connections for all routes (async).")
+ .tags(tags)
+ .register((MeterRegistry) registry);
+ Gauge
+ .builder("httpcomponents.httpclient.pool.total.connections", pacm,
+ (connPoolControl) -> connPoolControl.getTotalStats().getAvailable())
+ .description("The number of persistent and available connections for all routes (async).")
+ .tags(tags)
+ .tag("state", "available")
+ .register((MeterRegistry) registry);
+ Gauge
+ .builder("httpcomponents.httpclient.pool.total.connections", pacm,
+ (connPoolControl) -> connPoolControl.getTotalStats().getLeased())
+ .description("The number of persistent and leased connections for all routes (async).")
+ .tags(tags)
+ .tag("state", "leased")
+ .register((MeterRegistry) registry);
+ Gauge
+ .builder("httpcomponents.httpclient.pool.total.pending", pacm,
+ (connPoolControl) -> connPoolControl.getTotalStats().getPending())
+ .description("The number of connection requests being blocked awaiting a free connection for all routes (async).")
+ .tags(tags)
+ .register((MeterRegistry) registry);
+ Gauge
+ .builder("httpcomponents.httpclient.pool.route.max.default", pacm,
+ ConnPoolControl::getDefaultMaxPerRoute)
+ .description("The configured default maximum number of allowed persistent connections per route (async).")
+ .tags(tags)
+ .register((MeterRegistry) registry);
+
+ } else {
+ throw new ClientMisconfigurationException("Unsupported registry type." + registry.getClass());
+ }
+ }
+
}
diff --git a/client-v2/src/test/java/com/clickhouse/client/AsyncHttpManualValidation.java b/client-v2/src/test/java/com/clickhouse/client/AsyncHttpManualValidation.java
new file mode 100644
index 000000000..52e1fd1c3
--- /dev/null
+++ b/client-v2/src/test/java/com/clickhouse/client/AsyncHttpManualValidation.java
@@ -0,0 +1,365 @@
+package com.clickhouse.client;
+
+import com.clickhouse.client.api.Client;
+import com.clickhouse.client.api.insert.InsertResponse;
+import com.clickhouse.client.api.query.GenericRecord;
+import com.clickhouse.client.api.query.QueryResponse;
+import com.clickhouse.client.api.query.QuerySettings;
+import com.clickhouse.data.ClickHouseFormat;
+
+import java.io.BufferedReader;
+import java.io.ByteArrayInputStream;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Manual validation script for Async HTTP features.
+ *
+ * Run with:
+ * mvn exec:java -pl client-v2 \
+ * -Dexec.mainClass="com.clickhouse.client.AsyncHttpManualValidation" \
+ * -Dexec.classpathScope=test \
+ * -Dexec.args="http://localhost:8123 default password"
+ *
+ * Or for ClickHouse Cloud:
+ * mvn exec:java -pl client-v2 \
+ * -Dexec.mainClass="com.clickhouse.client.AsyncHttpManualValidation" \
+ * -Dexec.classpathScope=test \
+ * -Dexec.args="https://your-host.clickhouse.cloud:8443 default your-password"
+ */
+public class AsyncHttpManualValidation {
+
+ private static int passed = 0;
+ private static int failed = 0;
+
+ public static void main(String[] args) {
+ String endpoint = args.length > 0 ? args[0] : "http://localhost:8123";
+ String username = args.length > 1 ? args[1] : "default";
+ String password = args.length > 2 ? args[2] : "";
+
+ System.out.println("============================================================");
+ System.out.println("Async HTTP Manual Validation");
+ System.out.println("============================================================");
+ System.out.println("Endpoint: " + endpoint);
+ System.out.println("Username: " + username);
+ System.out.println();
+
+ // Test with SYNC client first (baseline)
+ System.out.println("--- SYNC CLIENT (Baseline) ---");
+ try (Client syncClient = new Client.Builder()
+ .addEndpoint(endpoint)
+ .setUsername(username)
+ .setPassword(password)
+ .useAsyncHttp(false)
+ .build()) {
+
+ testBasicQuery(syncClient, "SYNC");
+ } catch (Exception e) {
+ System.out.println("SYNC baseline failed: " + e.getMessage());
+ }
+
+ System.out.println();
+ System.out.println("--- ASYNC CLIENT ---");
+
+ // Test with ASYNC client
+ try (Client asyncClient = new Client.Builder()
+ .addEndpoint(endpoint)
+ .setUsername(username)
+ .setPassword(password)
+ .useAsyncHttp(true)
+ .build()) {
+
+ // Phase 1: Basic async queries
+ testBasicQuery(asyncClient, "ASYNC");
+ testQueryMetrics(asyncClient);
+ testConcurrentQueries(asyncClient);
+
+ // Phase 2: Streaming responses
+ testStreamingResponse(asyncClient);
+ testLargeResultStreaming(asyncClient);
+
+ // Phase 3: Request compression
+ testQueryWithHttpCompression(asyncClient, endpoint, username, password);
+ testQueryWithNativeLZ4(asyncClient, endpoint, username, password);
+
+ // Phase 4: Async inserts
+ testBasicInsert(asyncClient);
+ testLargeInsert(asyncClient);
+ testInsertWithCompression(asyncClient, endpoint, username, password);
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ failed++;
+ }
+
+ System.out.println();
+ System.out.println("============================================================");
+ System.out.println("RESULTS: " + passed + " passed, " + failed + " failed");
+ System.out.println("============================================================");
+
+ System.exit(failed > 0 ? 1 : 0);
+ }
+
+ private static void testBasicQuery(Client client, String mode) {
+ String testName = mode + " Basic Query";
+ try {
+ List records = client.queryAll("SELECT 1 as num, 'hello' as greeting");
+ if (records.size() == 1 && records.get(0).getLong("num") == 1) {
+ pass(testName);
+ } else {
+ fail(testName, "Unexpected result: " + records);
+ }
+ } catch (Exception e) {
+ fail(testName, e);
+ }
+ }
+
+ private static void testQueryMetrics(Client client) {
+ String testName = "Query Metrics";
+ try {
+ QueryResponse response = client.query("SELECT number FROM numbers(100)").get(30, TimeUnit.SECONDS);
+ if (response.getReadRows() > 0 && response.getQueryId() != null) {
+ pass(testName + " (rows=" + response.getReadRows() + ", queryId=" + response.getQueryId() + ")");
+ } else {
+ fail(testName, "Missing metrics");
+ }
+ response.close();
+ } catch (Exception e) {
+ fail(testName, e);
+ }
+ }
+
+ private static void testConcurrentQueries(Client client) {
+ String testName = "Concurrent Queries (10)";
+ try {
+ @SuppressWarnings("unchecked")
+ CompletableFuture[] futures = new CompletableFuture[10];
+ for (int i = 0; i < 10; i++) {
+ futures[i] = client.query("SELECT " + i + " as num, sleep(0.05)");
+ }
+ CompletableFuture.allOf(futures).get(60, TimeUnit.SECONDS);
+
+ int successCount = 0;
+ for (CompletableFuture f : futures) {
+ if (f.get().getReadRows() > 0) successCount++;
+ f.get().close();
+ }
+
+ if (successCount == 10) {
+ pass(testName);
+ } else {
+ fail(testName, "Only " + successCount + "/10 succeeded");
+ }
+ } catch (Exception e) {
+ fail(testName, e);
+ }
+ }
+
+ private static void testStreamingResponse(Client client) {
+ String testName = "Streaming Response";
+ try {
+ // Use TabSeparated format so we can count lines with BufferedReader
+ QuerySettings settings = new QuerySettings().setFormat(ClickHouseFormat.TabSeparated);
+ QueryResponse response = client.query("SELECT number FROM numbers(1000)", settings).get(30, TimeUnit.SECONDS);
+ BufferedReader reader = new BufferedReader(new InputStreamReader(response.getInputStream()));
+ int lineCount = 0;
+ while (reader.readLine() != null) lineCount++;
+ response.close();
+
+ if (lineCount == 1000) {
+ pass(testName + " (1000 rows streamed)");
+ } else {
+ fail(testName, "Expected 1000 rows, got " + lineCount);
+ }
+ } catch (Exception e) {
+ fail(testName, e);
+ }
+ }
+
+ private static void testLargeResultStreaming(Client client) {
+ String testName = "Large Result Streaming (100K rows)";
+ try {
+ long startMem = Runtime.getRuntime().totalMemory() - Runtime.getRuntime().freeMemory();
+
+ // Use TabSeparated format so we can count lines with BufferedReader
+ QuerySettings settings = new QuerySettings().setFormat(ClickHouseFormat.TabSeparated);
+ QueryResponse response = client.query("SELECT number, toString(number) FROM numbers(100000)", settings)
+ .get(60, TimeUnit.SECONDS);
+ BufferedReader reader = new BufferedReader(new InputStreamReader(response.getInputStream()));
+ int lineCount = 0;
+ while (reader.readLine() != null) lineCount++;
+ response.close();
+
+ long endMem = Runtime.getRuntime().totalMemory() - Runtime.getRuntime().freeMemory();
+ long memUsed = (endMem - startMem) / 1024 / 1024;
+
+ if (lineCount == 100000) {
+ pass(testName + " (mem delta ~" + memUsed + "MB)");
+ } else {
+ fail(testName, "Expected 100000 rows, got " + lineCount);
+ }
+ } catch (Exception e) {
+ fail(testName, e);
+ }
+ }
+
+ private static void testQueryWithHttpCompression(Client client, String endpoint, String user, String pass) {
+ String testName = "Query with HTTP LZ4 Compression";
+ try (Client compressClient = new Client.Builder()
+ .addEndpoint(endpoint)
+ .setUsername(user)
+ .setPassword(pass)
+ .useAsyncHttp(true)
+ .compressClientRequest(true)
+ .useHttpCompression(true)
+ .build()) {
+
+ List records = compressClient.queryAll("SELECT number FROM numbers(100)");
+ if (records.size() == 100) {
+ pass(testName);
+ } else {
+ fail(testName, "Expected 100 rows, got " + records.size());
+ }
+ } catch (Exception e) {
+ fail(testName, e);
+ }
+ }
+
+ private static void testQueryWithNativeLZ4(Client client, String endpoint, String user, String pass) {
+ String testName = "Query with Native LZ4 Compression";
+ try (Client compressClient = new Client.Builder()
+ .addEndpoint(endpoint)
+ .setUsername(user)
+ .setPassword(pass)
+ .useAsyncHttp(true)
+ .compressClientRequest(true)
+ .useHttpCompression(false)
+ .build()) {
+
+ List records = compressClient.queryAll("SELECT number FROM numbers(50)");
+ if (records.size() == 50) {
+ pass(testName);
+ } else {
+ fail(testName, "Expected 50 rows, got " + records.size());
+ }
+ } catch (Exception e) {
+ fail(testName, e);
+ }
+ }
+
+ private static void testBasicInsert(Client client) {
+ String testName = "Basic Async Insert";
+ String tableName = "async_test_basic_" + System.currentTimeMillis();
+ try {
+ client.query("CREATE TABLE " + tableName + " (id UInt64, name String) ENGINE = Memory")
+ .get(10, TimeUnit.SECONDS).close();
+
+ String csvData = "1,Alice\n2,Bob\n3,Charlie\n";
+ ByteArrayInputStream stream = new ByteArrayInputStream(csvData.getBytes(StandardCharsets.UTF_8));
+ InsertResponse response = client.insert(tableName, stream, ClickHouseFormat.CSV)
+ .get(10, TimeUnit.SECONDS);
+
+ List records = client.queryAll("SELECT count() FROM " + tableName);
+ long count = records.get(0).getLong(1);
+
+ client.query("DROP TABLE " + tableName).get(10, TimeUnit.SECONDS).close();
+
+ if (count == 3) {
+ pass(testName + " (3 rows inserted)");
+ } else {
+ fail(testName, "Expected 3 rows, got " + count);
+ }
+ } catch (Exception e) {
+ fail(testName, e);
+ try { client.query("DROP TABLE IF EXISTS " + tableName).get(5, TimeUnit.SECONDS); } catch (Exception ignored) {}
+ }
+ }
+
+ private static void testLargeInsert(Client client) {
+ String testName = "Large Async Insert (10K rows)";
+ String tableName = "async_test_large_" + System.currentTimeMillis();
+ try {
+ client.query("CREATE TABLE " + tableName + " (id UInt64, data String) ENGINE = Memory")
+ .get(10, TimeUnit.SECONDS).close();
+
+ StringBuilder csv = new StringBuilder();
+ for (int i = 0; i < 10000; i++) {
+ csv.append(i).append(",data_row_").append(i).append("\n");
+ }
+ ByteArrayInputStream stream = new ByteArrayInputStream(csv.toString().getBytes(StandardCharsets.UTF_8));
+
+ long start = System.currentTimeMillis();
+ client.insert(tableName, stream, ClickHouseFormat.CSV).get(60, TimeUnit.SECONDS);
+ long elapsed = System.currentTimeMillis() - start;
+
+ List records = client.queryAll("SELECT count() FROM " + tableName);
+ long count = records.get(0).getLong(1);
+
+ client.query("DROP TABLE " + tableName).get(10, TimeUnit.SECONDS).close();
+
+ if (count == 10000) {
+ pass(testName + " (" + elapsed + "ms)");
+ } else {
+ fail(testName, "Expected 10000 rows, got " + count);
+ }
+ } catch (Exception e) {
+ fail(testName, e);
+ try { client.query("DROP TABLE IF EXISTS " + tableName).get(5, TimeUnit.SECONDS); } catch (Exception ignored) {}
+ }
+ }
+
+ private static void testInsertWithCompression(Client client, String endpoint, String user, String pass) {
+ String testName = "Insert with Compression";
+ String tableName = "async_test_compress_" + System.currentTimeMillis();
+ try (Client compressClient = new Client.Builder()
+ .addEndpoint(endpoint)
+ .setUsername(user)
+ .setPassword(pass)
+ .useAsyncHttp(true)
+ .compressClientRequest(true)
+ .useHttpCompression(true)
+ .build()) {
+
+ compressClient.query("CREATE TABLE " + tableName + " (id UInt64, value String) ENGINE = Memory")
+ .get(10, TimeUnit.SECONDS).close();
+
+ StringBuilder csv = new StringBuilder();
+ for (int i = 0; i < 1000; i++) {
+ csv.append(i).append(",value_").append(i).append("\n");
+ }
+ ByteArrayInputStream stream = new ByteArrayInputStream(csv.toString().getBytes(StandardCharsets.UTF_8));
+ compressClient.insert(tableName, stream, ClickHouseFormat.CSV).get(30, TimeUnit.SECONDS);
+
+ List records = compressClient.queryAll("SELECT count() FROM " + tableName);
+ long count = records.get(0).getLong(1);
+
+ compressClient.query("DROP TABLE " + tableName).get(10, TimeUnit.SECONDS).close();
+
+ if (count == 1000) {
+ pass(testName + " (1000 rows)");
+ } else {
+ fail(testName, "Expected 1000 rows, got " + count);
+ }
+ } catch (Exception e) {
+ fail(testName, e);
+ }
+ }
+
+ private static void pass(String testName) {
+ System.out.println("[PASS] " + testName);
+ passed++;
+ }
+
+ private static void fail(String testName, String reason) {
+ System.out.println("[FAIL] " + testName + " - " + reason);
+ failed++;
+ }
+
+ private static void fail(String testName, Exception e) {
+ System.out.println("[FAIL] " + testName + " - " + e.getClass().getSimpleName() + ": " + e.getMessage());
+ failed++;
+ }
+}
diff --git a/client-v2/src/test/resources/logback-test.xml b/client-v2/src/test/resources/logback-test.xml
new file mode 100644
index 000000000..0d5c146a2
--- /dev/null
+++ b/client-v2/src/test/resources/logback-test.xml
@@ -0,0 +1,16 @@
+
+
+
+
+ %d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n
+
+
+
+
+
+
+
+
+
+
+
From c4662af235bd7090797ec4536d38d8370139d6fb Mon Sep 17 00:00:00 2001
From: David Karlsson <2795016+devdavidkarlsson@users.noreply.github.com>
Date: Tue, 17 Feb 2026 11:01:47 +0100
Subject: [PATCH 21/32] fix: address Copilot review comments on static
executors and test cleanup
1. Document static TIMEOUT_SCHEDULER and COMPRESSION_EXECUTOR:
- Explain they use daemon threads that terminate with JVM
- Document intentional design avoiding complex lifecycle management
2. Fix test table cleanup in AsyncHttpManualValidation:
- Move DROP TABLE to finally blocks in testBasicInsert, testLargeInsert,
and testInsertWithCompression
- Ensures cleanup happens regardless of test success/failure
Co-Authored-By: Claude Opus 4.5
---
.../com/clickhouse/client/api/Client.java | 8 ++++-
.../StreamingAsyncEntityProducer.java | 8 ++++-
.../client/AsyncHttpManualValidation.java | 35 ++++++++++---------
3 files changed, 33 insertions(+), 18 deletions(-)
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/Client.java b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
index 2b8be175d..bcd9c43e1 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/Client.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
@@ -113,7 +113,13 @@
public class Client implements AutoCloseable {
private static final Logger LOG = LoggerFactory.getLogger(Client.class);
- // Shared scheduler for async operation timeouts (Java 8 compatible alternative to orTimeout)
+ /**
+ * Shared scheduler for async operation timeouts (Java 8 compatible alternative to orTimeout).
+ * Uses daemon threads so it won't prevent JVM shutdown. This is intentionally static and
+ * not shut down explicitly - the daemon threads will terminate when the JVM exits.
+ * This design avoids complex lifecycle management while ensuring resources are not leaked
+ * beyond JVM termination.
+ */
private static final java.util.concurrent.ScheduledExecutorService TIMEOUT_SCHEDULER =
java.util.concurrent.Executors.newSingleThreadScheduledExecutor(r -> {
Thread t = new Thread(r, "clickhouse-async-timeout");
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncEntityProducer.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncEntityProducer.java
index 23c834b73..37a9ea433 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncEntityProducer.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncEntityProducer.java
@@ -36,7 +36,13 @@ public class StreamingAsyncEntityProducer implements AsyncEntityProducer {
private static final int PIPE_BUFFER_SIZE = 512 * 1024; // 512KB pipe buffer
private static final AtomicLong THREAD_COUNTER = new AtomicLong(0); // For unique thread names
- // Shared thread pool for compression tasks - bounded to prevent thread explosion under high concurrency
+ /**
+ * Shared thread pool for compression tasks - bounded to prevent thread explosion under high concurrency.
+ * Uses daemon threads so it won't prevent JVM shutdown. This is intentionally static and shared
+ * across all StreamingAsyncEntityProducer instances to limit total thread count. The pool is not
+ * shut down explicitly - daemon threads will terminate when the JVM exits. This design provides
+ * efficient resource sharing while avoiding complex lifecycle management.
+ */
private static final int COMPRESSION_POOL_SIZE = Math.max(2, Runtime.getRuntime().availableProcessors());
private static final ExecutorService COMPRESSION_EXECUTOR = new ThreadPoolExecutor(
COMPRESSION_POOL_SIZE,
diff --git a/client-v2/src/test/java/com/clickhouse/client/AsyncHttpManualValidation.java b/client-v2/src/test/java/com/clickhouse/client/AsyncHttpManualValidation.java
index 52e1fd1c3..8e86de966 100644
--- a/client-v2/src/test/java/com/clickhouse/client/AsyncHttpManualValidation.java
+++ b/client-v2/src/test/java/com/clickhouse/client/AsyncHttpManualValidation.java
@@ -265,8 +265,6 @@ private static void testBasicInsert(Client client) {
List records = client.queryAll("SELECT count() FROM " + tableName);
long count = records.get(0).getLong(1);
- client.query("DROP TABLE " + tableName).get(10, TimeUnit.SECONDS).close();
-
if (count == 3) {
pass(testName + " (3 rows inserted)");
} else {
@@ -274,7 +272,8 @@ private static void testBasicInsert(Client client) {
}
} catch (Exception e) {
fail(testName, e);
- try { client.query("DROP TABLE IF EXISTS " + tableName).get(5, TimeUnit.SECONDS); } catch (Exception ignored) {}
+ } finally {
+ try { client.query("DROP TABLE IF EXISTS " + tableName).get(5, TimeUnit.SECONDS).close(); } catch (Exception ignored) {}
}
}
@@ -298,8 +297,6 @@ private static void testLargeInsert(Client client) {
List records = client.queryAll("SELECT count() FROM " + tableName);
long count = records.get(0).getLong(1);
- client.query("DROP TABLE " + tableName).get(10, TimeUnit.SECONDS).close();
-
if (count == 10000) {
pass(testName + " (" + elapsed + "ms)");
} else {
@@ -307,21 +304,24 @@ private static void testLargeInsert(Client client) {
}
} catch (Exception e) {
fail(testName, e);
- try { client.query("DROP TABLE IF EXISTS " + tableName).get(5, TimeUnit.SECONDS); } catch (Exception ignored) {}
+ } finally {
+ try { client.query("DROP TABLE IF EXISTS " + tableName).get(5, TimeUnit.SECONDS).close(); } catch (Exception ignored) {}
}
}
private static void testInsertWithCompression(Client client, String endpoint, String user, String pass) {
String testName = "Insert with Compression";
String tableName = "async_test_compress_" + System.currentTimeMillis();
- try (Client compressClient = new Client.Builder()
- .addEndpoint(endpoint)
- .setUsername(user)
- .setPassword(pass)
- .useAsyncHttp(true)
- .compressClientRequest(true)
- .useHttpCompression(true)
- .build()) {
+ Client compressClient = null;
+ try {
+ compressClient = new Client.Builder()
+ .addEndpoint(endpoint)
+ .setUsername(user)
+ .setPassword(pass)
+ .useAsyncHttp(true)
+ .compressClientRequest(true)
+ .useHttpCompression(true)
+ .build();
compressClient.query("CREATE TABLE " + tableName + " (id UInt64, value String) ENGINE = Memory")
.get(10, TimeUnit.SECONDS).close();
@@ -336,8 +336,6 @@ private static void testInsertWithCompression(Client client, String endpoint, St
List records = compressClient.queryAll("SELECT count() FROM " + tableName);
long count = records.get(0).getLong(1);
- compressClient.query("DROP TABLE " + tableName).get(10, TimeUnit.SECONDS).close();
-
if (count == 1000) {
pass(testName + " (1000 rows)");
} else {
@@ -345,6 +343,11 @@ private static void testInsertWithCompression(Client client, String endpoint, St
}
} catch (Exception e) {
fail(testName, e);
+ } finally {
+ if (compressClient != null) {
+ try { compressClient.query("DROP TABLE IF EXISTS " + tableName).get(5, TimeUnit.SECONDS).close(); } catch (Exception ignored) {}
+ try { compressClient.close(); } catch (Exception ignored) {}
+ }
}
}
From f7a163304ae3a12836620ebcd441fb9dbbb8ab5c Mon Sep 17 00:00:00 2001
From: David Karlsson <2795016+devdavidkarlsson@users.noreply.github.com>
Date: Tue, 17 Feb 2026 11:22:27 +0100
Subject: [PATCH 22/32] feat: add proper lifecycle management for async
executors
- Implement reference-counted shared executors for compression and timeout
- StreamingAsyncEntityProducer: acquireExecutor/releaseExecutor for compression thread pool
- Client: acquireTimeoutScheduler/releaseTimeoutScheduler for timeout scheduler
- Graceful shutdown when last async client is closed (5s timeout then force)
- Executors are recreated if a new async client is created after shutdown
This ensures proper resource cleanup when clients are closed, while still
sharing executors efficiently across multiple concurrent async clients.
Co-Authored-By: Claude Opus 4.5
---
.../com/clickhouse/client/api/Client.java | 77 +++++++++++--
.../StreamingAsyncEntityProducer.java | 107 ++++++++++++++----
2 files changed, 152 insertions(+), 32 deletions(-)
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/Client.java b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
index bcd9c43e1..986dcd975 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/Client.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
@@ -18,6 +18,7 @@
import com.clickhouse.client.api.internal.ClientStatisticsHolder;
import com.clickhouse.client.api.internal.HttpAPIClientHelper;
import com.clickhouse.client.api.internal.MapUtils;
+import com.clickhouse.client.api.internal.StreamingAsyncEntityProducer;
import com.clickhouse.client.api.internal.TableSchemaParser;
import com.clickhouse.client.api.internal.ValidationUtils;
import com.clickhouse.client.api.metadata.ColumnToMethodMatchingStrategy;
@@ -115,20 +116,52 @@ public class Client implements AutoCloseable {
/**
* Shared scheduler for async operation timeouts (Java 8 compatible alternative to orTimeout).
- * Uses daemon threads so it won't prevent JVM shutdown. This is intentionally static and
- * not shut down explicitly - the daemon threads will terminate when the JVM exits.
- * This design avoids complex lifecycle management while ensuring resources are not leaked
- * beyond JVM termination.
+ * Uses daemon threads as a fallback so it won't prevent JVM shutdown if not properly released.
+ *
+ * Lifecycle management: The scheduler is lazily created when the first async client is created
+ * and shut down when the last async client is closed. This ensures graceful resource cleanup.
*/
- private static final java.util.concurrent.ScheduledExecutorService TIMEOUT_SCHEDULER =
- java.util.concurrent.Executors.newSingleThreadScheduledExecutor(r -> {
- Thread t = new Thread(r, "clickhouse-async-timeout");
- t.setDaemon(true);
- return t;
- });
+ private static final Object TIMEOUT_SCHEDULER_LOCK = new Object();
+ private static final java.util.concurrent.atomic.AtomicInteger TIMEOUT_SCHEDULER_REF_COUNT =
+ new java.util.concurrent.atomic.AtomicInteger(0);
+ private static volatile java.util.concurrent.ScheduledExecutorService timeoutScheduler = null;
+
+ private static void acquireTimeoutScheduler() {
+ synchronized (TIMEOUT_SCHEDULER_LOCK) {
+ if (TIMEOUT_SCHEDULER_REF_COUNT.getAndIncrement() == 0) {
+ timeoutScheduler = java.util.concurrent.Executors.newSingleThreadScheduledExecutor(r -> {
+ Thread t = new Thread(r, "ch-async-timeout");
+ t.setDaemon(true);
+ return t;
+ });
+ LOG.debug("Created async timeout scheduler");
+ }
+ }
+ }
+
+ private static void releaseTimeoutScheduler() {
+ synchronized (TIMEOUT_SCHEDULER_LOCK) {
+ if (TIMEOUT_SCHEDULER_REF_COUNT.decrementAndGet() == 0 && timeoutScheduler != null) {
+ LOG.debug("Shutting down async timeout scheduler");
+ timeoutScheduler.shutdown();
+ try {
+ if (!timeoutScheduler.awaitTermination(5, TimeUnit.SECONDS)) {
+ timeoutScheduler.shutdownNow();
+ }
+ } catch (InterruptedException e) {
+ timeoutScheduler.shutdownNow();
+ Thread.currentThread().interrupt();
+ }
+ timeoutScheduler = null;
+ }
+ }
+ }
private HttpAPIClientHelper httpClientHelper = null;
+ /** Tracks whether this client instance uses async HTTP (for proper executor cleanup) */
+ private final boolean usesAsyncHttp;
+
private final List endpoints;
private final Map configuration;
@@ -209,6 +242,13 @@ private Client(Collection endpoints, Map configuration,
this.serverVersion = configuration.getOrDefault(ClientConfigProperties.SERVER_VERSION.getKey(), "unknown");
this.dbUser = configuration.getOrDefault(ClientConfigProperties.USER.getKey(), ClientConfigProperties.USER.getDefObjVal());
this.typeHintMapping = (Map>) this.configuration.get(ClientConfigProperties.TYPE_HINT_MAPPING.getKey());
+
+ // Acquire shared async resources if async HTTP is enabled
+ this.usesAsyncHttp = httpClientHelper.isAsyncEnabled();
+ if (this.usesAsyncHttp) {
+ acquireTimeoutScheduler();
+ StreamingAsyncEntityProducer.acquireExecutor();
+ }
}
/**
@@ -246,6 +286,10 @@ public String getDefaultDatabase() {
* Frees the resources associated with the client.
*
* - Shuts down the shared operation executor by calling {@code shutdownNow()}
+ * - Closes the HTTP client helper
+ * - Releases shared async resources (compression executor, timeout scheduler) if this
+ * client was using async HTTP. When the last async client is closed, these shared
+ * resources are gracefully shut down.
*
*/
@Override
@@ -265,6 +309,12 @@ public void close() {
if (httpClientHelper != null) {
httpClientHelper.close();
}
+
+ // Release shared async resources if this client was using them
+ if (usesAsyncHttp) {
+ StreamingAsyncEntityProducer.releaseExecutor();
+ releaseTimeoutScheduler();
+ }
}
@@ -1910,8 +1960,13 @@ private CompletableFuture applyAsyncTimeout(CompletableFuture future,
return future;
}
// Java 8 compatible timeout implementation using shared scheduler
+ java.util.concurrent.ScheduledExecutorService scheduler = timeoutScheduler;
+ if (scheduler == null || scheduler.isShutdown()) {
+ LOG.warn("Timeout scheduler not available - timeout will not be applied");
+ return future;
+ }
CompletableFuture timeoutFuture = new CompletableFuture<>();
- java.util.concurrent.ScheduledFuture> scheduled = TIMEOUT_SCHEDULER.schedule(() -> {
+ java.util.concurrent.ScheduledFuture> scheduled = scheduler.schedule(() -> {
if (!future.isDone()) {
timeoutFuture.completeExceptionally(
new TimeoutException("Async operation timed out after " + timeoutMs + "ms"));
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncEntityProducer.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncEntityProducer.java
index 37a9ea433..b348c39c5 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncEntityProducer.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncEntityProducer.java
@@ -20,6 +20,7 @@
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
@@ -38,27 +39,81 @@ public class StreamingAsyncEntityProducer implements AsyncEntityProducer {
/**
* Shared thread pool for compression tasks - bounded to prevent thread explosion under high concurrency.
- * Uses daemon threads so it won't prevent JVM shutdown. This is intentionally static and shared
- * across all StreamingAsyncEntityProducer instances to limit total thread count. The pool is not
- * shut down explicitly - daemon threads will terminate when the JVM exits. This design provides
- * efficient resource sharing while avoiding complex lifecycle management.
+ * Uses daemon threads so it won't prevent JVM shutdown if shutdown is not called.
+ *
+ * Lifecycle management: Call {@link #acquireExecutor()} when creating an async client and
+ * {@link #releaseExecutor()} when closing it. The pool is lazily created on first acquire and
+ * shut down when the last client releases it.
*/
private static final int COMPRESSION_POOL_SIZE = Math.max(2, Runtime.getRuntime().availableProcessors());
- private static final ExecutorService COMPRESSION_EXECUTOR = new ThreadPoolExecutor(
- COMPRESSION_POOL_SIZE,
- COMPRESSION_POOL_SIZE,
- 60L, TimeUnit.SECONDS,
- new LinkedBlockingQueue<>(1000), // Bounded queue to provide backpressure
- new ThreadFactory() {
- @Override
- public Thread newThread(Runnable r) {
- Thread t = new Thread(r, "async-compression-" + THREAD_COUNTER.incrementAndGet());
- t.setDaemon(true);
- return t;
+ private static final Object EXECUTOR_LOCK = new Object();
+ private static final AtomicInteger EXECUTOR_REF_COUNT = new AtomicInteger(0);
+ private static volatile ExecutorService compressionExecutor = null;
+
+ /**
+ * Acquires a reference to the shared compression executor.
+ * Call this when creating an async HTTP client that may use compression.
+ * Must be paired with a call to {@link #releaseExecutor()} when the client is closed.
+ */
+ public static void acquireExecutor() {
+ synchronized (EXECUTOR_LOCK) {
+ if (EXECUTOR_REF_COUNT.getAndIncrement() == 0) {
+ compressionExecutor = createCompressionExecutor();
+ LOG.debug("Created compression executor pool");
+ }
+ }
+ }
+
+ /**
+ * Releases a reference to the shared compression executor.
+ * When the last reference is released, the executor is gracefully shut down.
+ */
+ public static void releaseExecutor() {
+ synchronized (EXECUTOR_LOCK) {
+ if (EXECUTOR_REF_COUNT.decrementAndGet() == 0 && compressionExecutor != null) {
+ LOG.debug("Shutting down compression executor pool");
+ compressionExecutor.shutdown();
+ try {
+ if (!compressionExecutor.awaitTermination(5, TimeUnit.SECONDS)) {
+ compressionExecutor.shutdownNow();
+ }
+ } catch (InterruptedException e) {
+ compressionExecutor.shutdownNow();
+ Thread.currentThread().interrupt();
}
- },
- new ThreadPoolExecutor.CallerRunsPolicy() // If queue full, run in caller thread (backpressure)
- );
+ compressionExecutor = null;
+ }
+ }
+ }
+
+ private static ExecutorService createCompressionExecutor() {
+ return new ThreadPoolExecutor(
+ COMPRESSION_POOL_SIZE,
+ COMPRESSION_POOL_SIZE,
+ 60L, TimeUnit.SECONDS,
+ new LinkedBlockingQueue<>(1000), // Bounded queue to provide backpressure
+ new ThreadFactory() {
+ @Override
+ public Thread newThread(Runnable r) {
+ Thread t = new Thread(r, "ch-async-compress-" + THREAD_COUNTER.incrementAndGet());
+ t.setDaemon(true);
+ return t;
+ }
+ },
+ new ThreadPoolExecutor.CallerRunsPolicy() // If queue full, run in caller thread (backpressure)
+ );
+ }
+
+ private static ExecutorService getExecutor() {
+ ExecutorService exec = compressionExecutor;
+ if (exec == null || exec.isShutdown()) {
+ // Fallback: if executor not acquired properly, create inline (caller's thread)
+ // This handles edge cases but logs a warning
+ LOG.warn("Compression executor not acquired - compression will run in caller thread");
+ return null;
+ }
+ return exec;
+ }
private final ContentType contentType;
private final InputStream sourceStream;
@@ -103,8 +158,9 @@ private void initializeStreams() throws IOException {
compressedInputStream = new PipedInputStream(compressedOutputStream, PIPE_BUFFER_SIZE);
activeInputStream = compressedInputStream;
- // Submit compression task to shared thread pool
- COMPRESSION_EXECUTOR.submit(() -> {
+ // Submit compression task to shared thread pool (or run inline if not available)
+ ExecutorService executor = getExecutor();
+ Runnable compressionTask = () -> {
try {
OutputStream compressingStream;
if (useHttpCompression) {
@@ -134,7 +190,16 @@ private void initializeStreams() throws IOException {
} catch (IOException ignored) {
}
}
- });
+ };
+
+ if (executor != null) {
+ executor.submit(compressionTask);
+ } else {
+ // Fallback: run compression in a new thread (less efficient but functional)
+ Thread t = new Thread(compressionTask, "ch-compress-fallback-" + THREAD_COUNTER.incrementAndGet());
+ t.setDaemon(true);
+ t.start();
+ }
} else {
// No compression - read directly from source
activeInputStream = sourceStream;
From 852cb9d44b9418b6b25790cc1ab4dc5ca9c03e42 Mon Sep 17 00:00:00 2001
From: David Karlsson <2795016+devdavidkarlsson@users.noreply.github.com>
Date: Tue, 17 Feb 2026 15:44:42 +0100
Subject: [PATCH 23/32] Update
client-v2/src/main/java/com/clickhouse/client/api/Client.java
Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
---
.../com/clickhouse/client/api/Client.java | 34 ++++++++++++++-----
1 file changed, 26 insertions(+), 8 deletions(-)
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/Client.java b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
index 986dcd975..a4fb0cc6d 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/Client.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
@@ -1965,24 +1965,42 @@ private CompletableFuture applyAsyncTimeout(CompletableFuture future,
LOG.warn("Timeout scheduler not available - timeout will not be applied");
return future;
}
- CompletableFuture timeoutFuture = new CompletableFuture<>();
+
+ // Wrapper future that enforces timeout and propagates cancellation
+ CompletableFuture resultFuture = new CompletableFuture<>();
+
java.util.concurrent.ScheduledFuture> scheduled = scheduler.schedule(() -> {
- if (!future.isDone()) {
- timeoutFuture.completeExceptionally(
- new TimeoutException("Async operation timed out after " + timeoutMs + "ms"));
+ if (resultFuture.isDone()) {
+ return;
}
+ // Complete the wrapper with a timeout and cancel the underlying operation
+ TimeoutException timeoutException =
+ new TimeoutException("Async operation timed out after " + timeoutMs + "ms");
+ resultFuture.completeExceptionally(timeoutException);
+ future.cancel(true);
}, timeoutMs, TimeUnit.MILLISECONDS);
- future.whenComplete((result, ex) -> {
+ // When the underlying future completes first, propagate its result and cancel the timeout task
+ future.whenComplete((value, ex) -> {
+ if (resultFuture.isDone()) {
+ return;
+ }
scheduled.cancel(false);
if (ex != null) {
- timeoutFuture.completeExceptionally(ex);
+ resultFuture.completeExceptionally(ex);
} else {
- timeoutFuture.complete(result);
+ resultFuture.complete(value);
+ }
+ });
+
+ // If callers cancel the wrapper, propagate cancellation to the underlying future
+ resultFuture.whenComplete((v, ex) -> {
+ if (resultFuture.isCancelled() && !future.isDone()) {
+ future.cancel(true);
}
});
- return timeoutFuture;
+ return resultFuture;
}
public CompletableFuture query(String sqlQuery, Map queryParams) {
From ee1aeb144069085d4e1dcec13d70d633d3e5fe37 Mon Sep 17 00:00:00 2001
From: David Karlsson <2795016+devdavidkarlsson@users.noreply.github.com>
Date: Tue, 17 Feb 2026 15:45:01 +0100
Subject: [PATCH 24/32] Update
client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
---
.../test/java/com/clickhouse/client/AsyncHttpClientTests.java | 1 +
1 file changed, 1 insertion(+)
diff --git a/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java b/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
index 42e73c2f0..1fef5e170 100644
--- a/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
+++ b/client-v2/src/test/java/com/clickhouse/client/AsyncHttpClientTests.java
@@ -382,6 +382,7 @@ public void testAsyncQueryCancellation() {
boolean cancelled = future.cancel(true);
// The future should be cancelled
+ Assert.assertTrue(cancelled, "Cancellation should return true");
Assert.assertTrue(future.isCancelled() || future.isDone());
}
From ba7b17dbe02b905a846bb8d6a8d485b106d43cfb Mon Sep 17 00:00:00 2001
From: David Karlsson <2795016+devdavidkarlsson@users.noreply.github.com>
Date: Tue, 17 Feb 2026 15:45:57 +0100
Subject: [PATCH 25/32] Update client-v2/src/test/resources/logback-test.xml
Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
---
client-v2/src/test/resources/logback-test.xml | 6 +++---
1 file changed, 3 insertions(+), 3 deletions(-)
diff --git a/client-v2/src/test/resources/logback-test.xml b/client-v2/src/test/resources/logback-test.xml
index 0d5c146a2..9c722f002 100644
--- a/client-v2/src/test/resources/logback-test.xml
+++ b/client-v2/src/test/resources/logback-test.xml
@@ -6,9 +6,9 @@
-
-
-
+
+
+
From a87204ee09d342f6895b39a4e11133ff440c9254 Mon Sep 17 00:00:00 2001
From: David Karlsson <2795016+devdavidkarlsson@users.noreply.github.com>
Date: Tue, 17 Feb 2026 15:46:38 +0100
Subject: [PATCH 26/32] Update
client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
---
.../clickhouse/client/api/internal/HttpAPIClientHelper.java | 5 ++++-
1 file changed, 4 insertions(+), 1 deletion(-)
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
index 9757dae0b..6cab1524e 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java
@@ -784,7 +784,10 @@ private Exception readErrorFromStreamingResponse(StreamingAsyncResponseConsumer.
int errorCode = getHeaderVal(response.getFirstHeader(ClickHouseHttpProto.HEADER_EXCEPTION_CODE),
0, Integer::parseInt);
- return new ServerException(errorCode, errorBody, response.getCode(), null);
+ Header queryIdHeader = response.getFirstHeader(ClickHouseHttpProto.HEADER_QUERY_ID);
+ String queryId = queryIdHeader != null ? queryIdHeader.getValue() : null;
+
+ return new ServerException(errorCode, errorBody, response.getCode(), queryId);
} catch (Exception e) {
return new ClientException("Failed to read error response", e);
} finally {
From aa30e362ca6a9eda1f7ea79a7b6e6beefc4fb128 Mon Sep 17 00:00:00 2001
From: David Karlsson <2795016+devdavidkarlsson@users.noreply.github.com>
Date: Tue, 17 Feb 2026 15:47:36 +0100
Subject: [PATCH 27/32] Update
client-v2/src/main/java/com/clickhouse/client/api/Client.java
Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
---
.../com/clickhouse/client/api/Client.java | 33 ++++++++++++++-----
1 file changed, 24 insertions(+), 9 deletions(-)
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/Client.java b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
index a4fb0cc6d..631768dd8 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/Client.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
@@ -1901,16 +1901,31 @@ private CompletableFuture executeQueryAsync(String sqlQuery,
throw new java.util.concurrent.CompletionException(wrappedException);
}
- if (response.getCode() == HttpStatus.SC_SERVICE_UNAVAILABLE && attempt < retries) {
- LOG.warn("Failed to get response. Server returned {}. Retrying. (Duration: {})",
- response.getCode(), durationSince(startTime));
- // Close the streaming response before retrying to avoid resource leaks
- try {
- response.close();
- } catch (Exception closeEx) {
- LOG.debug("Failed to close streaming response before retry", closeEx);
+ if (response.getCode() == HttpStatus.SC_SERVICE_UNAVAILABLE) {
+ if (attempt < retries) {
+ LOG.warn("Failed to get response. Server returned {}. Retrying. (Duration: {})",
+ response.getCode(), durationSince(startTime));
+ // Close the streaming response before retrying to avoid resource leaks
+ try {
+ response.close();
+ } catch (Exception closeEx) {
+ LOG.debug("Failed to close streaming response before retry", closeEx);
+ }
+ return new AsyncRetryMarker(attempt + 1);
+ } else {
+ String msg = requestExMsg("Query", (attempt + 1),
+ durationSince(startTime).toMillis(), requestSettings.getQueryId());
+ IOException cause = new IOException("Failed to get response. Server returned HTTP 503 (Service Unavailable).");
+ RuntimeException wrappedException = httpClientHelper.wrapException(
+ msg, cause, requestSettings.getQueryId());
+ // Close the streaming response before completing exceptionally
+ try {
+ response.close();
+ } catch (Exception closeEx) {
+ LOG.debug("Failed to close streaming response after retries exhausted", closeEx);
+ }
+ throw new java.util.concurrent.CompletionException(wrappedException);
}
- return new AsyncRetryMarker(attempt + 1);
}
OperationMetrics metrics = new OperationMetrics(clientStats);
From 3cfc0836912a9e601a7c0ad0fcd2425d537ad940 Mon Sep 17 00:00:00 2001
From: David Karlsson <2795016+devdavidkarlsson@users.noreply.github.com>
Date: Tue, 17 Feb 2026 15:48:51 +0100
Subject: [PATCH 28/32] fix: properly handle 503 responses in async
query/insert paths
- executeQueryAsync: throw ServerException when 503 received after all retries exhausted
- executeInsertAsync: throw ServerException on 503 (async inserts don't support retry)
- Both paths now close the response before throwing to prevent resource leaks
Previously, final 503 responses were incorrectly treated as successful operations
and converted to QueryResponse/InsertResponse objects.
Co-Authored-By: Claude Opus 4.5
---
.../main/java/com/clickhouse/client/api/Client.java | 13 +++++++++++++
1 file changed, 13 insertions(+)
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/Client.java b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
index 631768dd8..04241b24a 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/Client.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
@@ -1589,6 +1589,19 @@ private CompletableFuture executeInsertAsync(String tableName,
return httpClientHelper.executeInsertAsyncStreaming(selectedEndpoint, requestSettings.getAllSettings(), data)
.thenApply(response -> {
+ // Check for 503 Service Unavailable - async inserts don't support retry
+ if (response.getCode() == HttpStatus.SC_SERVICE_UNAVAILABLE) {
+ try {
+ response.close();
+ } catch (IOException closeEx) {
+ LOG.debug("Failed to close 503 response", closeEx);
+ }
+ throw new java.util.concurrent.CompletionException(
+ new ServerException(ServerException.CODE_UNKNOWN,
+ "Service Unavailable - async inserts do not support automatic retry",
+ HttpStatus.SC_SERVICE_UNAVAILABLE, requestSettings.getQueryId()));
+ }
+
OperationMetrics metrics = new OperationMetrics(finalClientStats);
String summary = HttpAPIClientHelper.getHeaderVal(
response.getFirstHeader(ClickHouseHttpProto.HEADER_SRV_SUMMARY), "{}");
From e57e89cb2eb7ef38f89b7cd3c7cc5f96068f59e2 Mon Sep 17 00:00:00 2001
From: David Karlsson <2795016+devdavidkarlsson@users.noreply.github.com>
Date: Tue, 17 Feb 2026 16:37:01 +0100
Subject: [PATCH 29/32] fix: update expected config count for new
USE_ASYNC_HTTP property
The USE_ASYNC_HTTP config property was added for async HTTP support,
increasing the total config count from 34 to 35.
Co-Authored-By: Claude Opus 4.5
---
.../src/test/java/com/clickhouse/client/ClientTests.java | 4 ++--
1 file changed, 2 insertions(+), 2 deletions(-)
diff --git a/client-v2/src/test/java/com/clickhouse/client/ClientTests.java b/client-v2/src/test/java/com/clickhouse/client/ClientTests.java
index 52815237b..8a2e5e010 100644
--- a/client-v2/src/test/java/com/clickhouse/client/ClientTests.java
+++ b/client-v2/src/test/java/com/clickhouse/client/ClientTests.java
@@ -260,7 +260,7 @@ public void testDefaultSettings() {
Assert.assertEquals(config.get(p.getKey()), p.getDefaultValue(), "Default value doesn't match");
}
}
- Assert.assertEquals(config.size(), 34); // to check everything is set. Increment when new added.
+ Assert.assertEquals(config.size(), 35); // to check everything is set. Increment when new added.
}
try (Client client = new Client.Builder()
@@ -360,7 +360,7 @@ public void testWithOldDefaults() {
Assert.assertEquals(config.get(p.getKey()), p.getDefaultValue(), "Default value doesn't match");
}
}
- Assert.assertEquals(config.size(), 34); // to check everything is set. Increment when new added.
+ Assert.assertEquals(config.size(), 35); // to check everything is set. Increment when new added.
}
}
From d02855caf4e46e9590bfabc6bc4d29bf78dd890f Mon Sep 17 00:00:00 2001
From: David Karlsson <2795016+devdavidkarlsson@users.noreply.github.com>
Date: Tue, 17 Feb 2026 16:44:32 +0100
Subject: [PATCH 30/32] fix: address Copilot review on executor lifecycle and
error handling
1. Add thread-safety documentation to acquire/release methods explaining
that the synchronized block prevents the race condition concern
(mutual exclusion ensures no concurrent acquire/release)
2. Fix potential resource leak in Client constructor - if acquiring the
second executor fails, release the first one
3. Fix streamCompleteFuture not completing if close() throws - moved
future completion outside try block to ensure it always happens
4. Change unsubstantiated "96-99%" performance claim to "Substantial
reduction" per Copilot suggestion
Co-Authored-By: Claude Opus 4.5
---
README.md | 2 +-
.../com/clickhouse/client/api/Client.java | 24 +++++++++++++++++--
.../StreamingAsyncEntityProducer.java | 8 +++++++
.../StreamingAsyncResponseConsumer.java | 20 ++++++++++------
4 files changed, 44 insertions(+), 10 deletions(-)
diff --git a/README.md b/README.md
index 4bfbe2b01..cf3fe6d05 100644
--- a/README.md
+++ b/README.md
@@ -100,7 +100,7 @@ Client V2 supports true async HTTP using Apache HttpClient 5 NIO API for high-co
- Non-blocking I/O - no thread-per-request blocking
- Streaming responses with constant memory usage
- Streaming request compression (HTTP and native LZ4)
-- 96-99% thread reduction under high concurrency
+- Substantial reduction in thread usage under high concurrency
**Usage:**
```java
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/Client.java b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
index 04241b24a..fe117a719 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/Client.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
@@ -126,6 +126,11 @@ public class Client implements AutoCloseable {
new java.util.concurrent.atomic.AtomicInteger(0);
private static volatile java.util.concurrent.ScheduledExecutorService timeoutScheduler = null;
+ /**
+ * Thread-safety: Synchronizes on TIMEOUT_SCHEDULER_LOCK, ensuring mutual exclusion
+ * with releaseTimeoutScheduler(). No race condition exists because threads cannot
+ * concurrently execute acquire and release.
+ */
private static void acquireTimeoutScheduler() {
synchronized (TIMEOUT_SCHEDULER_LOCK) {
if (TIMEOUT_SCHEDULER_REF_COUNT.getAndIncrement() == 0) {
@@ -139,6 +144,11 @@ private static void acquireTimeoutScheduler() {
}
}
+ /**
+ * Thread-safety: Synchronizes on TIMEOUT_SCHEDULER_LOCK, ensuring mutual exclusion
+ * with acquireTimeoutScheduler(). The synchronized block guarantees that between
+ * checking the ref count and shutting down, no other thread can acquire a new reference.
+ */
private static void releaseTimeoutScheduler() {
synchronized (TIMEOUT_SCHEDULER_LOCK) {
if (TIMEOUT_SCHEDULER_REF_COUNT.decrementAndGet() == 0 && timeoutScheduler != null) {
@@ -246,8 +256,18 @@ private Client(Collection endpoints, Map configuration,
// Acquire shared async resources if async HTTP is enabled
this.usesAsyncHttp = httpClientHelper.isAsyncEnabled();
if (this.usesAsyncHttp) {
- acquireTimeoutScheduler();
- StreamingAsyncEntityProducer.acquireExecutor();
+ boolean timeoutSchedulerAcquired = false;
+ try {
+ acquireTimeoutScheduler();
+ timeoutSchedulerAcquired = true;
+ StreamingAsyncEntityProducer.acquireExecutor();
+ } catch (Exception e) {
+ // Release any acquired resources on failure to prevent leaks
+ if (timeoutSchedulerAcquired) {
+ releaseTimeoutScheduler();
+ }
+ throw e;
+ }
}
}
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncEntityProducer.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncEntityProducer.java
index b348c39c5..99cd85915 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncEntityProducer.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncEntityProducer.java
@@ -54,6 +54,10 @@ public class StreamingAsyncEntityProducer implements AsyncEntityProducer {
* Acquires a reference to the shared compression executor.
* Call this when creating an async HTTP client that may use compression.
* Must be paired with a call to {@link #releaseExecutor()} when the client is closed.
+ *
+ * Thread-safety: This method synchronizes on EXECUTOR_LOCK, ensuring mutual exclusion
+ * with releaseExecutor(). No race condition exists because a thread cannot enter
+ * releaseExecutor() while another is in acquireExecutor() (and vice versa).
*/
public static void acquireExecutor() {
synchronized (EXECUTOR_LOCK) {
@@ -67,6 +71,10 @@ public static void acquireExecutor() {
/**
* Releases a reference to the shared compression executor.
* When the last reference is released, the executor is gracefully shut down.
+ *
+ * Thread-safety: This method synchronizes on EXECUTOR_LOCK, ensuring mutual exclusion
+ * with acquireExecutor(). The synchronized block guarantees that between checking the
+ * ref count and shutting down, no other thread can acquire a new reference.
*/
public static void releaseExecutor() {
synchronized (EXECUTOR_LOCK) {
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncResponseConsumer.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncResponseConsumer.java
index b814c8268..c43bd2207 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncResponseConsumer.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/StreamingAsyncResponseConsumer.java
@@ -145,17 +145,23 @@ public void failed(Exception cause) {
private void closeOutputStream() {
if (outputClosed.compareAndSet(false, true)) {
LOG.debug("closeOutputStream() called, total bytes written: {}", totalBytesWritten);
+ Exception closeException = null;
try {
pipedOutputStream.close();
- if (streamError != null) {
- streamCompleteFuture.completeExceptionally(streamError);
- LOG.debug("closeOutputStream() completed with error");
- } else {
- streamCompleteFuture.complete(null);
- LOG.debug("closeOutputStream() completed successfully");
- }
} catch (IOException e) {
LOG.debug("Error closing piped output stream: {}", e.getMessage());
+ closeException = e;
+ }
+ // Always complete the future, even if close() threw an exception
+ if (streamError != null) {
+ streamCompleteFuture.completeExceptionally(streamError);
+ LOG.debug("closeOutputStream() completed with stream error");
+ } else if (closeException != null) {
+ streamCompleteFuture.completeExceptionally(closeException);
+ LOG.debug("closeOutputStream() completed with close error");
+ } else {
+ streamCompleteFuture.complete(null);
+ LOG.debug("closeOutputStream() completed successfully");
}
} else {
LOG.debug("closeOutputStream() already closed, skipping");
From f89db10f3d83c17fddea1cdd2e07a6ba0aebba1b Mon Sep 17 00:00:00 2001
From: David Karlsson <2795016+devdavidkarlsson@users.noreply.github.com>
Date: Wed, 18 Feb 2026 14:31:49 +0100
Subject: [PATCH 31/32] Update
client-v2/src/main/java/com/clickhouse/client/api/Client.java
Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
---
.../src/main/java/com/clickhouse/client/api/Client.java | 6 ++++++
1 file changed, 6 insertions(+)
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/Client.java b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
index fe117a719..f83100cbc 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/Client.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
@@ -266,6 +266,12 @@ private Client(Collection endpoints, Map configuration,
if (timeoutSchedulerAcquired) {
releaseTimeoutScheduler();
}
+ // Ensure HTTP client helper is also closed on initialization failure
+ try {
+ httpClientHelper.close();
+ } catch (Exception closeEx) {
+ // Ignore to avoid masking the original initialization failure
+ }
throw e;
}
}
From 942e8b7f780f90b4ff37eb760a8e2d671677029b Mon Sep 17 00:00:00 2001
From: David Karlsson <2795016+devdavidkarlsson@users.noreply.github.com>
Date: Wed, 18 Feb 2026 14:32:07 +0100
Subject: [PATCH 32/32] Update
client-v2/src/main/java/com/clickhouse/client/api/Client.java
Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
---
.../com/clickhouse/client/api/Client.java | 47 +++++++++----------
1 file changed, 22 insertions(+), 25 deletions(-)
diff --git a/client-v2/src/main/java/com/clickhouse/client/api/Client.java b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
index f83100cbc..ac608892f 100644
--- a/client-v2/src/main/java/com/clickhouse/client/api/Client.java
+++ b/client-v2/src/main/java/com/clickhouse/client/api/Client.java
@@ -1615,35 +1615,32 @@ private CompletableFuture executeInsertAsync(String tableName,
return httpClientHelper.executeInsertAsyncStreaming(selectedEndpoint, requestSettings.getAllSettings(), data)
.thenApply(response -> {
- // Check for 503 Service Unavailable - async inserts don't support retry
- if (response.getCode() == HttpStatus.SC_SERVICE_UNAVAILABLE) {
- try {
- response.close();
- } catch (IOException closeEx) {
- LOG.debug("Failed to close 503 response", closeEx);
+ try {
+ // Check for 503 Service Unavailable - async inserts don't support retry
+ if (response.getCode() == HttpStatus.SC_SERVICE_UNAVAILABLE) {
+ throw new java.util.concurrent.CompletionException(
+ new ServerException(ServerException.CODE_UNKNOWN,
+ "Service Unavailable - async inserts do not support automatic retry",
+ HttpStatus.SC_SERVICE_UNAVAILABLE, requestSettings.getQueryId()));
}
- throw new java.util.concurrent.CompletionException(
- new ServerException(ServerException.CODE_UNKNOWN,
- "Service Unavailable - async inserts do not support automatic retry",
- HttpStatus.SC_SERVICE_UNAVAILABLE, requestSettings.getQueryId()));
- }
- OperationMetrics metrics = new OperationMetrics(finalClientStats);
- String summary = HttpAPIClientHelper.getHeaderVal(
- response.getFirstHeader(ClickHouseHttpProto.HEADER_SRV_SUMMARY), "{}");
- ProcessParser.parseSummary(summary, metrics);
- String queryId = HttpAPIClientHelper.getHeaderVal(
- response.getFirstHeader(ClickHouseHttpProto.HEADER_QUERY_ID), requestSettings.getQueryId());
- metrics.setQueryId(queryId);
- metrics.operationComplete();
+ OperationMetrics metrics = new OperationMetrics(finalClientStats);
+ String summary = HttpAPIClientHelper.getHeaderVal(
+ response.getFirstHeader(ClickHouseHttpProto.HEADER_SRV_SUMMARY), "{}");
+ ProcessParser.parseSummary(summary, metrics);
+ String queryId = HttpAPIClientHelper.getHeaderVal(
+ response.getFirstHeader(ClickHouseHttpProto.HEADER_QUERY_ID), requestSettings.getQueryId());
+ metrics.setQueryId(queryId);
+ metrics.operationComplete();
- try {
- response.close();
- } catch (IOException e) {
- LOG.debug("Error closing insert response", e);
+ return new InsertResponse(metrics);
+ } finally {
+ try {
+ response.close();
+ } catch (IOException e) {
+ LOG.debug("Error closing insert response", e);
+ }
}
-
- return new InsertResponse(metrics);
});
}