Skip to content
Original file line number Diff line number Diff line change
@@ -0,0 +1,18 @@
package com.clickhouse.client.api;

public class ClickHouseException extends RuntimeException {
protected boolean isRetryable = false;

public ClickHouseException(String message) {
super(message);
}

public ClickHouseException(String message, Throwable cause) {
super(message, cause);
}

public ClickHouseException(Throwable cause) {
super(cause);
}
public boolean isRetryable() { return isRetryable; }
}
9 changes: 5 additions & 4 deletions client-v2/src/main/java/com/clickhouse/client/api/Client.java
Original file line number Diff line number Diff line change
Expand Up @@ -813,7 +813,7 @@ public Builder setClientNetworkBufferSize(int size) {

/**
* Sets list of causes that should be retried on.
* Default {@code [NoHttpResponse, ConnectTimeout, ConnectionRequestTimeout]}
* Default {@code [NoHttpResponse, ConnectTimeout, ConnectionRequestTimeout, ServerRetryable]}
* Use {@link ClientFaultCause#None} to disable retries.
*
* @param causes - list of causes
Expand Down Expand Up @@ -1464,7 +1464,8 @@ public CompletableFuture<InsertResponse> insert(String tableName,
}
}
}
throw new ClientException("Insert request failed after attempts: " + (retries + 1) + " - Duration: " + (System.nanoTime() - startTime), lastException);
LOG.warn("Insert request failed after attempts: " + (retries + 1) + " - Duration: " + (System.nanoTime() - startTime));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The warning log is missing the exception object parameter. Consider adding the lastException as the last parameter to the LOG.warn() call to include the stack trace in logs:

Suggested change
LOG.warn("Insert request failed after attempts: " + (retries + 1) + " - Duration: " + (System.nanoTime() - startTime));
LOG.warn("Insert request failed after attempts: " + (retries + 1) + " - Duration: " + (System.nanoTime() - startTime), lastException);

throw lastException;
};

return runAsyncOperation(responseSupplier, settings.getAllSettings());
Expand Down Expand Up @@ -1586,8 +1587,8 @@ public CompletableFuture<QueryResponse> query(String sqlQuery, Map<String, Objec
}
}
}

throw new ClientException("Query request failed after attempts: " + (retries + 1) + " - Duration: " + (System.nanoTime() - startTime), lastException);
LOG.warn("Query request failed after attempts: " + (retries + 1) + " - Duration: " + (System.nanoTime() - startTime));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The warning log is missing the exception object parameter. Consider adding the lastException as the last parameter to the LOG.warn() call to include the stack trace in logs:

Suggested change
LOG.warn("Query request failed after attempts: " + (retries + 1) + " - Duration: " + (System.nanoTime() - startTime));
LOG.warn("Query request failed after attempts: " + (retries + 1) + " - Duration: " + (System.nanoTime() - startTime), lastException);

throw lastException;
};

return runAsyncOperation(responseSupplier, settings.getAllSettings());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -120,7 +120,7 @@ public enum ClientConfigProperties {

CLIENT_RETRY_ON_FAILURE("client_retry_on_failures",
String.join(",", ClientFaultCause.NoHttpResponse.name(), ClientFaultCause.ConnectTimeout.name(),
ClientFaultCause.ConnectionRequestTimeout.name())),
ClientFaultCause.ConnectionRequestTimeout.name(), ClientFaultCause.ServerRetryable.name())),

CLIENT_NAME("client_name"),

Expand Down
Original file line number Diff line number Diff line change
@@ -1,12 +1,12 @@
package com.clickhouse.client.api;
public class ClientException extends RuntimeException {
public ClientException(String message) {
super(message);
}
public ClientException(String message, Throwable cause) {
super(message, cause);
}
}
package com.clickhouse.client.api;

public class ClientException extends ClickHouseException {

public ClientException(String message) {
super(message);
}

public ClientException(String message, Throwable cause) {
super(message, cause);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -8,4 +8,5 @@ public enum ClientFaultCause {
ConnectTimeout,
ConnectionRequestTimeout,
SocketTimeout,
ServerRetryable,
}
Original file line number Diff line number Diff line change
@@ -1,12 +1,14 @@
package com.clickhouse.client.api;

public class ConnectionInitiationException extends ClientException {

public ConnectionInitiationException(String message) {
super(message);
}

public ConnectionInitiationException(String message, Throwable cause) {
super(message, cause);
}
}
package com.clickhouse.client.api;

public class ConnectionInitiationException extends ClickHouseException {

public ConnectionInitiationException(String message) {
super(message);
this.isRetryable = true;
}

public ConnectionInitiationException(String message, Throwable cause) {
super(message, cause);
this.isRetryable = true;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,12 @@
package com.clickhouse.client.api;

public class DataTransferException extends ClickHouseException {

public DataTransferException(String message) {
super(message);
}

public DataTransferException(String message, Throwable cause) {
super(message, cause);
}
}
109 changes: 69 additions & 40 deletions client-v2/src/main/java/com/clickhouse/client/api/ServerException.java
Original file line number Diff line number Diff line change
@@ -1,40 +1,69 @@
package com.clickhouse.client.api;

public class ServerException extends RuntimeException {

public static final int CODE_UNKNOWN = 0;

public static final int TABLE_NOT_FOUND = 60;

private final int code;

private final int transportProtocolCode;

public ServerException(int code, String message) {
this(code, message, 500);
}

public ServerException(int code, String message, int transportProtocolCode) {
super(message);
this.code = code;
this.transportProtocolCode = transportProtocolCode;
}

/**
* Returns CH server error code. May return 0 if code is unknown.
* @return - error code from server response
*/
public int getCode() {
return code;
}

/**
* Returns error code of underlying transport protocol. For example, HTTP status.
* By default, will return {@code 500 } what is derived from HTTP Server Internal Error.
*
* @return - transport status code
*/
public int getTransportProtocolCode() {
return transportProtocolCode;
}
}
package com.clickhouse.client.api;

public class ServerException extends ClickHouseException {

public static final int CODE_UNKNOWN = 0;

public static final int TABLE_NOT_FOUND = 60;

private final int code;

private final int transportProtocolCode;

public ServerException(int code, String message) {
this(code, message, 500);
}

public ServerException(int code, String message, int transportProtocolCode) {
super(message);
this.code = code;
this.transportProtocolCode = transportProtocolCode;
this.isRetryable = discoverIsRetryable(code, message, transportProtocolCode);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The code references this.isRetryable but there's no field declaration visible in this class. If this is inherited from ClickHouseException, consider adding a comment to clarify this, or use a more explicit reference to make the code more maintainable.

}

/**
* Returns CH server error code. May return 0 if code is unknown.
* @return - error code from server response
*/
public int getCode() {
return code;
}

/**
* Returns error code of underlying transport protocol. For example, HTTP status.
* By default, will return {@code 500 } what is derived from HTTP Server Internal Error.
*
* @return - transport status code
*/
public int getTransportProtocolCode() {
return transportProtocolCode;
}

public boolean isRetryable() {
return isRetryable;
}

private boolean discoverIsRetryable(int code, String message, int transportProtocolCode) {
//Let's check if we have a ServerException to reference the error code
//https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/ErrorCodes.cpp
switch (code) { // UNEXPECTED_END_OF_FILE
case 3: // UNEXPECTED_END_OF_FILE
case 107: // FILE_DOESNT_EXIST
case 159: // TIMEOUT_EXCEEDED
case 164: // READONLY
case 202: // TOO_MANY_SIMULTANEOUS_QUERIES
case 203: // NO_FREE_CONNECTION
case 209: // SOCKET_TIMEOUT
case 210: // NETWORK_ERROR
case 241: // MEMORY_LIMIT_EXCEEDED
case 242: // TABLE_IS_READ_ONLY
case 252: // TOO_MANY_PARTS
case 285: // TOO_FEW_LIVE_REPLICAS
case 319: // UNKNOWN_STATUS_OF_INSERT
case 425: // SYSTEM_ERROR
case 999: // KEEPER_EXCEPTION
return true;
};
return false;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -33,10 +33,8 @@
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.Stack;
import java.util.TimeZone;
import java.util.UUID;
import java.util.concurrent.TimeUnit;

/**
* This class is not thread safe and should not be shared between multiple threads.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5,15 +5,13 @@
import com.clickhouse.client.api.query.GenericRecord;
import com.clickhouse.client.api.query.NullValueException;
import com.clickhouse.data.ClickHouseColumn;
import com.clickhouse.data.ClickHouseDataType;
import com.clickhouse.data.value.*;

import java.math.BigDecimal;
import java.math.BigInteger;
import java.net.Inet4Address;
import java.net.Inet6Address;
import java.time.*;
import java.time.temporal.ChronoUnit;
import java.time.temporal.TemporalAmount;
import java.util.HashMap;
import java.util.List;
Expand Down
Original file line number Diff line number Diff line change
@@ -1,14 +1,7 @@
package com.clickhouse.client.api.internal;

import com.clickhouse.client.ClickHouseSslContextProvider;
import com.clickhouse.client.api.Client;
import com.clickhouse.client.api.ClientConfigProperties;
import com.clickhouse.client.api.ClientException;
import com.clickhouse.client.api.ClientFaultCause;
import com.clickhouse.client.api.ClientMisconfigurationException;
import com.clickhouse.client.api.ConnectionInitiationException;
import com.clickhouse.client.api.ConnectionReuseStrategy;
import com.clickhouse.client.api.ServerException;
import com.clickhouse.client.api.*;
import com.clickhouse.client.api.data_formats.internal.SerializerUtils;
import com.clickhouse.client.api.enums.ProxyType;
import com.clickhouse.client.api.http.ClickHouseHttpProto;
Expand Down Expand Up @@ -379,7 +372,7 @@ public Exception readError(ClassicHttpResponse httpResponse) {
private AtomicLong timeToPoolVent = new AtomicLong(0);

public ClassicHttpResponse executeRequest(Endpoint server, Map<String, Object> requestConfig, LZ4Factory lz4Factory,
IOCallback<OutputStream> writeCallback) throws IOException {
IOCallback<OutputStream> writeCallback) throws Exception {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The method signature has been changed from throwing IOException to throwing Exception, which is a more general exception type. This makes the method contract less specific and could lead to unexpected exception handling behavior in calling code. Consider keeping the original exception type or using a more specific exception type.

if (poolControl != null && timeToPoolVent.get() < System.currentTimeMillis()) {
timeToPoolVent.set(System.currentTimeMillis() + POOL_VENT_TIMEOUT);
poolControl.closeExpired();
Expand Down Expand Up @@ -432,14 +425,10 @@ public ClassicHttpResponse executeRequest(Endpoint server, Map<String, Object> r

} catch (UnknownHostException e) {
LOG.warn("Host '{}' unknown", server.getBaseURL());
throw new ClientException("Unknown host", e);
throw e;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This change removes the wrapping of UnknownHostException into ClientException. Callers might be expecting a ClientException rather than the original exception. Consider maintaining consistent exception wrapping behavior or updating all callers to handle the new exception type.

} catch (ConnectException | NoRouteToHostException e) {
LOG.warn("Failed to connect to '{}': {}", server.getBaseURL(), e.getMessage());
throw new ClientException("Failed to connect", e);
} catch (ConnectionRequestTimeoutException | ServerException | NoHttpResponseException | ClientException | SocketTimeoutException e) {
throw e;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This change removes the wrapping of ConnectException and NoRouteToHostException into ClientException. This could break existing error handling code that expects these exceptions to be wrapped. Consider maintaining consistent exception wrapping behavior.

} catch (Exception e) {
throw new ClientException(e.getMessage(), e);
}
}

Expand Down Expand Up @@ -651,6 +640,12 @@ public boolean shouldRetry(Throwable ex, Map<String, Object> requestSettings) {
return retryCauses.contains(ClientFaultCause.SocketTimeout);
}

// there are some db retryable error codes
if (ex instanceof ServerException || ex.getCause() instanceof ServerException) {
ServerException se = (ServerException) ex;
return se.isRetryable() && retryCauses.contains(ClientFaultCause.ServerRetryable);
}

return false;
}

Expand All @@ -664,11 +659,17 @@ public RuntimeException wrapException(String message, Exception cause) {
if (cause instanceof ConnectionRequestTimeoutException ||
cause instanceof NoHttpResponseException ||
cause instanceof ConnectTimeoutException ||
cause instanceof ConnectException) {
cause instanceof ConnectException ||
cause instanceof UnknownHostException ||
cause instanceof NoRouteToHostException) {
return new ConnectionInitiationException(message, cause);
}

return new ClientException(message, cause);
if (cause instanceof SocketTimeoutException || cause instanceof IOException) {
return new DataTransferException(message, cause);
}
// if we can not identify the exception explicitly we catch as our base exception ClickHouseException
return new ClickHouseException(message, cause);
}


Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -285,7 +285,7 @@ public void testWithOldDefaults() {
.enableConnectionPool(true)
.setConnectionTTL(-1, MILLIS)
.retryOnFailures(ClientFaultCause.NoHttpResponse, ClientFaultCause.ConnectTimeout,
ClientFaultCause.ConnectionRequestTimeout)
ClientFaultCause.ConnectionRequestTimeout, ClientFaultCause.ServerRetryable)
.setClientNetworkBufferSize(300_000)
.setMaxRetries(3)
.allowBinaryReaderToReuseBuffers(false)
Expand Down
Original file line number Diff line number Diff line change
@@ -1,5 +1,6 @@
package com.clickhouse.client;

import com.clickhouse.client.api.ClickHouseException;
import com.clickhouse.client.api.Client;
import com.clickhouse.client.api.ClientConfigProperties;
import com.clickhouse.client.api.ClientException;
Expand Down Expand Up @@ -280,7 +281,7 @@ public void testInsertAndNoHttpResponseFailure(String body, int maxRetries, Thro

try {
function.apply(mockServerClient);
} catch (ClientException e) {
} catch (ConnectionInitiationException e) {
e.printStackTrace();
if (!shouldFail) {
Assert.fail("Unexpected exception", e);
Expand Down Expand Up @@ -777,7 +778,8 @@ public void testErrorWithSendProgressHeaders() throws Exception {
try (QueryResponse resp = client.query("INSERT INTO test_omm_table SELECT randomString(16) FROM numbers(300000000)", settings).get()) {

} catch (ServerException e) {
Assert.assertEquals(e.getCode(), 241);
// 241 - MEMORY_LIMIT_EXCEEDED or 243 -NOT_ENOUGH_SPACE
Assert.assertTrue(e.getCode() == 241 || e.getCode() == 243);
}
}
}
Expand Down
Loading
Loading