Skip to content

Commit d23be8b

Browse files
authored
added proper exception handling for query (#1678)
1 parent 21af845 commit d23be8b

File tree

2 files changed

+22
-19
lines changed

2 files changed

+22
-19
lines changed

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

Lines changed: 14 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -16,12 +16,12 @@
1616
import com.clickhouse.client.api.insert.InsertSettings;
1717
import com.clickhouse.client.api.insert.POJOSerializer;
1818
import com.clickhouse.client.api.insert.SerializerNotFoundException;
19+
import com.clickhouse.client.api.internal.ClientStatisticsHolder;
1920
import com.clickhouse.client.api.internal.SerializerUtils;
2021
import com.clickhouse.client.api.internal.SettingsConverter;
2122
import com.clickhouse.client.api.internal.TableSchemaParser;
2223
import com.clickhouse.client.api.internal.ValidationUtils;
2324
import com.clickhouse.client.api.metadata.TableSchema;
24-
import com.clickhouse.client.api.internal.ClientStatisticsHolder;
2525
import com.clickhouse.client.api.metrics.ClientMetrics;
2626
import com.clickhouse.client.api.query.GenericRecord;
2727
import com.clickhouse.client.api.query.QueryResponse;
@@ -35,7 +35,6 @@
3535
import com.clickhouse.data.format.BinaryStreamUtils;
3636
import org.slf4j.Logger;
3737
import org.slf4j.LoggerFactory;
38-
import org.slf4j.MDC;
3938

4039
import java.io.ByteArrayInputStream;
4140
import java.io.ByteArrayOutputStream;
@@ -708,21 +707,19 @@ public CompletableFuture<QueryResponse> query(String sqlQuery, Map<String, Objec
708707
final ClickHouseFormat format = settings.getFormat();
709708
request.format(format);
710709

711-
CompletableFuture<QueryResponse> future = new CompletableFuture<>();
712710
final QuerySettings finalSettings = settings;
713-
queryExecutor.submit(() -> {
714-
MDC.put("queryId", finalSettings.getQueryId());
711+
CompletableFuture<QueryResponse> future = CompletableFuture.supplyAsync(() -> {
715712
LOG.trace("Executing request: {}", request);
716713
try {
717714
QueryResponse queryResponse = new QueryResponse(client, request.execute(), finalSettings, format, clientStats);
718715
queryResponse.ensureDone();
719-
future.complete(queryResponse);
716+
return queryResponse;
717+
} catch (ClientException e) {
718+
throw e;
720719
} catch (Exception e) {
721-
future.completeExceptionally(e);
722-
} finally {
723-
MDC.remove("queryId");
720+
throw new ClientException("Failed to get query response", e);
724721
}
725-
});
722+
}, queryExecutor);
726723
return future;
727724
}
728725

@@ -764,21 +761,20 @@ public CompletableFuture<Records> queryRecords(String sqlQuery, QuerySettings se
764761
final ClickHouseFormat format = settings.getFormat();
765762
request.format(format);
766763

767-
CompletableFuture<Records> future = new CompletableFuture<>();
768764
final QuerySettings finalSettings = settings;
769-
queryExecutor.submit(() -> {
770-
MDC.put("queryId", finalSettings.getQueryId());
765+
CompletableFuture<Records> future = CompletableFuture.supplyAsync(() -> {
771766
LOG.trace("Executing request: {}", request);
772767
try {
773768
QueryResponse queryResponse = new QueryResponse(client, request.execute(), finalSettings, format, clientStats);
774769
queryResponse.ensureDone();
775-
future.complete(new Records(queryResponse, finalSettings));
770+
return new Records(queryResponse, finalSettings);
771+
} catch (ClientException e) {
772+
throw e;
776773
} catch (Exception e) {
777-
future.completeExceptionally(e);
778-
} finally {
779-
MDC.remove("queryId");
774+
throw new ClientException("Failed to get query response", e);
780775
}
781-
});
776+
}, queryExecutor);
777+
782778
return future;
783779
}
784780

client-v2/src/test/java/com/clickhouse/client/query/QueryTests.java

Lines changed: 8 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -10,6 +10,7 @@
1010
import com.clickhouse.client.ClickHouseRequest;
1111
import com.clickhouse.client.ClickHouseResponse;
1212
import com.clickhouse.client.api.Client;
13+
import com.clickhouse.client.api.ClientException;
1314
import com.clickhouse.client.api.DataTypeUtils;
1415
import com.clickhouse.client.api.Protocol;
1516
import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader;
@@ -364,8 +365,14 @@ public void testMapValues() throws Exception {
364365

365366

366367
@Test(groups = {"integration"})
367-
public void testQueryExceptionHandling() {
368+
public void testQueryExceptionHandling() throws Exception {
368369

370+
try {
371+
client.queryRecords("SELECT * FROM unknown_table").get(3, TimeUnit.SECONDS);
372+
Assert.fail("expected exception");
373+
} catch (ExecutionException e) {
374+
Assert.assertTrue(e.getCause() instanceof ClientException);
375+
}
369376
}
370377

371378

0 commit comments

Comments
 (0)