Skip to content

Commit 190983e

Browse files
committed
Ensure proper closure of httpResponse in error scenarios during retries.
1 parent d4792b7 commit 190983e

File tree

1 file changed

+14
-5
lines changed
  • client-v2/src/main/java/com/clickhouse/client/api

1 file changed

+14
-5
lines changed

client-v2/src/main/java/com/clickhouse/client/api/Client.java

Lines changed: 14 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -55,6 +55,7 @@
5555
import java.io.InputStream;
5656
import java.io.OutputStream;
5757
import java.lang.reflect.InvocationTargetException;
58+
import java.net.MalformedURLException;
5859
import java.net.URL;
5960
import java.nio.charset.StandardCharsets;
6061
import java.time.Duration;
@@ -121,7 +122,7 @@ public class Client implements AutoCloseable {
121122
private final Map<String, Object> configuration;
122123

123124
private final Map<String, String> readOnlyConfig;
124-
125+
125126
private final POJOSerDe pojoSerDe;
126127

127128
private final ExecutorService sharedOperationExecutor;
@@ -291,7 +292,7 @@ public Builder() {
291292
*/
292293
public Builder addEndpoint(String endpoint) {
293294
try {
294-
URL endpointURL = new java.net.URL(endpoint);
295+
URL endpointURL = new URL(endpoint);
295296

296297
if (endpointURL.getProtocol().equalsIgnoreCase("https")) {
297298
addEndpoint(Protocol.HTTP, endpointURL.getHost(), endpointURL.getPort(), true);
@@ -300,7 +301,7 @@ public Builder addEndpoint(String endpoint) {
300301
} else {
301302
throw new IllegalArgumentException("Only HTTP and HTTPS protocols are supported");
302303
}
303-
} catch (java.net.MalformedURLException e) {
304+
} catch (MalformedURLException e) {
304305
throw new IllegalArgumentException("Endpoint should be a valid URL string, but was " + endpoint, e);
305306
}
306307
return this;
@@ -380,7 +381,7 @@ public Builder setAccessToken(String accessToken) {
380381

381382
/**
382383
* Makes client to use SSL Client Certificate to authenticate with server.
383-
* Client certificate should be set as well. {@link Client.Builder#setClientCertificate(String)}
384+
* Client certificate should be set as well. {@link Builder#setClientCertificate(String)}
384385
* @param useSSLAuthentication
385386
* @return
386387
*/
@@ -1583,8 +1584,9 @@ public CompletableFuture<QueryResponse> query(String sqlQuery, Map<String, Objec
15831584
Endpoint selectedEndpoint = getNextAliveNode();
15841585
RuntimeException lastException = null;
15851586
for (int i = 0; i <= retries; i++) {
1587+
ClassicHttpResponse httpResponse = null;
15861588
try {
1587-
ClassicHttpResponse httpResponse =
1589+
httpResponse =
15881590
httpClientHelper.executeRequest(selectedEndpoint, finalSettings.getAllSettings(), lz4Factory, output -> {
15891591
output.write(sqlQuery.getBytes(StandardCharsets.UTF_8));
15901592
output.close();
@@ -1614,6 +1616,13 @@ public CompletableFuture<QueryResponse> query(String sqlQuery, Map<String, Objec
16141616
return new QueryResponse(httpResponse, responseFormat, finalSettings, metrics);
16151617

16161618
} catch (Exception e) {
1619+
if (httpResponse != null) {
1620+
try {
1621+
httpResponse.close();
1622+
} catch (IOException ex) {
1623+
throw new ClientException("Failed to close response", e);
1624+
}
1625+
}
16171626
lastException = httpClientHelper.wrapException(String.format("Query request failed (Attempt: %s/%s - Duration: %s)",
16181627
(i + 1), (retries + 1), System.nanoTime() - startTime), e);
16191628
if (httpClientHelper.shouldRetry(e, finalSettings.getAllSettings())) {

0 commit comments

Comments
 (0)