Skip to content

Commit 7656f84

Browse files
authored
Polish excpetion api + build in code list that is retryable (#2453)
* polish excpetion api + build in code list that is retryable * Adding ClickHouseException as base Excpetion * Change to ClickHouseException * Clean wrap of expection in the internal layer. * Added ServerRetryable to defaults * throw real exception instead of wrapping with ClientException * Change to the correct Exception * Support in to two diffreant error code that can cause the issue * Fix error code 243 -NOT_ENOUGH_SPACE * Remove ServerException wrong logic
1 parent 9993564 commit 7656f84

File tree

14 files changed

+156
-104
lines changed

14 files changed

+156
-104
lines changed
Lines changed: 18 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,18 @@
1+
package com.clickhouse.client.api;
2+
3+
public class ClickHouseException extends RuntimeException {
4+
protected boolean isRetryable = false;
5+
6+
public ClickHouseException(String message) {
7+
super(message);
8+
}
9+
10+
public ClickHouseException(String message, Throwable cause) {
11+
super(message, cause);
12+
}
13+
14+
public ClickHouseException(Throwable cause) {
15+
super(cause);
16+
}
17+
public boolean isRetryable() { return isRetryable; }
18+
}

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

Lines changed: 5 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -813,7 +813,7 @@ public Builder setClientNetworkBufferSize(int size) {
813813

814814
/**
815815
* Sets list of causes that should be retried on.
816-
* Default {@code [NoHttpResponse, ConnectTimeout, ConnectionRequestTimeout]}
816+
* Default {@code [NoHttpResponse, ConnectTimeout, ConnectionRequestTimeout, ServerRetryable]}
817817
* Use {@link ClientFaultCause#None} to disable retries.
818818
*
819819
* @param causes - list of causes
@@ -1464,7 +1464,8 @@ public CompletableFuture<InsertResponse> insert(String tableName,
14641464
}
14651465
}
14661466
}
1467-
throw new ClientException("Insert request failed after attempts: " + (retries + 1) + " - Duration: " + (System.nanoTime() - startTime), lastException);
1467+
LOG.warn("Insert request failed after attempts: " + (retries + 1) + " - Duration: " + (System.nanoTime() - startTime));
1468+
throw lastException;
14681469
};
14691470

14701471
return runAsyncOperation(responseSupplier, settings.getAllSettings());
@@ -1586,8 +1587,8 @@ public CompletableFuture<QueryResponse> query(String sqlQuery, Map<String, Objec
15861587
}
15871588
}
15881589
}
1589-
1590-
throw new ClientException("Query request failed after attempts: " + (retries + 1) + " - Duration: " + (System.nanoTime() - startTime), lastException);
1590+
LOG.warn("Query request failed after attempts: " + (retries + 1) + " - Duration: " + (System.nanoTime() - startTime));
1591+
throw lastException;
15911592
};
15921593

15931594
return runAsyncOperation(responseSupplier, settings.getAllSettings());

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -120,7 +120,7 @@ public enum ClientConfigProperties {
120120

121121
CLIENT_RETRY_ON_FAILURE("client_retry_on_failures",
122122
String.join(",", ClientFaultCause.NoHttpResponse.name(), ClientFaultCause.ConnectTimeout.name(),
123-
ClientFaultCause.ConnectionRequestTimeout.name())),
123+
ClientFaultCause.ConnectionRequestTimeout.name(), ClientFaultCause.ServerRetryable.name())),
124124

125125
CLIENT_NAME("client_name"),
126126

Lines changed: 12 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -1,12 +1,12 @@
1-
package com.clickhouse.client.api;
2-
3-
public class ClientException extends RuntimeException {
4-
5-
public ClientException(String message) {
6-
super(message);
7-
}
8-
9-
public ClientException(String message, Throwable cause) {
10-
super(message, cause);
11-
}
12-
}
1+
package com.clickhouse.client.api;
2+
3+
public class ClientException extends ClickHouseException {
4+
5+
public ClientException(String message) {
6+
super(message);
7+
}
8+
9+
public ClientException(String message, Throwable cause) {
10+
super(message, cause);
11+
}
12+
}

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

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -8,4 +8,5 @@ public enum ClientFaultCause {
88
ConnectTimeout,
99
ConnectionRequestTimeout,
1010
SocketTimeout,
11+
ServerRetryable,
1112
}
Lines changed: 14 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -1,12 +1,14 @@
1-
package com.clickhouse.client.api;
2-
3-
public class ConnectionInitiationException extends ClientException {
4-
5-
public ConnectionInitiationException(String message) {
6-
super(message);
7-
}
8-
9-
public ConnectionInitiationException(String message, Throwable cause) {
10-
super(message, cause);
11-
}
12-
}
1+
package com.clickhouse.client.api;
2+
3+
public class ConnectionInitiationException extends ClickHouseException {
4+
5+
public ConnectionInitiationException(String message) {
6+
super(message);
7+
this.isRetryable = true;
8+
}
9+
10+
public ConnectionInitiationException(String message, Throwable cause) {
11+
super(message, cause);
12+
this.isRetryable = true;
13+
}
14+
}
Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,12 @@
1+
package com.clickhouse.client.api;
2+
3+
public class DataTransferException extends ClickHouseException {
4+
5+
public DataTransferException(String message) {
6+
super(message);
7+
}
8+
9+
public DataTransferException(String message, Throwable cause) {
10+
super(message, cause);
11+
}
12+
}
Lines changed: 69 additions & 40 deletions
Original file line numberDiff line numberDiff line change
@@ -1,40 +1,69 @@
1-
package com.clickhouse.client.api;
2-
3-
public class ServerException extends RuntimeException {
4-
5-
public static final int CODE_UNKNOWN = 0;
6-
7-
public static final int TABLE_NOT_FOUND = 60;
8-
9-
private final int code;
10-
11-
private final int transportProtocolCode;
12-
13-
public ServerException(int code, String message) {
14-
this(code, message, 500);
15-
}
16-
17-
public ServerException(int code, String message, int transportProtocolCode) {
18-
super(message);
19-
this.code = code;
20-
this.transportProtocolCode = transportProtocolCode;
21-
}
22-
23-
/**
24-
* Returns CH server error code. May return 0 if code is unknown.
25-
* @return - error code from server response
26-
*/
27-
public int getCode() {
28-
return code;
29-
}
30-
31-
/**
32-
* Returns error code of underlying transport protocol. For example, HTTP status.
33-
* By default, will return {@code 500 } what is derived from HTTP Server Internal Error.
34-
*
35-
* @return - transport status code
36-
*/
37-
public int getTransportProtocolCode() {
38-
return transportProtocolCode;
39-
}
40-
}
1+
package com.clickhouse.client.api;
2+
3+
public class ServerException extends ClickHouseException {
4+
5+
public static final int CODE_UNKNOWN = 0;
6+
7+
public static final int TABLE_NOT_FOUND = 60;
8+
9+
private final int code;
10+
11+
private final int transportProtocolCode;
12+
13+
public ServerException(int code, String message) {
14+
this(code, message, 500);
15+
}
16+
17+
public ServerException(int code, String message, int transportProtocolCode) {
18+
super(message);
19+
this.code = code;
20+
this.transportProtocolCode = transportProtocolCode;
21+
this.isRetryable = discoverIsRetryable(code, message, transportProtocolCode);
22+
}
23+
24+
/**
25+
* Returns CH server error code. May return 0 if code is unknown.
26+
* @return - error code from server response
27+
*/
28+
public int getCode() {
29+
return code;
30+
}
31+
32+
/**
33+
* Returns error code of underlying transport protocol. For example, HTTP status.
34+
* By default, will return {@code 500 } what is derived from HTTP Server Internal Error.
35+
*
36+
* @return - transport status code
37+
*/
38+
public int getTransportProtocolCode() {
39+
return transportProtocolCode;
40+
}
41+
42+
public boolean isRetryable() {
43+
return isRetryable;
44+
}
45+
46+
private boolean discoverIsRetryable(int code, String message, int transportProtocolCode) {
47+
//Let's check if we have a ServerException to reference the error code
48+
//https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/ErrorCodes.cpp
49+
switch (code) { // UNEXPECTED_END_OF_FILE
50+
case 3: // UNEXPECTED_END_OF_FILE
51+
case 107: // FILE_DOESNT_EXIST
52+
case 159: // TIMEOUT_EXCEEDED
53+
case 164: // READONLY
54+
case 202: // TOO_MANY_SIMULTANEOUS_QUERIES
55+
case 203: // NO_FREE_CONNECTION
56+
case 209: // SOCKET_TIMEOUT
57+
case 210: // NETWORK_ERROR
58+
case 241: // MEMORY_LIMIT_EXCEEDED
59+
case 242: // TABLE_IS_READ_ONLY
60+
case 252: // TOO_MANY_PARTS
61+
case 285: // TOO_FEW_LIVE_REPLICAS
62+
case 319: // UNKNOWN_STATUS_OF_INSERT
63+
case 425: // SYSTEM_ERROR
64+
case 999: // KEEPER_EXCEPTION
65+
return true;
66+
};
67+
return false;
68+
}
69+
}

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

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -33,10 +33,8 @@
3333
import java.util.List;
3434
import java.util.Map;
3535
import java.util.Set;
36-
import java.util.Stack;
3736
import java.util.TimeZone;
3837
import java.util.UUID;
39-
import java.util.concurrent.TimeUnit;
4038

4139
/**
4240
* This class is not thread safe and should not be shared between multiple threads.

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

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -5,15 +5,13 @@
55
import com.clickhouse.client.api.query.GenericRecord;
66
import com.clickhouse.client.api.query.NullValueException;
77
import com.clickhouse.data.ClickHouseColumn;
8-
import com.clickhouse.data.ClickHouseDataType;
98
import com.clickhouse.data.value.*;
109

1110
import java.math.BigDecimal;
1211
import java.math.BigInteger;
1312
import java.net.Inet4Address;
1413
import java.net.Inet6Address;
1514
import java.time.*;
16-
import java.time.temporal.ChronoUnit;
1715
import java.time.temporal.TemporalAmount;
1816
import java.util.HashMap;
1917
import java.util.List;

0 commit comments

Comments
 (0)