diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseCache.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseCache.java index d96e7e79d..53ed65ccc 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseCache.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseCache.java @@ -51,7 +51,7 @@ static ClickHouseCache create(int capacity, long expireSeconds, Fun * @param type of the cache * @param clazz non-null class of the cache * @return inner cache object - * @throws NullPointerException + * @throws NullPointerException when {@code clazz} is null */ T unwrap(Class clazz); } diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseChecker.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseChecker.java index 533f333d1..2fe231a24 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseChecker.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseChecker.java @@ -198,6 +198,23 @@ public static boolean isNullOrBlank(CharSequence value) { return true; } + /** + * Checks if the given string is not {@code null}, empty or blank and throws a + * customized {@link IllegalArgumentException} if it is. + * + * @param value the string to check + * @param name name of the string + * @return the exact same string + * @throws IllegalArgumentException if the string is null, empty or blank + */ + public static String nonBlank(String value, String name) { + if (isNullOrBlank(value)) { + throw newException("%s cannot be null, empty or blank string", name); + } + + return value; + } + /** * Checks if the given string is neither {@code null} nor empty and throws a * customized {@link IllegalArgumentException} if it is. diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseClient.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseClient.java index 594bd96bd..ce5cc626d 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseClient.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseClient.java @@ -5,7 +5,6 @@ import java.io.InputStream; import java.io.OutputStream; import java.util.ArrayList; -import java.util.Collections; import java.util.LinkedList; import java.util.List; import java.util.Map; @@ -38,10 +37,6 @@ * implementation properly in runtime. */ public interface ClickHouseClient extends AutoCloseable { - /** - * Empty stringified parameters. - */ - static final List> EMPTY_STRINGIFIED_PARAMETERS = Collections.emptyList(); /** * Returns a builder for creating a new client. @@ -74,6 +69,7 @@ static ExecutorService getExecutorService() { * @param return type of the task * @param task non-null task * @return future object to get result + * @throws CompletionException when failed to complete the task */ static CompletableFuture submit(Callable task) { try { @@ -101,7 +97,10 @@ static CompletableFuture submit(Callable task) { * @param compression compression algorithm to use * @param file output file * @return future object to get result - * @throws IOException when failed to create the file or its parent directories + * @throws IllegalArgumentException if any of server, tableOrQuery, and output + * is null + * @throws IOException when failed to create the file or its parent + * directories */ static CompletableFuture dump(ClickHouseNode server, String tableOrQuery, ClickHouseFormat format, ClickHouseCompression compression, String file) throws IOException { @@ -120,6 +119,8 @@ static CompletableFuture dump(ClickHouseNode server, * @param output output stream, which will be closed automatically at the * end of the call * @return future object to get result + * @throws IllegalArgumentException if any of server, tableOrQuery, and output + * is null */ static CompletableFuture dump(ClickHouseNode server, String tableOrQuery, ClickHouseFormat format, ClickHouseCompression compression, OutputStream output) { @@ -148,8 +149,11 @@ static CompletableFuture dump(ClickHouseNode server, response.dump(output); return response.getSummary(); } - } catch (InterruptedException | ExecutionException e) { - throw ClickHouseExceptionSpecifier.specify(e, theServer.getHost(), theServer.getPort()); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw ClickHouseExceptionSpecifier.specify(e, theServer); + } catch (ExecutionException e) { + throw ClickHouseExceptionSpecifier.handle(e, theServer); } finally { try { output.close(); @@ -170,7 +174,8 @@ static CompletableFuture dump(ClickHouseNode server, * @param compression compression algorithm to use * @param file file to load * @return future object to get result - * @throws FileNotFoundException when file not found + * @throws IllegalArgumentException if any of server, table, and input is null + * @throws FileNotFoundException when file not found */ static CompletableFuture load(ClickHouseNode server, String table, ClickHouseFormat format, ClickHouseCompression compression, String file) throws FileNotFoundException { @@ -187,6 +192,7 @@ static CompletableFuture load(ClickHouseNode server, * @param compression compression algorithm to use * @param writer non-null custom writer to generate data * @return future object to get result + * @throws IllegalArgumentException if any of server, table, and writer is null */ static CompletableFuture load(ClickHouseNode server, String table, ClickHouseFormat format, ClickHouseCompression compression, ClickHouseWriter writer) { @@ -221,8 +227,11 @@ static CompletableFuture load(ClickHouseNode server, try (ClickHouseResponse response = future.get()) { return response.getSummary(); } - } catch (InterruptedException | ExecutionException e) { - throw ClickHouseExceptionSpecifier.specify(e, theServer.getHost(), theServer.getPort()); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw ClickHouseExceptionSpecifier.specify(e, theServer); + } catch (ExecutionException e) { + throw ClickHouseExceptionSpecifier.handle(e, theServer); } finally { if (input != null) { try { @@ -246,6 +255,7 @@ static CompletableFuture load(ClickHouseNode server, * @param input input stream, which will be closed automatically at the * end of the call * @return future object to get result + * @throws IllegalArgumentException if any of server, table, and input is null */ static CompletableFuture load(ClickHouseNode server, String table, ClickHouseFormat format, ClickHouseCompression compression, InputStream input) { @@ -264,8 +274,11 @@ static CompletableFuture load(ClickHouseNode server, ClickHouseResponse response = client.connect(theServer).write().table(table) .compression(theCompression).format(theFormat).data(input).execute().get()) { return response.getSummary(); - } catch (InterruptedException | ExecutionException e) { - throw ClickHouseExceptionSpecifier.specify(e, theServer.getHost(), theServer.getPort()); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw ClickHouseExceptionSpecifier.specify(e, theServer); + } catch (ExecutionException e) { + throw ClickHouseExceptionSpecifier.handle(e, theServer); } finally { try { input.close(); @@ -295,6 +308,7 @@ static ClickHouseClient newInstance(ClickHouseProtocol... preferredProtocols) { * @param sql non-null SQL query * @param more more SQL queries if any * @return list of {@link ClickHouseResponseSummary} one for each execution + * @throws IllegalArgumentException if server or sql is null */ static CompletableFuture> send(ClickHouseNode server, String sql, String... more) { if (server == null || sql == null) { @@ -342,6 +356,7 @@ static CompletableFuture> send(ClickHouseNode se * @param sql non-null SQL query * @param params non-null stringified parameters * @return list of {@link ClickHouseResponseSummary} one for each execution + * @throws IllegalArgumentException if any of server, sql, and params is null */ static CompletableFuture send(ClickHouseNode server, String sql, Map params) { @@ -373,6 +388,8 @@ static CompletableFuture send(ClickHouseNode server, * @param columns non-empty column list * @param params non-empty raw parameters * @return list of {@link ClickHouseResponseSummary} one for each execution + * @throws IllegalArgumentException if columns is null, empty or contains null + * column */ static CompletableFuture> send(ClickHouseNode server, String sql, List columns, Object[]... params) { @@ -399,6 +416,8 @@ static CompletableFuture> send(ClickHouseNode se * @param templates non-empty template objects to stringify parameters * @param params non-empty raw parameters * @return list of {@link ClickHouseResponseSummary} one for each execution + * @throws IllegalArgumentException if no named parameter in the query, or + * templates or params is null or empty */ static CompletableFuture> send(ClickHouseNode server, String sql, ClickHouseValue[] templates, Object[]... params) { @@ -455,6 +474,8 @@ static CompletableFuture> send(ClickHouseNode se * @param sql non-null SQL query * @param params non-null stringified parameters * @return list of {@link ClickHouseResponseSummary} one for each execution + * @throws IllegalArgumentException if any of server, sql, or params is null; or + * no named parameter in the query */ static CompletableFuture> send(ClickHouseNode server, String sql, String[][] params) { @@ -493,13 +514,13 @@ static CompletableFuture> send(ClickHouseNode se } /** - * Tests if the given server is healthy or not. Unlike other methods, it's a + * Tests if the given server is alive or not. Unlike other methods, it's a * synchronous call with minimum overhead(e.g. tiny buffer, no compression and * no deserialization etc). * * @param server server to test * @param timeout timeout in millisecond - * @return true if the server is healthy; false otherwise + * @return true if the server is alive; false otherwise */ static boolean test(ClickHouseNode server, int timeout) { if (server != null) { diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseImmediateFuture.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseImmediateFuture.java deleted file mode 100644 index f9f358cab..000000000 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseImmediateFuture.java +++ /dev/null @@ -1,60 +0,0 @@ -package com.clickhouse.client; - -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; - -/** - * Encapsulates a {@link java.util.concurrent.Callable} when - * {@link com.clickhouse.client.config.ClickHouseClientOption#ASYNC} is set to - * {@code false}. It's not cancellable and the task will be only executed when - * {@link #get()} is called. - * - * @deprecated - */ -@Deprecated -public final class ClickHouseImmediateFuture implements Future { - public static Future of(Callable task) { - return new ClickHouseImmediateFuture<>(task); - } - - private final Callable task; - - private ClickHouseImmediateFuture(Callable task) { - this.task = ClickHouseChecker.nonNull(task, "task"); - } - - @Override - public boolean cancel(boolean mayInterruptIfRunning) { - return false; - } - - @Override - public boolean isCancelled() { - return false; - } - - @Override - public boolean isDone() { - return true; - } - - @Override - public T get() throws InterruptedException, ExecutionException { - try { - return task.call(); - } catch (InterruptedException | ExecutionException e) { - throw e; - } catch (Exception e) { - throw new ExecutionException(e); - } - } - - @Override - public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { - ClickHouseChecker.nonNull(unit, "unit"); - return get(); - } -} diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseParameterizedQuery.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseParameterizedQuery.java index 1e07f957d..7ff5806a7 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseParameterizedQuery.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseParameterizedQuery.java @@ -1,5 +1,6 @@ package com.clickhouse.client; +import java.io.Serializable; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -9,12 +10,15 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Objects; /** * A parameterized query is a parsed query with named parameters being extracted * for substitution. */ -public final class ClickHouseParameterizedQuery { +public final class ClickHouseParameterizedQuery implements Serializable { + private static final long serialVersionUID = 8108887349618342152L; + /** * Substitute named parameters in given SQL. * @@ -97,11 +101,12 @@ public static ClickHouseParameterizedQuery of(String query) { private final String originalQuery; // 0 - from; 1 - to; 2 - parameter index(-1 means no parameter) private final List parts; + private int[] lastPart; private String[] names; private ClickHouseParameterizedQuery(String query) { - originalQuery = ClickHouseChecker.nonEmpty(query, "query"); + originalQuery = ClickHouseChecker.nonBlank(query, "query"); parts = new LinkedList<>(); lastPart = null; @@ -396,4 +401,30 @@ public List getQueryParts() { public boolean hasParameter() { return names.length > 0; } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + Arrays.hashCode(lastPart); + result = prime * result + Arrays.hashCode(names); + result = prime * result + ((originalQuery == null) ? 0 : originalQuery.hashCode()); + result = prime * result + ((parts == null) ? 0 : parts.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + ClickHouseParameterizedQuery other = (ClickHouseParameterizedQuery) obj; + return Arrays.equals(lastPart, other.lastPart) && Arrays.equals(names, other.names) + && Objects.equals(originalQuery, other.originalQuery) && ((parts.isEmpty() && other.parts.isEmpty()) + || Arrays.deepEquals(parts.toArray(new int[0][]), other.parts.toArray(new int[0][]))); + } } diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseRequest.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseRequest.java index ade618e6f..9e2daa1d4 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseRequest.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseRequest.java @@ -42,9 +42,6 @@ protected Mutation(ClickHouseRequest request, boolean sealed) { this.options.putAll(request.options); this.settings.putAll(request.settings); - this.namedParameters.putAll(request.namedParameters); - - this.queryId = request.queryId; this.sessionId = request.sessionId; } @@ -137,7 +134,7 @@ public Mutation table(String table, String queryId) { this.queryId = queryId; - String sql = "INSERT INTO " + ClickHouseChecker.nonEmpty(table, "table"); + String sql = "INSERT INTO " + ClickHouseChecker.nonBlank(table, "table"); if (!sql.equals(this.sql)) { this.sql = sql; this.preparedQuery = null; @@ -154,6 +151,7 @@ public Mutation seal() { if (!isSealed()) { // no idea which node we'll connect to until now req = new Mutation(this, true); + req.externalTables.addAll(externalTables); req.options.putAll(options); req.settings.putAll(settings); @@ -189,10 +187,10 @@ public Mutation seal() { protected String queryId; protected String sessionId; protected String sql; + protected ClickHouseParameterizedQuery preparedQuery; // cache protected transient ClickHouseConfig config; - protected transient ClickHouseParameterizedQuery preparedQuery; protected transient List statements; @SuppressWarnings("unchecked") @@ -255,6 +253,25 @@ protected void resetCache() { } } + /** + * Creates a copy of this request object. + * + * @return copy of this request + */ + public ClickHouseRequest copy() { + ClickHouseRequest req = new ClickHouseRequest<>(getClient(), server, false); + req.externalTables.addAll(externalTables); + req.options.putAll(options); + req.settings.putAll(settings); + req.namedParameters.putAll(namedParameters); + req.input = input; + req.queryId = queryId; + req.sessionId = sessionId; + req.sql = sql; + req.preparedQuery = preparedQuery; + return req; + } + /** * Checks if the request is sealed(immutable). * @@ -825,7 +842,7 @@ public SelfT query(ClickHouseParameterizedQuery query, String queryId) { public SelfT query(String sql, String queryId) { checkSealed(); - if (!ClickHouseChecker.nonEmpty(sql, "sql").equals(this.sql)) { + if (!ClickHouseChecker.nonBlank(sql, "sql").equals(this.sql)) { this.sql = sql; this.preparedQuery = null; resetCache(); @@ -944,7 +961,7 @@ public SelfT session(String sessionId, Boolean check, Integer timeout) { public SelfT set(String setting, Serializable value) { checkSealed(); - Serializable oldValue = settings.put(ClickHouseChecker.nonEmpty(setting, "setting"), + Serializable oldValue = settings.put(ClickHouseChecker.nonBlank(setting, "setting"), ClickHouseChecker.nonNull(value, "value")); if (oldValue == null || !oldValue.equals(value)) { resetCache(); @@ -988,7 +1005,7 @@ public SelfT table(String table) { * @return the request itself */ public SelfT table(String table, String queryId) { - return query("SELECT * FROM " + ClickHouseChecker.nonEmpty(table, "table"), queryId); + return query("SELECT * FROM " + ClickHouseChecker.nonBlank(table, "table"), queryId); } /** @@ -1002,7 +1019,7 @@ public SelfT use(String database) { checkSealed(); Object oldValue = options.put(ClickHouseClientOption.DATABASE, - ClickHouseChecker.nonEmpty(database, "database")); + ClickHouseChecker.nonBlank(database, "database")); if (oldValue == null || !oldValue.equals(database)) { resetCache(); } @@ -1083,7 +1100,7 @@ public SelfT removeOption(ClickHouseConfigOption option) { public SelfT removeSetting(String setting) { checkSealed(); - if (settings.remove(ClickHouseChecker.nonEmpty(setting, "setting")) != null) { + if (settings.remove(ClickHouseChecker.nonBlank(setting, "setting")) != null) { resetCache(); } diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseResponse.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseResponse.java index 8d5be411f..49ae83aaa 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseResponse.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseResponse.java @@ -86,7 +86,7 @@ protected ClickHouseResponse(ClickHouseConfig config, ClickHouseNode server, Map } } - throw ClickHouseExceptionSpecifier.specify(e, server.getHost(), server.getPort()); + throw ClickHouseExceptionSpecifier.specify(e, server); } } @@ -98,7 +98,7 @@ protected void throwErrorIfAny() throws ClickHouseException { if (error instanceof ClickHouseException) { throw (ClickHouseException) error; } else { - throw ClickHouseExceptionSpecifier.specify(error, server.getHost(), server.getPort()); + throw ClickHouseExceptionSpecifier.specify(error, server); } } @@ -181,7 +181,7 @@ public void dump(OutputStream output) throws ClickHouseException { output.write(buffer, 0, counter); } } catch (IOException e) { - throw ClickHouseExceptionSpecifier.specify(e, server.getHost(), server.getPort()); + throw ClickHouseExceptionSpecifier.specify(e, server); } } diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseUtils.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseUtils.java index 17e2da6a2..5fe3029ec 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseUtils.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseUtils.java @@ -295,7 +295,7 @@ public static T getService(Class serviceInterface, Supplier * @throws FileNotFoundException when the file does not exists */ public static InputStream getFileInputStream(String file) throws FileNotFoundException { - Path path = Paths.get(ClickHouseChecker.nonEmpty(file, "file")); + Path path = Paths.get(ClickHouseChecker.nonBlank(file, "file")); StringBuilder builder = new StringBuilder(); InputStream in = null; @@ -332,7 +332,7 @@ public static InputStream getFileInputStream(String file) throws FileNotFoundExc * @throws IOException when failed to create directories and/or file */ public static OutputStream getFileOutputStream(String file) throws IOException { - Path path = Paths.get(ClickHouseChecker.nonEmpty(file, "file")); + Path path = Paths.get(ClickHouseChecker.nonBlank(file, "file")); if (Files.notExists(path)) { Files.createDirectories(path.getParent()); diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseExternalTable.java b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseExternalTable.java index 52f45e5d3..7e0457978 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseExternalTable.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseExternalTable.java @@ -22,17 +22,24 @@ protected Builder() { columns = new LinkedList<>(); } - public Builder withName(String name) { + public Builder name(String name) { this.name = name; return this; } - public Builder withContent(InputStream content) { + public Builder content(InputStream content) { this.content = content; return this; } - public Builder withFormat(ClickHouseFormat format) { + public Builder format(String format) { + if (!ClickHouseChecker.isNullOrBlank(format)) { + this.format = ClickHouseFormat.valueOf(format); + } + return this; + } + + public Builder format(ClickHouseFormat format) { this.format = format; return this; } @@ -59,7 +66,11 @@ public Builder removeColumn(ClickHouseColumn column) { return this; } - public Builder withColumns(Collection columns) { + public Builder columns(String columns) { + return !ClickHouseChecker.isNullOrBlank(columns) ? columns(ClickHouseColumn.parse(columns)) : this; + } + + public Builder columns(Collection columns) { if (columns != null) { for (ClickHouseColumn c : columns) { this.columns.add(c); diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/exception/ClickHouseErrorCode.java b/clickhouse-client/src/main/java/com/clickhouse/client/exception/ClickHouseErrorCode.java index 82f1bdb8f..34acdf0ff 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/exception/ClickHouseErrorCode.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/exception/ClickHouseErrorCode.java @@ -608,7 +608,11 @@ public enum ClickHouseErrorCode { } public static ClickHouseErrorCode fromCode(Integer code) { - return byCodes.get(code); + return fromCodeOrDefault(code, null); + } + + public static ClickHouseErrorCode fromCodeOrDefault(Integer code, ClickHouseErrorCode defaultErrorCode) { + return byCodes.getOrDefault(code, defaultErrorCode); } @Override diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/exception/ClickHouseException.java b/clickhouse-client/src/main/java/com/clickhouse/client/exception/ClickHouseException.java index e42836ef8..a471b258c 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/exception/ClickHouseException.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/exception/ClickHouseException.java @@ -1,5 +1,10 @@ package com.clickhouse.client.exception; +import com.clickhouse.client.ClickHouseNode; + +/** + * Exception thrown from ClickHouse server. + */ public class ClickHouseException extends Exception { /** * Generated ID. @@ -8,7 +13,15 @@ public class ClickHouseException extends Exception { private final int errorCode; - private static String buildErrorMessage(String message, int code, String host, int port, Throwable cause) { + private static String buildErrorMessage(ClickHouseErrorCode error, ClickHouseNode server, Throwable cause) { + if (error == null) { + error = ClickHouseErrorCode.UNKNOWN_EXCEPTION; + } + + return buildErrorMessage(error.name(), error.code, server, cause); + } + + private static String buildErrorMessage(String message, int code, ClickHouseNode server, Throwable cause) { StringBuilder builder = new StringBuilder(); builder.append("ClickHouse exception, "); @@ -18,11 +31,8 @@ private static String buildErrorMessage(String message, int code, String host, i builder.append(" code: ").append(code); } - if (host != null) { - builder.append(", host: ").append(host).append(", port: ").append(port); - } + builder.append(", server: ").append(server).append(';'); - builder.append(';'); if (cause != null) { builder.append(' ').append(cause.getMessage()); } @@ -30,24 +40,38 @@ private static String buildErrorMessage(String message, int code, String host, i return builder.toString(); } - public ClickHouseException(int code, Throwable cause, String host, int port) { - super(buildErrorMessage(null, code, host, port, cause), cause); + public ClickHouseException(int code, Throwable cause, ClickHouseNode server) { + super(buildErrorMessage( + ClickHouseErrorCode.fromCodeOrDefault(code, ClickHouseErrorCode.UNKNOWN_EXCEPTION).name(), code, server, + cause), cause); errorCode = code; } - public ClickHouseException(int code, String message, Throwable cause, String host, int port) { - super(buildErrorMessage(message, code, host, port, cause), cause); + public ClickHouseException(int code, String message, Throwable cause, ClickHouseNode server) { + super(buildErrorMessage( + message != null ? message + : ClickHouseErrorCode.fromCodeOrDefault(code, ClickHouseErrorCode.UNKNOWN_EXCEPTION).name(), + code, server, cause), cause); errorCode = code; } public ClickHouseException(int code, String message, Throwable cause) { - super(buildErrorMessage(message, code, null, 0, cause), cause); + super(buildErrorMessage( + message != null ? message + : ClickHouseErrorCode.fromCodeOrDefault(code, ClickHouseErrorCode.UNKNOWN_EXCEPTION).name(), + code, null, cause), cause); errorCode = code; } + public ClickHouseException(ClickHouseErrorCode error, Throwable cause, ClickHouseNode server) { + super(buildErrorMessage(error, server, cause), cause); + + errorCode = error != null ? error.code : ClickHouseErrorCode.UNKNOWN_EXCEPTION.code; + } + public int getErrorCode() { return errorCode; } diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/exception/ClickHouseExceptionSpecifier.java b/clickhouse-client/src/main/java/com/clickhouse/client/exception/ClickHouseExceptionSpecifier.java index f50ac4818..78f0f4eab 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/exception/ClickHouseExceptionSpecifier.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/exception/ClickHouseExceptionSpecifier.java @@ -2,7 +2,10 @@ import java.net.ConnectException; import java.net.SocketTimeoutException; +import java.util.concurrent.ExecutionException; + import com.clickhouse.client.ClickHouseChecker; +import com.clickhouse.client.ClickHouseNode; import com.clickhouse.client.logging.Logger; import com.clickhouse.client.logging.LoggerFactory; @@ -17,25 +20,39 @@ public final class ClickHouseExceptionSpecifier { private ClickHouseExceptionSpecifier() { } - public static ClickHouseException specify(Throwable cause, String host, int port) { - return specify(cause != null ? cause.getMessage() : null, cause, host, port); + /** + * Handles ExecutionException. + * + * @param e ExecutionException + * @param server server + */ + public static ClickHouseException handle(ExecutionException e, ClickHouseNode server) { + Throwable cause = e.getCause(); + if (cause == null) { + cause = e; + } + return ClickHouseExceptionSpecifier.specify(cause, server); + } + + public static ClickHouseException specify(Throwable cause, ClickHouseNode server) { + return specify(cause != null ? cause.getMessage() : null, cause, server); } - public static ClickHouseException specify(String clickHouseMessage, String host, int port) { - return specify(clickHouseMessage, null, host, port); + public static ClickHouseException specify(String clickHouseMessage, ClickHouseNode server) { + return specify(clickHouseMessage, null, server); } public static ClickHouseException specify(String clickHouseMessage) { - return specify(clickHouseMessage, "unknown", -1); + return specify(clickHouseMessage, null); } /** * Here we expect the ClickHouse error message to be of the following format: * "Code: 10, e.displayText() = DB::Exception: ...". */ - private static ClickHouseException specify(String clickHouseMessage, Throwable cause, String host, int port) { + private static ClickHouseException specify(String clickHouseMessage, Throwable cause, ClickHouseNode server) { if (ClickHouseChecker.isNullOrEmpty(clickHouseMessage) && cause != null) { - return getException(cause, host, port); + return getException(cause, server); } try { @@ -49,15 +66,15 @@ private static ClickHouseException specify(String clickHouseMessage, Throwable c // ошибку в изначальном виде все-таки укажем Throwable messageHolder = cause != null ? cause : new Throwable(clickHouseMessage); if (code == -1) { - return getException(messageHolder, host, port); + return getException(messageHolder, server); } - return new ClickHouseException(code, messageHolder, host, port); + return new ClickHouseException(code, messageHolder, server); } catch (Exception e) { log.error( "Unsupported ClickHouse error format, please fix ClickHouseExceptionSpecifier, message: {}, error: {}", clickHouseMessage, e.getMessage()); - return new ClickHouseUnknownException(clickHouseMessage, cause, host, port); + return new ClickHouseUnknownException(clickHouseMessage, cause, server); } } @@ -76,19 +93,19 @@ private static int getErrorCode(String errorMessage) { } } - private static ClickHouseException getException(Throwable cause, String host, int port) { + private static ClickHouseException getException(Throwable cause, ClickHouseNode server) { if (cause instanceof SocketTimeoutException) // if we've got SocketTimeoutException, we'll say that the query is not good. // This is not the same as SOCKET_TIMEOUT of clickhouse // but it actually could be a failing ClickHouse { - return new ClickHouseException(ClickHouseErrorCode.TIMEOUT_EXCEEDED.code, cause, host, port); + return new ClickHouseException(ClickHouseErrorCode.TIMEOUT_EXCEEDED.code, cause, server); } else if (cause instanceof ConnectException) // couldn't connect to ClickHouse during connectTimeout { - return new ClickHouseException(ClickHouseErrorCode.NETWORK_ERROR.code, cause, host, port); + return new ClickHouseException(ClickHouseErrorCode.NETWORK_ERROR.code, cause, server); } else { - return new ClickHouseUnknownException(cause, host, port); + return new ClickHouseUnknownException(cause, server); } } diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/exception/ClickHouseUnknownException.java b/clickhouse-client/src/main/java/com/clickhouse/client/exception/ClickHouseUnknownException.java index 21e548227..cab313a61 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/exception/ClickHouseUnknownException.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/exception/ClickHouseUnknownException.java @@ -1,24 +1,26 @@ package com.clickhouse.client.exception; +import com.clickhouse.client.ClickHouseNode; + public class ClickHouseUnknownException extends ClickHouseException { /** * Generated ID. */ private static final long serialVersionUID = -1724790228244438601L; - public ClickHouseUnknownException(Throwable cause, String host, int port) { - super(ClickHouseErrorCode.UNKNOWN_EXCEPTION.code, cause, host, port); + public ClickHouseUnknownException(Throwable cause, ClickHouseNode server) { + super(ClickHouseErrorCode.UNKNOWN_EXCEPTION.code, cause, server); } - public ClickHouseUnknownException(String message, Throwable cause, String host, int port) { - super(ClickHouseErrorCode.UNKNOWN_EXCEPTION.code, message, cause, host, port); + public ClickHouseUnknownException(String message, Throwable cause, ClickHouseNode server) { + super(ClickHouseErrorCode.UNKNOWN_EXCEPTION.code, message, cause, server); } public ClickHouseUnknownException(String message, Throwable cause) { super(ClickHouseErrorCode.UNKNOWN_EXCEPTION.code, message, cause); } - public ClickHouseUnknownException(Integer code, Throwable cause, String host, int port) { - super(code, cause, host, port); + public ClickHouseUnknownException(Integer code, Throwable cause, ClickHouseNode server) { + super(code, cause, server); } } diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseCheckerTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseCheckerTest.java index 0cc051d33..085bfddd8 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseCheckerTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseCheckerTest.java @@ -46,6 +46,15 @@ public void testIsNullOrBlank() { Assert.assertTrue(ClickHouseChecker.isNullOrBlank(" \t\r\n ")); } + @Test(groups = { "unit" }) + public void testNonBlank() { + Assert.assertEquals(ClickHouseChecker.nonBlank(" 1", "value"), " 1"); + + Assert.assertThrows(IllegalArgumentException.class, () -> ClickHouseChecker.nonBlank(null, null)); + Assert.assertThrows(IllegalArgumentException.class, () -> ClickHouseChecker.nonBlank("", "")); + Assert.assertThrows(IllegalArgumentException.class, () -> ClickHouseChecker.nonBlank(" ", "")); + } + @Test(groups = { "unit" }) public void testNonEmpty() { Assert.assertEquals(ClickHouseChecker.nonEmpty(" ", "value"), " "); diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseRequestTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseRequestTest.java index 09ec92388..4961e0f95 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseRequestTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseRequestTest.java @@ -15,6 +15,7 @@ import com.clickhouse.client.data.ClickHouseBigIntegerValue; import com.clickhouse.client.data.ClickHouseByteValue; import com.clickhouse.client.data.ClickHouseDateTimeValue; +import com.clickhouse.client.data.ClickHouseExternalTable; import com.clickhouse.client.data.ClickHouseFloatValue; import com.clickhouse.client.data.ClickHouseIntegerValue; import com.clickhouse.client.data.ClickHouseStringValue; @@ -62,6 +63,56 @@ public void testBuild() { Assert.assertEquals(m.getStatements().get(0), sql); } + @Test(groups = { "unit" }) + public void testCopy() { + ClickHouseRequest request = ClickHouseClient.newInstance().connect(ClickHouseNode.builder().build()); + request.compression(ClickHouseCompression.LZ4); + request.external(ClickHouseExternalTable.builder().content(new ByteArrayInputStream(new byte[0])).build()); + request.format(ClickHouseFormat.Avro); + request.table("table1", "query_id1"); + request.query("select :a", UUID.randomUUID().toString()); + request.params("a"); + request.session(UUID.randomUUID().toString(), true, 120); + request.set("key", "value"); + request.use("db1"); + + ClickHouseRequest copy = request.copy(); + Assert.assertFalse(copy.isSealed(), "Should NOT be sealed"); + Assert.assertFalse(copy == request, "Should be two different instances"); + Assert.assertEquals(copy.namedParameters, request.namedParameters); + Assert.assertEquals(copy.options, request.options); + Assert.assertEquals(copy.queryId, request.queryId); + Assert.assertEquals(copy.sessionId, request.sessionId); + Assert.assertEquals(copy.sql, request.sql); + Assert.assertEquals(copy.getPreparedQuery(), request.getPreparedQuery()); + + copy = copy.write(); + Assert.assertFalse(copy.isSealed(), "Should NOT be sealed"); + Assert.assertFalse(copy == request, "Should be two different instances"); + Assert.assertTrue(copy.namedParameters.isEmpty(), "Named parameters should be empty"); + Assert.assertEquals(copy.options, request.options); + Assert.assertNull(copy.queryId, "Query ID should be null"); + Assert.assertEquals(copy.sessionId, request.sessionId); + Assert.assertNull(copy.sql, "SQL should be null"); + + ClickHouseRequest newCopy = copy; + Assert.assertThrows(IllegalArgumentException.class, () -> newCopy.getPreparedQuery()); + + copy.external(ClickHouseExternalTable.builder().content(new ByteArrayInputStream(new byte[0])).build()); + copy.table("table1", "query_id1"); + copy.query("select :a", request.queryId); + copy.params("a"); + + Assert.assertFalse(copy.isSealed(), "Should NOT be sealed"); + Assert.assertFalse(copy == request, "Should be two different instances"); + Assert.assertEquals(copy.namedParameters, request.namedParameters); + Assert.assertEquals(copy.options, request.options); + Assert.assertEquals(copy.queryId, request.queryId); + Assert.assertEquals(copy.sessionId, request.sessionId); + Assert.assertEquals(copy.sql, request.sql); + Assert.assertEquals(copy.getPreparedQuery(), request.getPreparedQuery()); + } + @Test(groups = { "unit" }) public void testFormat() { ClickHouseRequest request = ClickHouseClient.newInstance().connect(ClickHouseNode.builder().build()); @@ -117,6 +168,32 @@ public void testParams() { "select '' as one, '2012-12-12 12:23:34.56789' as two, * from my_table where key='key' and arr[1] in numbers(10)"); } + @Test(groups = { "unit" }) + public void testSeal() { + ClickHouseRequest request = ClickHouseClient.newInstance().connect(ClickHouseNode.builder().build()); + request.compression(ClickHouseCompression.LZ4); + request.external(ClickHouseExternalTable.builder().content(new ByteArrayInputStream(new byte[0])).build()); + request.format(ClickHouseFormat.Avro); + request.table("table1", "query_id1"); + request.query("select :a", UUID.randomUUID().toString()); + request.params("a"); + request.session(UUID.randomUUID().toString(), true, 120); + request.set("key", "value"); + request.use("db1"); + + ClickHouseRequest sealed = request.seal(); + Assert.assertTrue(sealed.isSealed(), "Should be sealed"); + Assert.assertFalse(sealed == request, "Should be two different instances"); + Assert.assertEquals(sealed.namedParameters, request.namedParameters); + Assert.assertEquals(sealed.options, request.options); + Assert.assertEquals(sealed.queryId, request.queryId); + Assert.assertEquals(sealed.sessionId, request.sessionId); + Assert.assertEquals(sealed.sql, request.sql); + Assert.assertEquals(sealed.getPreparedQuery(), request.getPreparedQuery()); + + Assert.assertThrows(IllegalStateException.class, () -> sealed.write()); + } + @Test(groups = { "unit" }) public void testSession() { String sessionId = UUID.randomUUID().toString(); diff --git a/clickhouse-grpc-client/src/test/java/com/clickhouse/client/grpc/ClickHouseGrpcClientTest.java b/clickhouse-grpc-client/src/test/java/com/clickhouse/client/grpc/ClickHouseGrpcClientTest.java index 84b064209..a75fc3acc 100644 --- a/clickhouse-grpc-client/src/test/java/com/clickhouse/client/grpc/ClickHouseGrpcClientTest.java +++ b/clickhouse-grpc-client/src/test/java/com/clickhouse/client/grpc/ClickHouseGrpcClientTest.java @@ -532,8 +532,8 @@ public void testQueryWithMultipleExternalTables() throws Exception { } String dnExtString = String.join("\n", valueList); InputStream inputStream = new ByteArrayInputStream(dnExtString.getBytes(Charset.forName("UTF-8"))); - ClickHouseExternalTable extTable = ClickHouseExternalTable.builder().withName("L" + i) - .withContent(inputStream).addColumn("Cb", "String").build(); + ClickHouseExternalTable extTable = ClickHouseExternalTable.builder().name("L" + i).content(inputStream) + .addColumn("Cb", "String").build(); extTableList.add(extTable); }