Skip to content

Commit cdf11ce

Browse files
committed
fixed handling error when successful http status by error code header present
1 parent 38a5ef5 commit cdf11ce

File tree

4 files changed

+55
-4
lines changed

4 files changed

+55
-4
lines changed

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -194,11 +194,11 @@ private void setHeaders(HttpRequest request, Map<String, String> headers) {
194194
}
195195

196196
private void checkResponse(ClickHouseConfig config, CloseableHttpResponse response) throws IOException {
197-
if (response.getCode() == HttpURLConnection.HTTP_OK) {
197+
final Header errorCode = response.getFirstHeader(ClickHouseHttpProto.HEADER_EXCEPTION_CODE);
198+
if (response.getCode() == HttpURLConnection.HTTP_OK && errorCode == null) {
198199
return;
199200
}
200201

201-
final Header errorCode = response.getFirstHeader(ClickHouseHttpProto.HEADER_EXCEPTION_CODE);
202202
final Header serverName = response.getFirstHeader(ClickHouseHttpProto.HEADER_SRV_DISPLAY_NAME);
203203
if (response.getEntity() == null) {
204204
throw new ConnectException(

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

Lines changed: 13 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1,6 +1,7 @@
11
package com.clickhouse.client.api;
22

33
import com.clickhouse.client.ClickHouseClient;
4+
import com.clickhouse.client.ClickHouseException;
45
import com.clickhouse.client.ClickHouseNode;
56
import com.clickhouse.client.ClickHouseRequest;
67
import com.clickhouse.client.ClickHouseResponse;
@@ -80,6 +81,7 @@
8081
import java.util.TimeZone;
8182
import java.util.UUID;
8283
import java.util.concurrent.CompletableFuture;
84+
import java.util.concurrent.CompletionException;
8385
import java.util.concurrent.ConcurrentHashMap;
8486
import java.util.concurrent.ExecutionException;
8587
import java.util.concurrent.ExecutorService;
@@ -1455,6 +1457,11 @@ public CompletableFuture<InsertResponse> insert(String tableName,
14551457
return response;
14561458
} catch (ExecutionException e) {
14571459
throw new ClientException("Failed to get insert response", e.getCause());
1460+
} catch (CompletionException e) {
1461+
if (e.getCause() instanceof ClickHouseException) {
1462+
throw new ServerException(((ClickHouseException)e.getCause()).getErrorCode(), e.getCause().getMessage().trim());
1463+
}
1464+
throw new ClientException("Failed to get query response", e.getCause());
14581465
} catch (InterruptedException | TimeoutException e) {
14591466
throw new ClientException("Operation has likely timed out.", e);
14601467
}
@@ -1575,7 +1582,7 @@ public CompletableFuture<QueryResponse> query(String sqlQuery, Map<String, Objec
15751582
} else {
15761583
throw lastException;
15771584
}
1578-
} catch (ClientException e) {
1585+
} catch (ClientException | ServerException e) {
15791586
throw e;
15801587
} catch (Exception e) {
15811588
throw new ClientException("Query request failed", e);
@@ -1609,6 +1616,11 @@ public CompletableFuture<QueryResponse> query(String sqlQuery, Map<String, Objec
16091616
return new QueryResponse(clickHouseResponse, format, clientStats, finalSettings);
16101617
} catch (ClientException e) {
16111618
throw e;
1619+
} catch (CompletionException e) {
1620+
if (e.getCause() instanceof ClickHouseException) {
1621+
throw new ServerException(((ClickHouseException)e.getCause()).getErrorCode(), e.getCause().getMessage().trim());
1622+
}
1623+
throw new ClientException("Failed to get query response", e.getCause());
16121624
} catch (Exception e) {
16131625
throw new ClientException("Failed to get query response", e);
16141626
}

client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -348,13 +348,14 @@ public ClassicHttpResponse executeRequest(ClickHouseNode server, Map<String, Obj
348348

349349
try {
350350
ClassicHttpResponse httpResponse = httpClient.executeOpen(null, req, context);
351+
httpResponse.getEntity().getTrailers().get();
351352
httpResponse.setEntity(wrapEntity(httpResponse.getEntity(), true));
352353
if (httpResponse.getCode() == HttpStatus.SC_PROXY_AUTHENTICATION_REQUIRED) {
353354
throw new ClientMisconfigurationException("Proxy authentication required. Please check your proxy settings.");
354355
} else if (httpResponse.getCode() == HttpStatus.SC_BAD_GATEWAY) {
355356
httpResponse.close();
356357
throw new ClientException("Server returned '502 Bad gateway'. Check network and proxy settings.");
357-
} else if (httpResponse.getCode() >= HttpStatus.SC_BAD_REQUEST) {
358+
} else if (httpResponse.getCode() >= HttpStatus.SC_BAD_REQUEST || httpResponse.containsHeader(ClickHouseHttpProto.HEADER_EXCEPTION_CODE)) {
358359
try {
359360
throw readError(httpResponse);
360361
} finally {

client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java

Lines changed: 38 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@
2525
import org.apache.hc.core5.http.ConnectionRequestTimeoutException;
2626
import org.apache.hc.core5.http.HttpStatus;
2727
import org.apache.hc.core5.net.URIBuilder;
28+
import org.eclipse.jetty.server.Server;
2829
import org.testcontainers.utility.ThrowingFunction;
2930
import org.testng.Assert;
3031
import org.testng.annotations.DataProvider;
@@ -338,6 +339,43 @@ public static Object[] testServerErrorHandlingDataProvider() {
338339
return new Object[] { ClickHouseFormat.JSON, ClickHouseFormat.TabSeparated, ClickHouseFormat.RowBinary };
339340
}
340341

342+
343+
@Test(groups = { "integration" })
344+
public void testErrorWithSuccessfulResponse() {
345+
WireMockServer mockServer = new WireMockServer( WireMockConfiguration
346+
.options().port(9090).notifier(new ConsoleNotifier(false)));
347+
mockServer.start();
348+
349+
try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false)
350+
.setUsername("default")
351+
.setPassword("")
352+
.compressServerResponse(false)
353+
.useNewImplementation(true)
354+
.build()) {
355+
mockServer.addStubMapping(WireMock.post(WireMock.anyUrl())
356+
.willReturn(WireMock.aResponse()
357+
.withStatus(HttpStatus.SC_OK)
358+
.withChunkedDribbleDelay(2, 200)
359+
.withHeader("X-ClickHouse-Exception-Code", "241")
360+
.withHeader("X-ClickHouse-Summary",
361+
"{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")
362+
.withBody("Code: 241. DB::Exception: Memory limit (for query) exceeded: would use 97.21 MiB"))
363+
.build());
364+
365+
try (QueryResponse response = client.query("SELECT 1").get(1, TimeUnit.SECONDS)) {
366+
Assert.fail("Expected exception");
367+
} catch (ServerException e) {
368+
e.printStackTrace();
369+
Assert.assertEquals(e.getMessage(), "Code: 241. DB::Exception: Memory limit (for query) exceeded: would use 97.21 MiB");
370+
} catch (Exception e) {
371+
e.printStackTrace();
372+
Assert.fail("Unexpected exception", e);
373+
}
374+
} finally {
375+
mockServer.stop();
376+
}
377+
}
378+
341379
@Test(groups = { "integration" })
342380
public void testAdditionalHeaders() {
343381
WireMockServer mockServer = new WireMockServer( WireMockConfiguration

0 commit comments

Comments
 (0)