Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[client-v2] Fixes handling error when successful http status by error code header in response #1879

Merged
merged 3 commits into from
Oct 24, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -194,11 +194,11 @@ private void setHeaders(HttpRequest request, Map<String, String> headers) {
}

private void checkResponse(ClickHouseConfig config, CloseableHttpResponse response) throws IOException {
if (response.getCode() == HttpURLConnection.HTTP_OK) {
final Header errorCode = response.getFirstHeader(ClickHouseHttpProto.HEADER_EXCEPTION_CODE);
if (response.getCode() == HttpURLConnection.HTTP_OK && errorCode == null) {
return;
}

final Header errorCode = response.getFirstHeader(ClickHouseHttpProto.HEADER_EXCEPTION_CODE);
final Header serverName = response.getFirstHeader(ClickHouseHttpProto.HEADER_SRV_DISPLAY_NAME);
if (response.getEntity() == null) {
throw new ConnectException(
Expand Down
16 changes: 15 additions & 1 deletion client-v2/src/main/java/com/clickhouse/client/api/Client.java
Original file line number Diff line number Diff line change
@@ -1,6 +1,7 @@
package com.clickhouse.client.api;

import com.clickhouse.client.ClickHouseClient;
import com.clickhouse.client.ClickHouseException;
import com.clickhouse.client.ClickHouseNode;
import com.clickhouse.client.ClickHouseRequest;
import com.clickhouse.client.ClickHouseResponse;
Expand Down Expand Up @@ -80,6 +81,7 @@
import java.util.TimeZone;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
Expand Down Expand Up @@ -1455,6 +1457,11 @@ public CompletableFuture<InsertResponse> insert(String tableName,
return response;
} catch (ExecutionException e) {
throw new ClientException("Failed to get insert response", e.getCause());
} catch (CompletionException e) {
if (e.getCause() instanceof ClickHouseException) {
throw new ServerException(((ClickHouseException)e.getCause()).getErrorCode(), e.getCause().getMessage().trim());
}
throw new ClientException("Failed to get query response", e.getCause());
} catch (InterruptedException | TimeoutException e) {
throw new ClientException("Operation has likely timed out.", e);
}
Expand Down Expand Up @@ -1575,7 +1582,7 @@ public CompletableFuture<QueryResponse> query(String sqlQuery, Map<String, Objec
} else {
throw lastException;
}
} catch (ClientException e) {
} catch (ClientException | ServerException e) {
throw e;
} catch (Exception e) {
throw new ClientException("Query request failed", e);
Expand Down Expand Up @@ -1609,6 +1616,11 @@ public CompletableFuture<QueryResponse> query(String sqlQuery, Map<String, Objec
return new QueryResponse(clickHouseResponse, format, clientStats, finalSettings);
} catch (ClientException e) {
throw e;
} catch (CompletionException e) {
if (e.getCause() instanceof ClickHouseException) {
throw new ServerException(((ClickHouseException)e.getCause()).getErrorCode(), e.getCause().getMessage().trim());
}
throw new ClientException("Failed to get query response", e.getCause());
} catch (Exception e) {
throw new ClientException("Failed to get query response", e);
}
Expand Down Expand Up @@ -1795,6 +1807,8 @@ private TableSchema getTableSchemaImpl(String describeQuery, String name, String
throw new ClientException("Operation has likely timed out after " + getOperationTimeout() + " seconds.", e);
} catch (ExecutionException e) {
throw new ClientException("Failed to get table schema", e.getCause());
} catch (ServerException e) {
throw e;
} catch (Exception e) {
throw new ClientException("Failed to get table schema", e);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -354,7 +354,7 @@ public ClassicHttpResponse executeRequest(ClickHouseNode server, Map<String, Obj
} else if (httpResponse.getCode() == HttpStatus.SC_BAD_GATEWAY) {
httpResponse.close();
throw new ClientException("Server returned '502 Bad gateway'. Check network and proxy settings.");
} else if (httpResponse.getCode() >= HttpStatus.SC_BAD_REQUEST) {
} else if (httpResponse.getCode() >= HttpStatus.SC_BAD_REQUEST || httpResponse.containsHeader(ClickHouseHttpProto.HEADER_EXCEPTION_CODE)) {
try {
throw readError(httpResponse);
} finally {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import org.apache.hc.core5.http.ConnectionRequestTimeoutException;
import org.apache.hc.core5.http.HttpStatus;
import org.apache.hc.core5.net.URIBuilder;
import org.eclipse.jetty.server.Server;
import org.testcontainers.utility.ThrowingFunction;
import org.testng.Assert;
import org.testng.annotations.DataProvider;
Expand Down Expand Up @@ -320,12 +321,11 @@ public void testServerErrorHandling(ClickHouseFormat format) {
try (QueryResponse response =
client.query("SELECT invalid;statement", querySettings).get(1, TimeUnit.SECONDS)) {
Assert.fail("Expected exception");
} catch (ClientException e) {
} catch (ServerException e) {
e.printStackTrace();
ServerException serverException = (ServerException) e.getCause();
Assert.assertEquals(serverException.getCode(), 62);
Assert.assertTrue(serverException.getMessage().startsWith("Code: 62. DB::Exception: Syntax error (Multi-statements are not allowed): failed at position 15 (end of query)"),
"Unexpected error message: " + serverException.getMessage());
Assert.assertEquals(e.getCode(), 62);
Assert.assertTrue(e.getMessage().startsWith("Code: 62. DB::Exception: Syntax error (Multi-statements are not allowed): failed at position 15 (end of query)"),
"Unexpected error message: " + e.getMessage());
}
} catch (Exception e) {
e.printStackTrace();
Expand All @@ -338,6 +338,43 @@ public static Object[] testServerErrorHandlingDataProvider() {
return new Object[] { ClickHouseFormat.JSON, ClickHouseFormat.TabSeparated, ClickHouseFormat.RowBinary };
}


@Test(groups = { "integration" })
public void testErrorWithSuccessfulResponse() {
WireMockServer mockServer = new WireMockServer( WireMockConfiguration
.options().port(9090).notifier(new ConsoleNotifier(false)));
mockServer.start();

try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false)
.setUsername("default")
.setPassword("")
.compressServerResponse(false)
.useNewImplementation(true)
.build()) {
mockServer.addStubMapping(WireMock.post(WireMock.anyUrl())
.willReturn(WireMock.aResponse()
.withStatus(HttpStatus.SC_OK)
.withChunkedDribbleDelay(2, 200)
.withHeader("X-ClickHouse-Exception-Code", "241")
.withHeader("X-ClickHouse-Summary",
"{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")
.withBody("Code: 241. DB::Exception: Memory limit (for query) exceeded: would use 97.21 MiB"))
.build());

try (QueryResponse response = client.query("SELECT 1").get(1, TimeUnit.SECONDS)) {
Assert.fail("Expected exception");
} catch (ServerException e) {
e.printStackTrace();
Assert.assertEquals(e.getMessage(), "Code: 241. DB::Exception: Memory limit (for query) exceeded: would use 97.21 MiB");
} catch (Exception e) {
e.printStackTrace();
Assert.fail("Unexpected exception", e);
}
} finally {
mockServer.stop();
}
}

@Test(groups = { "integration" })
public void testAdditionalHeaders() {
WireMockServer mockServer = new WireMockServer( WireMockConfiguration
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@
import com.clickhouse.client.ClickHouseProtocol;
import com.clickhouse.client.api.Client;
import com.clickhouse.client.api.ClientException;
import com.clickhouse.client.api.ServerException;
import com.clickhouse.client.api.command.CommandResponse;
import com.clickhouse.client.api.enums.Protocol;
import org.testng.Assert;
Expand Down Expand Up @@ -45,8 +46,10 @@ public void testCreateTable() throws Exception {
public void testInvalidCommandExecution() throws Exception {
try {
client.execute("ALTER TABLE non_existing_table ADD COLUMN id2 UInt32").get(10, TimeUnit.SECONDS);
} catch (ServerException e) {
Assert.assertEquals(e.getCode(), 60);
} catch (ExecutionException e) {
Assert.assertTrue(e.getCause() instanceof ClientException);
Assert.assertTrue(e.getCause() instanceof ServerException);
} catch (ClientException e) {
// expected
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -550,8 +550,10 @@ public void testQueryExceptionHandling() throws Exception {
try {
client.queryRecords("SELECT * FROM unknown_table").get(3, TimeUnit.SECONDS);
Assert.fail("expected exception");
} catch (ServerException e) {
Assert.assertTrue(e.getMessage().contains("Unknown table"));
} catch (ExecutionException e) {
Assert.assertTrue(e.getCause() instanceof ClientException);
Assert.assertTrue(e.getCause() instanceof ServerException);
} catch (ClientException e) {
// expected
}
Expand Down
Loading