Skip to content

Commit db4e219

Browse files
committed
Call HttpURLConnection.disconnect when closing response, and avoid double buffering
1 parent 105de13 commit db4e219

File tree

5 files changed

+32
-23
lines changed

5 files changed

+32
-23
lines changed

clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseLZ4InputStream.java

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -90,7 +90,11 @@ private boolean readFully(byte[] b, int off, int len) throws IOException {
9090
}
9191

9292
public ClickHouseLZ4InputStream(InputStream stream) {
93-
super(null);
93+
this(stream, null);
94+
}
95+
96+
public ClickHouseLZ4InputStream(InputStream stream, Runnable afterClose) {
97+
super(afterClose);
9498

9599
this.decompressor = factory.fastDecompressor();
96100
this.stream = ClickHouseChecker.nonNull(stream, "InputStream");

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

Lines changed: 20 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@
2020
import com.clickhouse.client.ClickHouseCompression;
2121
import com.clickhouse.client.ClickHouseConfig;
2222
import com.clickhouse.client.ClickHouseCredentials;
23+
import com.clickhouse.client.ClickHouseInputStream;
2324
import com.clickhouse.client.ClickHouseNode;
2425
import com.clickhouse.client.ClickHouseRequest;
2526
import com.clickhouse.client.ClickHouseUtils;
@@ -203,6 +204,14 @@ protected ClickHouseHttpConnection(ClickHouseNode server, ClickHouseRequest<?> r
203204
this.defaultHeaders = Collections.unmodifiableMap(map);
204205
}
205206

207+
protected void closeQuietly() {
208+
try {
209+
close();
210+
} catch (Exception e) {
211+
// ignore
212+
}
213+
}
214+
206215
protected String getBaseUrl() {
207216
String baseUrl;
208217
int index = url.indexOf('?');
@@ -236,23 +245,30 @@ protected OutputStream getRequestOutputStream(OutputStream out) throws IOExcepti
236245
return out;
237246
}
238247

239-
protected InputStream getResponseInputStream(InputStream in) throws IOException {
248+
protected ClickHouseInputStream getResponseInputStream(InputStream in) throws IOException {
249+
Runnable afterClose = null;
250+
if (!isReusable()) {
251+
afterClose = this::closeQuietly;
252+
}
253+
ClickHouseInputStream chInput;
240254
if (config.isCompressServerResponse()) {
241255
// TODO support more algorithms
242256
ClickHouseCompression algorithm = config.getCompressAlgorithmForServerResponse();
243257
switch (algorithm) {
244258
case GZIP:
245-
in = new GZIPInputStream(in);
259+
chInput = ClickHouseInputStream.of(new GZIPInputStream(in), config.getMaxBufferSize(), afterClose);
246260
break;
247261
case LZ4:
248-
in = new ClickHouseLZ4InputStream(in);
262+
chInput = new ClickHouseLZ4InputStream(in, afterClose);
249263
break;
250264
default:
251265
throw new UnsupportedOperationException("Unsupported compression algorithm: " + algorithm);
252266
}
267+
} else {
268+
chInput = ClickHouseInputStream.of(in, config.getMaxBufferSize(), afterClose);
253269
}
254270

255-
return in;
271+
return chInput;
256272
}
257273

258274
/**

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

Lines changed: 3 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -1,6 +1,5 @@
11
package com.clickhouse.client.http;
22

3-
import java.io.InputStream;
43
import java.io.Serializable;
54
import java.util.HashMap;
65
import java.util.Map;
@@ -39,16 +38,6 @@ private static long getLongValue(Map<String, String> map, String key) {
3938

4039
protected final ClickHouseResponseSummary summary;
4140

42-
protected void closeConnection() {
43-
if (!connection.isReusable()) {
44-
try {
45-
connection.close();
46-
} catch (Exception e) {
47-
// ignore
48-
}
49-
}
50-
}
51-
5241
protected ClickHouseConfig getConfig(ClickHouseRequest<?> request) {
5342
ClickHouseConfig config = request.getConfig();
5443
if (format != null && format != config.getFormat()) {
@@ -61,14 +50,14 @@ protected ClickHouseConfig getConfig(ClickHouseRequest<?> request) {
6150
return config;
6251
}
6352

64-
public ClickHouseHttpResponse(ClickHouseHttpConnection connection, InputStream input, String serverDisplayName,
65-
String queryId, String summary, ClickHouseFormat format, TimeZone timeZone) {
53+
public ClickHouseHttpResponse(ClickHouseHttpConnection connection, ClickHouseInputStream input,
54+
String serverDisplayName, String queryId, String summary, ClickHouseFormat format, TimeZone timeZone) {
6655
if (connection == null || input == null) {
6756
throw new IllegalArgumentException("Non-null connection and input stream are required");
6857
}
6958

7059
this.connection = connection;
71-
this.input = ClickHouseInputStream.of(input, connection.config.getMaxBufferSize(), this::closeConnection);
60+
this.input = input;
7261

7362
this.serverDisplayName = !ClickHouseChecker.isNullOrEmpty(serverDisplayName) ? serverDisplayName
7463
: connection.server.getHost();

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -117,8 +117,8 @@ private void checkResponse(HttpURLConnection conn) throws IOException {
117117
// TODO get exception from response header, for example:
118118
// X-ClickHouse-Exception-Code: 47
119119
StringBuilder builder = new StringBuilder();
120-
try (Reader reader = new BufferedReader(
121-
new InputStreamReader(getResponseInputStream(conn.getErrorStream()), StandardCharsets.UTF_8))) {
120+
try (Reader reader = new InputStreamReader(getResponseInputStream(conn.getErrorStream()),
121+
StandardCharsets.UTF_8)) {
122122
int c = 0;
123123
while ((c = reader.read()) != -1) {
124124
builder.append((char) c);

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -76,8 +76,7 @@ private HttpResponse<InputStream> checkResponse(HttpResponse<InputStream> r) thr
7676
// TODO get exception from response header, for example:
7777
// X-ClickHouse-Exception-Code: 47
7878
StringBuilder builder = new StringBuilder();
79-
try (Reader reader = new BufferedReader(
80-
new InputStreamReader(getResponseInputStream(r.body()), StandardCharsets.UTF_8))) {
79+
try (Reader reader = new InputStreamReader(getResponseInputStream(r.body()), StandardCharsets.UTF_8)) {
8180
int c = 0;
8281
while ((c = reader.read()) != -1) {
8382
builder.append((char) c);
@@ -278,5 +277,6 @@ public boolean ping(int timeout) {
278277

279278
@Override
280279
public void close() {
280+
// nothing
281281
}
282282
}

0 commit comments

Comments
 (0)