5555import java .lang .reflect .InvocationTargetException ;
5656import java .lang .reflect .Method ;
5757import java .net .ConnectException ;
58+ import java .net .SocketTimeoutException ;
5859import java .net .URL ;
5960import java .nio .charset .StandardCharsets ;
6061import java .time .Duration ;
@@ -1302,7 +1303,7 @@ public CompletableFuture<InsertResponse> insert(String tableName, List<?> data,
13021303 // Selecting some node
13031304 ClickHouseNode selectedNode = getNextAliveNode ();
13041305
1305- ClientException lastException = null ;
1306+ RuntimeException lastException = null ;
13061307 for (int i = 0 ; i <= maxRetries ; i ++) {
13071308 // Execute request
13081309 try (ClassicHttpResponse httpResponse =
@@ -1342,19 +1343,17 @@ public CompletableFuture<InsertResponse> insert(String tableName, List<?> data,
13421343 metrics .operationComplete ();
13431344 metrics .setQueryId (queryId );
13441345 return new InsertResponse (metrics );
1345- } catch (NoHttpResponseException | ConnectionRequestTimeoutException | ConnectTimeoutException | ConnectException e ) {
1346- lastException = httpClientHelper .wrapException ("Insert request initiation failed" , e );
1346+ } catch (Exception e ) {
1347+ lastException = httpClientHelper .wrapException ("Query request failed (Attempt " + ( i + 1 ) + "/" + ( maxRetries + 1 ) + ") " , e );
13471348 if (httpClientHelper .shouldRetry (e , finalSettings .getAllSettings ())) {
1348- LOG .warn ("Retrying" , e );
1349+ LOG .warn ("Retrying. " , e );
13491350 selectedNode = getNextAliveNode ();
13501351 } else {
13511352 throw lastException ;
13521353 }
1353- } catch (IOException e ) {
1354- throw new ClientException ("Insert request failed" , e );
13551354 }
13561355 }
1357- throw new ClientException ("Insert request failed after retries" , lastException );
1356+ throw new ClientException ("Insert request failed after attempts: " + ( maxRetries + 1 ) , lastException );
13581357 };
13591358
13601359 return runAsyncOperation (supplier , settings .getAllSettings ());
@@ -1462,7 +1461,7 @@ public CompletableFuture<InsertResponse> insert(String tableName,
14621461 // Selecting some node
14631462 ClickHouseNode selectedNode = getNextAliveNode ();
14641463
1465- ClientException lastException = null ;
1464+ RuntimeException lastException = null ;
14661465 for (int i = 0 ; i <= maxRetries ; i ++) {
14671466 // Execute request
14681467 try (ClassicHttpResponse httpResponse =
@@ -1487,16 +1486,14 @@ public CompletableFuture<InsertResponse> insert(String tableName,
14871486 metrics .operationComplete ();
14881487 metrics .setQueryId (queryId );
14891488 return new InsertResponse (metrics );
1490- } catch (NoHttpResponseException | ConnectionRequestTimeoutException | ConnectTimeoutException | ConnectException e ) {
1491- lastException = httpClientHelper .wrapException ("Insert request initiation failed" , e );
1489+ } catch (Exception e ) {
1490+ lastException = httpClientHelper .wrapException ("Query request failed (Attempt " + ( i + 1 ) + "/" + ( maxRetries + 1 ) + ") " , e );
14921491 if (httpClientHelper .shouldRetry (e , finalSettings .getAllSettings ())) {
1493- LOG .warn ("Retrying" , e );
1492+ LOG .warn ("Retrying. " , e );
14941493 selectedNode = getNextAliveNode ();
14951494 } else {
14961495 throw lastException ;
14971496 }
1498- } catch (IOException e ) {
1499- throw new ClientException ("Insert request failed" , e );
15001497 }
15011498
15021499 if (i < maxRetries ) {
@@ -1507,7 +1504,7 @@ public CompletableFuture<InsertResponse> insert(String tableName,
15071504 }
15081505 }
15091506 }
1510- throw new ClientException ("Insert request failed after retries" , lastException );
1507+ throw new ClientException ("Insert request failed after attempts: " + ( maxRetries + 1 ) , lastException );
15111508 };
15121509
15131510 return runAsyncOperation (responseSupplier , settings .getAllSettings ());
@@ -1588,7 +1585,7 @@ public CompletableFuture<QueryResponse> query(String sqlQuery, Map<String, Objec
15881585 responseSupplier = () -> {
15891586 // Selecting some node
15901587 ClickHouseNode selectedNode = getNextAliveNode ();
1591- ClientException lastException = null ;
1588+ RuntimeException lastException = null ;
15921589 for (int i = 0 ; i <= maxRetries ; i ++) {
15931590 try {
15941591 ClassicHttpResponse httpResponse =
@@ -1615,22 +1612,18 @@ public CompletableFuture<QueryResponse> query(String sqlQuery, Map<String, Objec
16151612
16161613 return new QueryResponse (httpResponse , finalSettings .getFormat (), finalSettings , metrics );
16171614
1618- } catch (NoHttpResponseException | ConnectionRequestTimeoutException | ConnectTimeoutException | ConnectException e ) {
1619- lastException = httpClientHelper .wrapException ("Query request initiation failed" , e );
1615+ } catch (Exception e ) {
1616+ lastException = httpClientHelper .wrapException ("Query request failed (Attempt " + ( i + 1 ) + "/" + ( maxRetries + 1 ) + ") " , e );
16201617 if (httpClientHelper .shouldRetry (e , finalSettings .getAllSettings ())) {
16211618 LOG .warn ("Retrying." , e );
16221619 selectedNode = getNextAliveNode ();
16231620 } else {
16241621 throw lastException ;
16251622 }
1626- } catch (ClientException | ServerException e ) {
1627- throw e ;
1628- } catch (Exception e ) {
1629- throw new ClientException ("Query request failed" , e );
16301623 }
16311624 }
16321625
1633- throw new ClientException ("Query request failed after retries" , lastException );
1626+ throw new ClientException ("Query request failed after attempts: " + ( maxRetries + 1 ) , lastException );
16341627 };
16351628
16361629 return runAsyncOperation (responseSupplier , settings .getAllSettings ());
0 commit comments