6363import java .lang .reflect .InvocationTargetException ;
6464import java .lang .reflect .Method ;
6565import java .net .ConnectException ;
66+ import java .net .SocketTimeoutException ;
6667import java .net .URL ;
6768import java .nio .charset .StandardCharsets ;
6869import java .time .Duration ;
@@ -1336,7 +1337,7 @@ public CompletableFuture<InsertResponse> insert(String tableName, List<?> data,
13361337 // Selecting some node
13371338 ClickHouseNode selectedNode = getNextAliveNode ();
13381339
1339- ClientException lastException = null ;
1340+ RuntimeException lastException = null ;
13401341 for (int i = 0 ; i <= maxRetries ; i ++) {
13411342 // Execute request
13421343 try (ClassicHttpResponse httpResponse =
@@ -1376,19 +1377,17 @@ public CompletableFuture<InsertResponse> insert(String tableName, List<?> data,
13761377 metrics .operationComplete ();
13771378 metrics .setQueryId (queryId );
13781379 return new InsertResponse (metrics );
1379- } catch (NoHttpResponseException | ConnectionRequestTimeoutException | ConnectTimeoutException | ConnectException e ) {
1380- lastException = httpClientHelper .wrapException ("Insert request initiation failed" , e );
1380+ } catch (Exception e ) {
1381+ lastException = httpClientHelper .wrapException ("Query request failed (Attempt " + ( i + 1 ) + "/" + ( maxRetries + 1 ) + ") " , e );
13811382 if (httpClientHelper .shouldRetry (e , finalSettings .getAllSettings ())) {
1382- LOG .warn ("Retrying" , e );
1383+ LOG .warn ("Retrying. " , e );
13831384 selectedNode = getNextAliveNode ();
13841385 } else {
13851386 throw lastException ;
13861387 }
1387- } catch (IOException e ) {
1388- throw new ClientException ("Insert request failed" , e );
13891388 }
13901389 }
1391- throw new ClientException ("Insert request failed after retries" , lastException );
1390+ throw new ClientException ("Insert request failed after attempts: " + ( maxRetries + 1 ) , lastException );
13921391 };
13931392
13941393 return runAsyncOperation (supplier , settings .getAllSettings ());
@@ -1513,7 +1512,7 @@ public CompletableFuture<InsertResponse> insert(String tableName,
15131512 // Selecting some node
15141513 ClickHouseNode selectedNode = getNextAliveNode ();
15151514
1516- ClientException lastException = null ;
1515+ RuntimeException lastException = null ;
15171516 for (int i = 0 ; i <= maxRetries ; i ++) {
15181517 // Execute request
15191518 try (ClassicHttpResponse httpResponse =
@@ -1538,16 +1537,14 @@ public CompletableFuture<InsertResponse> insert(String tableName,
15381537 metrics .operationComplete ();
15391538 metrics .setQueryId (queryId );
15401539 return new InsertResponse (metrics );
1541- } catch (NoHttpResponseException | ConnectionRequestTimeoutException | ConnectTimeoutException | ConnectException e ) {
1542- lastException = httpClientHelper .wrapException ("Insert request initiation failed" , e );
1540+ } catch (Exception e ) {
1541+ lastException = httpClientHelper .wrapException ("Query request failed (Attempt " + ( i + 1 ) + "/" + ( maxRetries + 1 ) + ") " , e );
15431542 if (httpClientHelper .shouldRetry (e , finalSettings .getAllSettings ())) {
1544- LOG .warn ("Retrying" , e );
1543+ LOG .warn ("Retrying. " , e );
15451544 selectedNode = getNextAliveNode ();
15461545 } else {
15471546 throw lastException ;
15481547 }
1549- } catch (IOException e ) {
1550- throw new ClientException ("Insert request failed" , e );
15511548 }
15521549
15531550 if (i < maxRetries ) {
@@ -1558,7 +1555,7 @@ public CompletableFuture<InsertResponse> insert(String tableName,
15581555 }
15591556 }
15601557 }
1561- throw new ClientException ("Insert request failed after retries" , lastException );
1558+ throw new ClientException ("Insert request failed after attempts: " + ( maxRetries + 1 ) , lastException );
15621559 };
15631560 } else {
15641561 responseSupplier = () -> {
@@ -1672,7 +1669,7 @@ public CompletableFuture<QueryResponse> query(String sqlQuery, Map<String, Objec
16721669 responseSupplier = () -> {
16731670 // Selecting some node
16741671 ClickHouseNode selectedNode = getNextAliveNode ();
1675- ClientException lastException = null ;
1672+ RuntimeException lastException = null ;
16761673 for (int i = 0 ; i <= maxRetries ; i ++) {
16771674 try {
16781675 ClassicHttpResponse httpResponse =
@@ -1699,22 +1696,18 @@ public CompletableFuture<QueryResponse> query(String sqlQuery, Map<String, Objec
16991696
17001697 return new QueryResponse (httpResponse , finalSettings .getFormat (), finalSettings , metrics );
17011698
1702- } catch (NoHttpResponseException | ConnectionRequestTimeoutException | ConnectTimeoutException | ConnectException e ) {
1703- lastException = httpClientHelper .wrapException ("Query request initiation failed" , e );
1699+ } catch (Exception e ) {
1700+ lastException = httpClientHelper .wrapException ("Query request failed (Attempt " + ( i + 1 ) + "/" + ( maxRetries + 1 ) + ") " , e );
17041701 if (httpClientHelper .shouldRetry (e , finalSettings .getAllSettings ())) {
17051702 LOG .warn ("Retrying." , e );
17061703 selectedNode = getNextAliveNode ();
17071704 } else {
17081705 throw lastException ;
17091706 }
1710- } catch (ClientException | ServerException e ) {
1711- throw e ;
1712- } catch (Exception e ) {
1713- throw new ClientException ("Query request failed" , e );
17141707 }
17151708 }
17161709
1717- throw new ClientException ("Query request failed after retries" , lastException );
1710+ throw new ClientException ("Query request failed after attempts: " + ( maxRetries + 1 ) , lastException );
17181711 };
17191712 } else {
17201713 ClickHouseRequest <?> request = oldClient .read (getServerNode ());
0 commit comments