diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index ece750cb9..f3ad7fc05 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -37,12 +37,10 @@ jobs: strategy: matrix: # most recent LTS releases as well as latest stable builds - clickhouse: ["21.3", "21.8", "latest"] + clickhouse: ["21.8", "22.3", "latest"] # http2 here represents http protocol + JDK HttpClient(http_connection_provider=HTTP_CLIENT) protocol: ["http", "http2", "grpc"] exclude: - - clickhouse: "21.3" - protocol: grpc - clickhouse: "21.8" protocol: grpc fail-fast: false diff --git a/README.md b/README.md index 751582401..6b0f9dc58 100644 --- a/README.md +++ b/README.md @@ -51,6 +51,9 @@ Note: in general, the new driver(v0.3.2) is a few times faster with less memory | | UUID | :white_check_mark: | | | High Availability | Load Balancing | :white_check_mark: | supported since 0.3.2-patch10 | | | Failover | :white_check_mark: | supported since 0.3.2-patch10 | +| Transaction | Transaction | :white_check_mark: | supported since 0.3.2-patch11, use ClickHouse 22.7+ for native implicit transaction support | +| | Savepoint | :x: | | +| | XAConnection | :x: | | ## Examples @@ -178,7 +181,7 @@ It's time consuming to run all benchmarks against all drivers using different pa ## Testing -By default, docker container will be created automatically during integration test. You can pass system property like `-DclickhouseVersion=21.8` to specify version of ClickHouse. +By default, docker container will be created automatically during integration test. You can pass system property like `-DclickhouseVersion=22.3` to specify version of ClickHouse. In the case you prefer to test against an existing server, please follow instructions below: diff --git a/clickhouse-cli-client/src/main/java/com/clickhouse/client/cli/ClickHouseCommandLine.java b/clickhouse-cli-client/src/main/java/com/clickhouse/client/cli/ClickHouseCommandLine.java index 4595065f5..2f36c91e4 100644 --- a/clickhouse-cli-client/src/main/java/com/clickhouse/client/cli/ClickHouseCommandLine.java +++ b/clickhouse-cli-client/src/main/java/com/clickhouse/client/cli/ClickHouseCommandLine.java @@ -29,6 +29,7 @@ import com.clickhouse.client.ClickHouseChecker; import com.clickhouse.client.ClickHouseClient; +import com.clickhouse.client.ClickHouseCompression; import com.clickhouse.client.ClickHouseConfig; import com.clickhouse.client.ClickHouseCredentials; import com.clickhouse.client.ClickHouseFile; @@ -136,8 +137,9 @@ static void dockerCommand(ClickHouseConfig config, String hostDir, String contai commands.add(DEFAULT_CLICKHOUSE_CLI_PATH); } - static Process startProcess(ClickHouseNode server, ClickHouseRequest request) { + static Process startProcess(ClickHouseRequest request) { final ClickHouseConfig config = request.getConfig(); + final ClickHouseNode server = request.getServer(); final int timeout = config.getSocketTimeout(); String hostDir = (String) config.getOption(ClickHouseCommandLineOption.CLI_WORK_DIRECTORY); @@ -196,7 +198,7 @@ static Process startProcess(ClickHouseNode server, ClickHouseRequest request) if (!ClickHouseChecker.isNullOrBlank(str)) { commands.add("--query_id=".concat(str)); } - commands.add("--query=".concat(request.getStatements(false).get(0))); + commands.add("--query=".concat(str = request.getStatements(false).get(0))); for (ClickHouseExternalTable table : request.getExternalTables()) { ClickHouseFile tableFile = table.getFile(); @@ -331,30 +333,36 @@ static Process startProcess(ClickHouseNode server, ClickHouseRequest request) } } - private final ClickHouseNode server; private final ClickHouseRequest request; private final Process process; private String error; - public ClickHouseCommandLine(ClickHouseNode server, ClickHouseRequest request) { - this.server = server; + public ClickHouseCommandLine(ClickHouseRequest request) { this.request = request; - this.process = startProcess(server, request); + this.process = startProcess(request); this.error = null; } public ClickHouseInputStream getInputStream() throws IOException { ClickHouseOutputStream out = request.getOutputStream().orElse(null); + Runnable postCloseAction = () -> { + IOException exp = getError(); + if (exp != null) { + throw new UncheckedIOException(exp); + } + }; if (out != null && !out.getUnderlyingFile().isAvailable()) { try (OutputStream o = out) { ClickHouseInputStream.pipe(process.getInputStream(), o, request.getConfig().getWriteBufferSize()); } - return ClickHouseInputStream.empty(); + return ClickHouseInputStream.wrap(null, ClickHouseInputStream.empty(), + request.getConfig().getReadBufferSize(), postCloseAction, ClickHouseCompression.NONE, 0); } else { - return ClickHouseInputStream.of(process.getInputStream(), request.getConfig().getReadBufferSize()); + return ClickHouseInputStream.of(process.getInputStream(), request.getConfig().getReadBufferSize(), + postCloseAction); } } diff --git a/clickhouse-cli-client/src/main/java/com/clickhouse/client/cli/ClickHouseCommandLineClient.java b/clickhouse-cli-client/src/main/java/com/clickhouse/client/cli/ClickHouseCommandLineClient.java index 030976de4..f1ae44f62 100644 --- a/clickhouse-cli-client/src/main/java/com/clickhouse/client/cli/ClickHouseCommandLineClient.java +++ b/clickhouse-cli-client/src/main/java/com/clickhouse/client/cli/ClickHouseCommandLineClient.java @@ -45,7 +45,7 @@ protected ClickHouseCommandLine newConnection(ClickHouseCommandLine conn, ClickH closeConnection(conn, false); } - return new ClickHouseCommandLine(server, request); + return new ClickHouseCommandLine(request); } @Override diff --git a/clickhouse-cli-client/src/main/java/com/clickhouse/client/cli/ClickHouseCommandLineResponse.java b/clickhouse-cli-client/src/main/java/com/clickhouse/client/cli/ClickHouseCommandLineResponse.java index 2c28d7c8d..659daeebe 100644 --- a/clickhouse-cli-client/src/main/java/com/clickhouse/client/cli/ClickHouseCommandLineResponse.java +++ b/clickhouse-cli-client/src/main/java/com/clickhouse/client/cli/ClickHouseCommandLineResponse.java @@ -12,6 +12,7 @@ public class ClickHouseCommandLineResponse extends ClickHouseStreamResponse { protected ClickHouseCommandLineResponse(ClickHouseConfig config, ClickHouseCommandLine cli) throws IOException { super(config, cli.getInputStream(), null, null, ClickHouseResponseSummary.EMPTY); + this.cli = cli; if (this.input.available() < 1) { IOException exp = cli.getError(); @@ -19,8 +20,6 @@ protected ClickHouseCommandLineResponse(ClickHouseConfig config, ClickHouseComma throw exp; } } - - this.cli = cli; } @Override diff --git a/clickhouse-cli-client/src/test/java/com/clickhouse/client/cli/ClickHouseCommandLineClientTest.java b/clickhouse-cli-client/src/test/java/com/clickhouse/client/cli/ClickHouseCommandLineClientTest.java index 8a41bc7d2..b12cc1455 100644 --- a/clickhouse-cli-client/src/test/java/com/clickhouse/client/cli/ClickHouseCommandLineClientTest.java +++ b/clickhouse-cli-client/src/test/java/com/clickhouse/client/cli/ClickHouseCommandLineClientTest.java @@ -53,13 +53,6 @@ public void testCustomLoad() throws Exception { throw new SkipException("Skip due to time out error"); } - @Test(groups = { "integration" }) - @Override - public void testErrorDuringQuery() throws Exception { - throw new SkipException( - "Skip due to incomplete implementation(needs to consider ErrorOutputStream in deserialization as well)"); - } - @Test(groups = { "integration" }) @Override public void testLoadRawData() throws Exception { @@ -92,6 +85,12 @@ public void testReadWriteGeoTypes() { throw new SkipException("Skip due to session is not supported"); } + @Test(groups = { "integration" }) + @Override + public void testSessionLock() { + throw new SkipException("Skip due to session is not supported"); + } + @Test(groups = { "integration" }) @Override public void testTempTable() { 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 a608e7689..6c62235b5 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseClient.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseClient.java @@ -1,5 +1,6 @@ package com.clickhouse.client; +import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.io.UncheckedIOException; @@ -16,6 +17,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.function.Function; import com.clickhouse.client.config.ClickHouseBufferingMode; @@ -345,7 +347,7 @@ static CompletableFuture dump(ClickHouseNode server, } finally { try { output.close(); - } catch (Exception e) { + } catch (IOException e) { // ignore } } @@ -438,7 +440,8 @@ static CompletableFuture load(ClickHouseNode server, stream.close(); } // wait until write & read acomplished - try (ClickHouseResponse response = future.get()) { + try (ClickHouseResponse response = future.get(client.getConfig().getSocketTimeout(), + TimeUnit.MILLISECONDS)) { return response.getSummary(); } } catch (InterruptedException e) { @@ -446,13 +449,13 @@ static CompletableFuture load(ClickHouseNode server, throw ClickHouseException.forCancellation(e, theServer); } catch (CancellationException e) { throw ClickHouseException.forCancellation(e, theServer); - } catch (ExecutionException e) { + } catch (ExecutionException | TimeoutException e) { throw ClickHouseException.of(e, theServer); } finally { if (input != null) { try { input.close(); - } catch (Exception e) { + } catch (IOException e) { // ignore } } @@ -491,7 +494,7 @@ static CompletableFuture load(ClickHouseNode server, } finally { try { input.close(); - } catch (Exception e) { + } catch (IOException e) { // ignore } } @@ -546,7 +549,7 @@ static CompletableFuture> send(ClickHouseNode se .option(ClickHouseClientOption.ASYNC, false).build()) { ClickHouseRequest request = client.connect(theServer).format(ClickHouseFormat.RowBinary); if ((boolean) ClickHouseDefaults.AUTO_SESSION.getEffectiveDefaultValue() && queries.size() > 1) { - request.session(UUID.randomUUID().toString(), false); + request.session(request.getManager().createSessionId(), false); } for (String query : queries) { try (ClickHouseResponse resp = request.query(query).executeAndWait()) { @@ -818,13 +821,13 @@ default ClickHouseResponse executeAndWait(ClickHouseRequest request) throws C final ClickHouseRequest sealedRequest = request.seal(); try { - return execute(sealedRequest).get(); + return execute(sealedRequest).get(sealedRequest.getConfig().getSocketTimeout(), TimeUnit.MILLISECONDS); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw ClickHouseException.forCancellation(e, sealedRequest.getServer()); } catch (CancellationException e) { throw ClickHouseException.forCancellation(e, sealedRequest.getServer()); - } catch (CompletionException | ExecutionException | UncheckedIOException e) { + } catch (CompletionException | ExecutionException | TimeoutException | UncheckedIOException e) { Throwable cause = e.getCause(); if (cause == null) { cause = e; diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseClientBuilder.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseClientBuilder.java index 8e678e38e..bd094ce6d 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseClientBuilder.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseClientBuilder.java @@ -15,6 +15,8 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicReference; import com.clickhouse.client.config.ClickHouseOption; @@ -80,6 +82,10 @@ public boolean ping(ClickHouseNode server, int timeout) { static final class Agent implements ClickHouseClient { private static final Logger log = LoggerFactory.getLogger(Agent.class); + private static final long INITIAL_REPEAT_DELAY = 100; + private static final long MAX_REPEAT_DELAY = 1000; + private static final long REPEAT_DELAY_BACKOFF = 100; + private final AtomicReference client; Agent(ClickHouseClient client, ClickHouseConfig config) { @@ -110,6 +116,7 @@ ClickHouseResponse failover(ClickHouseRequest sealedRequest, ClickHouseExcept ClickHouseNode current = sealedRequest.getServer(); ClickHouseNodeManager manager = current.manager.get(); if (manager == null) { + log.debug("Cancel failover for unmanaged node: %s", current); break; } ClickHouseNode next = manager.suggestNode(current, exception); @@ -118,8 +125,10 @@ ClickHouseResponse failover(ClickHouseRequest sealedRequest, ClickHouseExcept break; } current.update(Status.FAULTY); - next = sealedRequest.changeServer(current, next); - if (next == current) { + if (sealedRequest.isTransactional()) { + log.debug("Cancel failover for transactional context: %s", sealedRequest.getTransaction()); + break; + } else if ((next = sealedRequest.changeServer(current, next)) == current) { log.debug("Cancel failover for no alternative of %s", current); break; } @@ -162,6 +171,59 @@ ClickHouseResponse failover(ClickHouseRequest sealedRequest, ClickHouseExcept throw new CompletionException(exception); } + /** + * Repeats sending same request until success, timed out or running into a + * different error. + * + * @param sealedRequest non-null sealed request + * @param exception non-null exception to start with + * @param timeout timeout in milliseconds, zero or negative numbers means + * no repeat + * @return non-null response + * @throws CompletionException when error occurred or timed out + */ + ClickHouseResponse repeat(ClickHouseRequest sealedRequest, ClickHouseException exception, long timeout) { + if (timeout > 0) { + final int errorCode = exception.getErrorCode(); + final long startTime = System.currentTimeMillis(); + + long delay = INITIAL_REPEAT_DELAY; + long elapsed = 0L; + int count = 1; + while (true) { + log.info("Repeating #%d (delay=%d, elapsed=%d, timeout=%d) due to: %s", count++, delay, elapsed, + timeout, exception.getMessage()); + try { + return sendOnce(sealedRequest); + } catch (Exception exp) { + exception = ClickHouseException.of(exp.getCause() != null ? exp.getCause() : exp, + sealedRequest.getServer()); + } + + elapsed = System.currentTimeMillis() - startTime; + if (exception.getErrorCode() != errorCode || elapsed + delay >= timeout) { + log.warn("Stopped repeating(delay=%d, elapsed=%d, timeout=%d) for %s", delay, elapsed, + timeout, exception.getMessage()); + break; + } + + try { + Thread.sleep(delay); + elapsed += delay; + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + break; + } + if (delay >= MAX_REPEAT_DELAY) { + delay = MAX_REPEAT_DELAY; + } else { + delay += REPEAT_DELAY_BACKOFF; + } + } + } + throw new CompletionException(exception); + } + ClickHouseResponse retry(ClickHouseRequest sealedRequest, ClickHouseException exception, int times) { for (int i = 1; i <= times; i++) { log.debug("Retry %d of %d due to: %s", i, times, exception.getMessage()); @@ -186,18 +248,27 @@ ClickHouseResponse handle(ClickHouseRequest sealedRequest, Throwable cause) { cause = ((UncheckedIOException) cause).getCause(); } - log.debug("Handling %s(failover=%d, retry=%d)", cause, sealedRequest.getConfig().getFailover(), - sealedRequest.getConfig().getRetry()); + ClickHouseConfig config = sealedRequest.getConfig(); + log.debug("Handling %s(failover=%d, retry=%d)", cause, config.getFailover(), config.getRetry()); + ClickHouseException ex = ClickHouseException.of(cause, sealedRequest.getServer()); try { + if (config.isRepeatOnSessionLock() + && ex.getErrorCode() == ClickHouseException.ERROR_SESSION_IS_LOCKED) { + // connection timeout is usually a small number(defaults to 5000 ms), making it + // better default compare to socket timeout and max execution time etc. + return repeat(sealedRequest, ex, config.getSessionTimeout() <= 0 ? config.getConnectionTimeout() + : TimeUnit.SECONDS.toMillis(config.getSessionTimeout())); + } + int times = sealedRequest.getConfig().getFailover(); if (times > 0) { - return failover(sealedRequest, ClickHouseException.of(cause, sealedRequest.getServer()), times); + return failover(sealedRequest, ex, times); } // different from failover: 1) retry on the same node; 2) never retry on timeout times = sealedRequest.getConfig().getRetry(); if (times > 0) { - return retry(sealedRequest, ClickHouseException.of(cause, sealedRequest.getServer()), times); + return retry(sealedRequest, ex, times); } throw new CompletionException(cause); @@ -210,11 +281,12 @@ ClickHouseResponse handle(ClickHouseRequest sealedRequest, Throwable cause) { ClickHouseResponse sendOnce(ClickHouseRequest sealedRequest) { try { - return getClient().execute(sealedRequest).get(); + return getClient().execute(sealedRequest).get(sealedRequest.getConfig().getSocketTimeout(), + TimeUnit.MILLISECONDS); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new CancellationException("Execution was interrupted"); - } catch (ExecutionException e) { + } catch (ExecutionException | TimeoutException e) { throw new CompletionException(e.getCause()); } } diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseConfig.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseConfig.java index 8e48dc236..2cdca4e7a 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseConfig.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseConfig.java @@ -188,6 +188,7 @@ public static Map toClientOptions(Map prop private final int nodeCheckInterval; private final int failover; private final int retry; + private final boolean repeatOnSessionLock; private final boolean reuseValueWrapper; private final boolean serverInfo; private final TimeZone serverTimeZone; @@ -200,6 +201,7 @@ public static Map toClientOptions(Map prop private final String sslRootCert; private final String sslCert; private final String sslKey; + private final int transactionTimeout; private final boolean useBlockingQueue; private final boolean useObjectsInArray; private final boolean useNoProxy; @@ -281,6 +283,7 @@ public ClickHouseConfig(Map options, ClickHouseC this.nodeCheckInterval = (int) getOption(ClickHouseClientOption.NODE_CHECK_INTERVAL); this.failover = (int) getOption(ClickHouseClientOption.FAILOVER); this.retry = (int) getOption(ClickHouseClientOption.RETRY); + this.repeatOnSessionLock = (boolean) getOption(ClickHouseClientOption.REPEAT_ON_SESSION_LOCK); this.reuseValueWrapper = (boolean) getOption(ClickHouseClientOption.REUSE_VALUE_WRAPPER); this.serverInfo = !ClickHouseChecker.isNullOrBlank((String) getOption(ClickHouseClientOption.SERVER_TIME_ZONE)) && !ClickHouseChecker.isNullOrBlank((String) getOption(ClickHouseClientOption.SERVER_VERSION)); @@ -296,6 +299,7 @@ public ClickHouseConfig(Map options, ClickHouseC this.sslRootCert = (String) getOption(ClickHouseClientOption.SSL_ROOT_CERTIFICATE); this.sslCert = (String) getOption(ClickHouseClientOption.SSL_CERTIFICATE); this.sslKey = (String) getOption(ClickHouseClientOption.SSL_KEY); + this.transactionTimeout = (int) getOption(ClickHouseClientOption.TRANSACTION_TIMEOUT); this.useBlockingQueue = (boolean) getOption(ClickHouseClientOption.USE_BLOCKING_QUEUE); this.useObjectsInArray = (boolean) getOption(ClickHouseClientOption.USE_OBJECTS_IN_ARRAYS); this.useNoProxy = (boolean) getOption(ClickHouseClientOption.USE_NO_PROXY); @@ -578,6 +582,10 @@ public int getRetry() { return retry; } + public boolean isRepeatOnSessionLock() { + return repeatOnSessionLock; + } + /** * Checks whether retry is enabled or not. * @@ -643,6 +651,10 @@ public String getSslKey() { return sslKey; } + public int getTransactionTimeout() { + return transactionTimeout < 1 ? sessionTimeout : transactionTimeout; + } + public boolean isUseBlockingQueue() { return useBlockingQueue; } diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDataProcessor.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDataProcessor.java index 6ddc78806..aa4760f3e 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDataProcessor.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDataProcessor.java @@ -119,11 +119,15 @@ protected static , T extends ClickHouseValue> void buildMappin * Checks whether there's more to read from input stream. * * @return true if there's more; false otherwise - * @throws UncheckedIOException when failed to read columns from input stream + * @throws UncheckedIOException when failed to read data from input stream */ private boolean hasNext() throws UncheckedIOException { try { - return input.available() > 0; + if (input.available() <= 0) { + input.close(); + return false; + } + return true; } catch (IOException e) { throw new UncheckedIOException(e); } @@ -135,7 +139,7 @@ private boolean hasNext() throws UncheckedIOException { * * @return non-null record * @throws NoSuchElementException when no more record to read - * @throws UncheckedIOException when failed to read columns from input stream + * @throws UncheckedIOException when failed to read data from input stream */ private ClickHouseRecord nextRecord() throws NoSuchElementException, UncheckedIOException { final ClickHouseRecord r = config.isReuseValueWrapper() ? currentRecord : currentRecord.copy(); @@ -164,7 +168,7 @@ private ClickHouseRecord nextRecord() throws NoSuchElementException, UncheckedIO * * @return non-null value * @throws NoSuchElementException when no more value to read - * @throws UncheckedIOException when failed to read columns from input stream + * @throws UncheckedIOException when failed to read data from input stream */ private ClickHouseValue nextValue() throws NoSuchElementException, UncheckedIOException { ClickHouseColumn column = columns[readPosition]; @@ -312,7 +316,7 @@ protected ClickHouseDataProcessor(ClickHouseConfig config, ClickHouseInputStream } } - if (this.columns.length == 0 || input == null) { + if (input == null) { this.currentRecord = ClickHouseRecord.EMPTY; this.records = Collections.emptyIterator(); this.values = Collections.emptyIterator(); @@ -341,25 +345,25 @@ public final List getColumns() { * Returns an iterable collection of records which can be walked through in a * foreach-loop. Please pay attention that: 1) * {@link java.io.UncheckedIOException} might be thrown when iterating through - * the collection; and 2) it's not supposed to be called for more than once. + * the collection; and 2) it's not supposed to be called for more than once + * because the input stream will be closed at the end of reading. * * @return non-null iterable records + * @throws UncheckedIOException when failed to access the input stream */ public final Iterable records() { - if (columns.length == 0) { - return Collections.emptyList(); - } - return () -> records; } /** * Returns an iterable collection of values which can be walked through in a - * foreach-loop. It's slower than {@link #records()}, because the latter - * reads data in bulk. However, it's particular useful when you're reading large - * values with limited memory - e.g. a binary field with a few GB bytes. - * + * foreach-loop. In general, this is slower than {@link #records()}, because the + * latter reads data in bulk. However, it's particular useful when you're + * reading large values with limited memory - e.g. a binary field with a few GB + * bytes. Similarly, the input stream will be closed at the end of reading. + * * @return non-null iterable values + * @throws UncheckedIOException when failed to access the input stream */ public final Iterable values() { if (columns.length == 0) { diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseException.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseException.java index 0728fed12..fe2cbdf5d 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseException.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseException.java @@ -19,10 +19,13 @@ public class ClickHouseException extends Exception { public static final int ERROR_ABORTED = 236; public static final int ERROR_CANCELLED = 394; public static final int ERROR_NETWORK = 210; + public static final int ERROR_SESSION_NOT_FOUND = 372; + public static final int ERROR_SESSION_IS_LOCKED = 373; public static final int ERROR_POCO = 1000; public static final int ERROR_TIMEOUT = 159; public static final int ERROR_UNKNOWN = 1002; + static final String MSG_CODE = "Code: "; static final String MSG_CONNECT_TIMED_OUT = "connect timed out"; private final int errorCode; @@ -36,6 +39,12 @@ private static String buildErrorMessage(int code, String message, ClickHouseNode if (message != null && !message.isEmpty()) { builder.append(message); + } else if (code == ERROR_ABORTED) { + builder.append(MSG_CODE).append(code).append(". Execution aborted"); + } else if (code == ERROR_CANCELLED) { + builder.append(MSG_CODE).append(code).append(". Execution cancelled"); + } else if (code == ERROR_TIMEOUT) { + builder.append(MSG_CODE).append(code).append(". Execution timed out"); } else { builder.append("Unknown error ").append(code); } @@ -86,6 +95,17 @@ static Throwable getRootCause(Throwable t) { return rootCause; } + /** + * Builds error message like {@code "Code: , "}. + * + * @param code error code + * @param detail detail of the error + * @return non-null error message + */ + public static String buildErrorMessage(int code, String detail) { + return new StringBuilder().append(MSG_CODE).append(code).append(", ").append(detail).toString(); + } + /** * Creates an exception for cancellation. * @@ -188,6 +208,19 @@ public ClickHouseException(int code, String message, ClickHouseNode server) { errorCode = code; } + /** + * Constructs an exception. + * + * @param code error code + * @param message error message + * @param cause cause + */ + protected ClickHouseException(int code, String message, Throwable cause) { + super(message, cause); + + errorCode = code; + } + /** * Gets error code. * diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseFile.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseFile.java index 8887c476e..fa75c13b8 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseFile.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseFile.java @@ -4,6 +4,7 @@ import java.io.FileInputStream; import java.io.FileNotFoundException; import java.io.FileOutputStream; +import java.io.Serializable; import java.nio.file.Path; import com.clickhouse.client.config.ClickHouseClientOption; @@ -12,7 +13,9 @@ * Wrapper of {@link java.io.File} with additional information like compression * and format. */ -public class ClickHouseFile { +public class ClickHouseFile implements Serializable { + private static final long serialVersionUID = -2641191818870839568L; + /** * Null file which has no compression and format. */ diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseInputStream.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseInputStream.java index d7967e117..b3ad5a75b 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseInputStream.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseInputStream.java @@ -452,21 +452,11 @@ public static long pipe(InputStream input, OutputStream output, byte[] buffer) t int size = buffer.length; long count = 0L; int written = 0; - try { - while ((written = input.read(buffer, 0, size)) >= 0) { + try (InputStream in = input) { + while ((written = in.read(buffer, 0, size)) >= 0) { output.write(buffer, 0, written); count += written; } - input.close(); - input = null; - } finally { - if (input != null) { - try { - input.close(); - } catch (Exception e) { - // ignore - } - } } return count; } 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 e188f9450..a3d787cde 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseRequest.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseRequest.java @@ -20,6 +20,8 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicReference; import java.util.Optional; import java.util.Properties; @@ -47,7 +49,12 @@ public static class Mutation extends ClickHouseRequest { protected Mutation(ClickHouseRequest request, boolean sealed) { super(request.getClient(), request.server, request.serverRef, request.options, sealed); + this.settings.putAll(request.settings); + this.txRef.set(request.txRef.get()); + + this.changeListener = request.changeListener; + this.serverListener = request.serverListener; } @Override @@ -182,7 +189,7 @@ public Mutation data(String file, ClickHouseCompression compression) { * Loads data from input stream. * * @param input input stream - * @return mutation requets + * @return mutation request */ public Mutation data(InputStream input) { return data(ClickHouseInputStream.of(input)); @@ -192,7 +199,7 @@ public Mutation data(InputStream input) { * Loads data from input stream. * * @param input input stream - * @return mutation requets + * @return mutation request */ public Mutation data(ClickHouseInputStream input) { checkSealed(); @@ -207,7 +214,7 @@ public Mutation data(ClickHouseInputStream input) { * Loads data from input stream. * * @param input input stream - * @return mutation requets + * @return mutation request */ public Mutation data(ClickHouseDeferredValue input) { checkSealed(); @@ -226,7 +233,7 @@ public CompletableFuture execute() { data(stream.getInputStream()); CompletableFuture future = null; if (c.isAsync()) { - future = getClient().execute(isSealed() ? this : seal()); + future = getClient().execute(this); } try (ClickHouseOutputStream out = stream) { writer.write(out); @@ -238,7 +245,7 @@ public CompletableFuture execute() { } } - return getClient().execute(isSealed() ? this : seal()); + return getClient().execute(this); } @Override @@ -250,7 +257,7 @@ public ClickHouseResponse executeAndWait() throws ClickHouseException { data(stream.getInputStream()); CompletableFuture future = null; if (c.isAsync()) { - future = getClient().execute(isSealed() ? this : seal()); + future = getClient().execute(this); } try (ClickHouseOutputStream out = stream) { writer.write(out); @@ -259,13 +266,13 @@ public ClickHouseResponse executeAndWait() throws ClickHouseException { } if (future != null) { try { - return future.get(); + return future.get(c.getSocketTimeout(), TimeUnit.MILLISECONDS); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw ClickHouseException.forCancellation(e, getServer()); } catch (CancellationException e) { throw ClickHouseException.forCancellation(e, getServer()); - } catch (ExecutionException | UncheckedIOException e) { + } catch (ExecutionException | TimeoutException | UncheckedIOException e) { Throwable cause = e.getCause(); if (cause == null) { cause = e; @@ -278,16 +285,19 @@ public ClickHouseResponse executeAndWait() throws ClickHouseException { } } - return getClient().executeAndWait(isSealed() ? this : seal()); + return getClient().executeAndWait(this); } /** - * Sends mutation requets for execution. Same as + * Sends mutation request for execution. Same as * {@code client.execute(request.seal())}. * * @return non-null future to get response * @throws CompletionException when error occurred + * @deprecated will be removed in v0.3.3, please use {@link #execute()} + * instead */ + @Deprecated public CompletableFuture send() { return execute(); } @@ -297,6 +307,8 @@ public CompletableFuture send() { * * @return non-null response * @throws ClickHouseException when error occurred during execution + * @deprecated will be removed in v0.3.3, please use {@link #executeAndWait()} + * instead */ public ClickHouseResponse sendAndWait() throws ClickHouseException { return executeAndWait(); @@ -327,6 +339,8 @@ public Mutation seal() { req.sql = sql; req.preparedQuery = preparedQuery; + req.managerRef.set(managerRef.get()); + req.txRef.set(txRef.get()); } return req; @@ -336,19 +350,24 @@ public Mutation seal() { private static final long serialVersionUID = 4990313525960702287L; static final String PROP_DATA = "data"; + static final String PROP_MANAGER = "manager"; static final String PROP_OUTPUT = "output"; static final String PROP_PREPARED_QUERY = "preparedQuery"; static final String PROP_QUERY = "query"; static final String PROP_QUERY_ID = "queryId"; + static final String PROP_TRANSACTION = "transaction"; static final String PROP_WRITER = "writer"; private final boolean sealed; private final ClickHouseClient client; + protected final AtomicReference managerRef; protected final Function server; protected final AtomicReference serverRef; - protected final transient List externalTables; + protected final AtomicReference txRef; + + protected final List externalTables; protected final Map options; protected final Map settings; @@ -375,8 +394,11 @@ protected ClickHouseRequest(ClickHouseClient client, Function(null); this.server = (Function & Serializable) server; this.serverRef = ref == null ? new AtomicReference<>(null) : ref; + this.txRef = new AtomicReference<>(null); this.sealed = sealed; this.externalTables = new LinkedList<>(); @@ -408,12 +430,13 @@ protected void checkSealed() { } } + /** + * Gets client associated with the request. + * + * @return non-null client + */ protected ClickHouseClient getClient() { - if (client == null) { - throw new IllegalStateException("Non-null client is required"); - } - - return client; + return client != null ? client : ClickHouseClient.newInstance(getServer().getProtocol()); } /** @@ -436,9 +459,10 @@ protected void resetCache() { } /** - * Creates a copy of this request. Pay attention that the same node reference - * (returned from {@link #getServer()}) will be copied to the new request as - * well, meaning failover will change node for both requets. + * Creates a copy of this request including listeners. Please pay attention that + * the same node reference (returned from {@link #getServer()}) will be copied + * to the new request as well, meaning failover will change node for both + * requests. * * @return copy of this request */ @@ -452,9 +476,30 @@ public ClickHouseRequest copy() { req.queryId = queryId; req.sql = sql; req.preparedQuery = preparedQuery; + req.managerRef.set(managerRef.get()); + req.txRef.set(txRef.get()); + req.changeListener = changeListener; + req.serverListener = serverListener; return req; } + /** + * Gets manager for the request, which defaults to + * {@link ClickHouseRequestManager#getInstance()}. + * + * @return non-null request manager + */ + public ClickHouseRequestManager getManager() { + ClickHouseRequestManager m = managerRef.get(); + if (m == null) { + m = ClickHouseRequestManager.getInstance(); + if (!managerRef.compareAndSet(null, m)) { + m = managerRef.get(); + } + } + return m; + } + /** * Checks if the request is sealed(immutable). * @@ -464,6 +509,15 @@ public boolean isSealed() { return this.sealed; } + /** + * Checks if the request is bounded with a transaction. + * + * @return true if the request is bounded with a transaction; false otherwise + */ + public boolean isTransactional() { + return txRef.get() != null; + } + /** * Checks if the request contains any input stream. * @@ -524,27 +578,16 @@ public ClickHouseConfig getConfig() { return config; } - /** - * Sets change listener. - * - * @param listener change listener which may or may not be null - * @return the request itself - */ - @SuppressWarnings("unchecked") - public final SelfT setChangeListener(ClickHouseConfigChangeListener> listener) { - this.changeListener = listener; - return (SelfT) this; + public ClickHouseTransaction getTransaction() { + return txRef.get(); } - /** - * Sets server change listener. - * - * @param listener change listener which may or may not be null - * @return the request itself - */ - public final SelfT setServerListener(BiConsumer listener) { - this.serverListener = listener; - return (SelfT) this; + public final ClickHouseConfigChangeListener> getChangeListener() { + return this.changeListener; + } + + public final BiConsumer getServerListener() { + return this.serverListener; } /** @@ -914,6 +957,23 @@ public SelfT format(ClickHouseFormat format) { return (SelfT) this; } + /** + * Sets request manager which is responsible for generating query ID and session + * ID, as well as transaction creation. + * + * @param manager request manager + * @return the request itself + */ + @SuppressWarnings("unchecked") + public SelfT manager(ClickHouseRequestManager manager) { + checkSealed(); + ClickHouseRequestManager current = managerRef.get(); + if (!Objects.equals(current, manager) && managerRef.compareAndSet(current, manager)) { + changeProperty(PROP_MANAGER, current, manager); + } + return (SelfT) this; + } + /** * Sets an option. {@code option} is for configuring client's behaviour, while * {@code setting} is for server. @@ -1428,8 +1488,41 @@ public SelfT query(String query, String queryId) { } /** - * Clears session configuration including session id, whether to validate the id - * and session timeout. + * Sets all server settings. + * + * @param settings settings + * @return the request itself + */ + @SuppressWarnings("unchecked") + public SelfT settings(Map settings) { + checkSealed(); + + if (changeListener == null) { + this.settings.clear(); + if (settings != null) { + this.settings.putAll(settings); + } + resetCache(); + } else { + Map m = new HashMap<>(); + m.putAll(this.settings); + if (options != null) { + for (Entry e : settings.entrySet()) { + set(e.getKey(), e.getValue()); + m.remove(e.getKey()); + } + } + for (String s : m.keySet()) { + removeSetting(s); + } + } + + return (SelfT) this; + } + + /** + * Clears session configuration including session id, session check(whether to + * validate the id), and session timeout. Transaction will be removed as well. * * @return the request itself */ @@ -1441,6 +1534,12 @@ public SelfT clearSession() { removeOption(ClickHouseClientOption.SESSION_CHECK); removeOption(ClickHouseClientOption.SESSION_TIMEOUT); + // assume the transaction is managed somewhere else + ClickHouseTransaction tx = txRef.get(); + if (tx != null && txRef.compareAndSet(tx, null)) { + changeProperty(PROP_TRANSACTION, tx, null); + } + return (SelfT) this; } @@ -1471,7 +1570,7 @@ public SelfT session(String sessionId, Boolean check) { * Sets current session. Same as {@code session(sessionId, null, timeout)}. * * @param sessionId session id, null means no session - * @param timeout timeout in milliseconds + * @param timeout timeout in seconds * @return the request itself */ public SelfT session(String sessionId, Integer timeout) { @@ -1484,16 +1583,23 @@ public SelfT session(String sessionId, Integer timeout) { * @param sessionId session id, null means no session * @param check whether the server should check if the session id exists or * not - * @param timeout timeout in milliseconds + * @param timeout timeout in seconds * @return the request itself */ @SuppressWarnings("unchecked") public SelfT session(String sessionId, Boolean check, Integer timeout) { checkSealed(); - option(ClickHouseClientOption.SESSION_ID, sessionId); - option(ClickHouseClientOption.SESSION_CHECK, check); - option(ClickHouseClientOption.SESSION_TIMEOUT, timeout); + ClickHouseTransaction tx = txRef.get(); + if (tx != null) { + throw new IllegalArgumentException(ClickHouseUtils.format( + "Please complete %s (or clear session) before changing session to (id=%s, check=%s, timeout=%s)", + tx, sessionId, check, timeout)); + } else { + option(ClickHouseClientOption.SESSION_ID, sessionId); + option(ClickHouseClientOption.SESSION_CHECK, check); + option(ClickHouseClientOption.SESSION_TIMEOUT, timeout); + } return (SelfT) this; } @@ -1541,6 +1647,32 @@ public SelfT set(String setting, String value) { return set(setting, (Serializable) ClickHouseUtils.escape(value, '\'')); } + /** + * Sets thread-safe change listener. Please keep in mind that the same listener + * might be shared by multiple requests. + * + * @param listener thread-safe change listener which may or may not be null + * @return the request itself + */ + @SuppressWarnings("unchecked") + public final SelfT setChangeListener(ClickHouseConfigChangeListener> listener) { + this.changeListener = listener; + return (SelfT) this; + } + + /** + * Sets thread-safe server change listener. Please keep in mind that the same + * listener might be shared by multiple requests. + * + * @param listener thread-safe server listener which may or may not be null + * @return the request itself + */ + @SuppressWarnings("unchecked") + public final SelfT setServerListener(BiConsumer listener) { + this.serverListener = listener; + return (SelfT) this; + } + /** * Sets target table. Same as {@code table(table, null)}. * @@ -1564,6 +1696,74 @@ public SelfT table(String table, String queryId) { return query("SELECT * FROM ".concat(ClickHouseChecker.nonBlank(table, "table")), queryId); } + /** + * Creates and starts a transaction. Same as {@code transaction(0)}. + * + * @return the request itself + * @throws ClickHouseException when failed to start or reuse transaction + */ + public SelfT transaction() throws ClickHouseException { + return transaction(0); + } + + /** + * Creates and starts a transaction immediately. Please pay attention that + * unlike other methods in this class, it will connect to ClickHouse server, + * allocate session and start transaction right away. + * + * @param timeout transaction timeout in seconds, zero or negative means + * same as session timeout + * @return the request itself + * @throws ClickHouseException when failed to start or reuse transaction + */ + @SuppressWarnings("unchecked") + public SelfT transaction(int timeout) throws ClickHouseException { + ClickHouseTransaction tx = txRef.get(); + if (tx != null && tx.getTimeout() == (timeout > 0 ? timeout : 0)) { + return (SelfT) this; + } + return transaction(getManager().getOrStartTransaction(this, timeout)); + } + + /** + * Sets transaction. Any existing transaction, regardless its state, will be + * replaced by the given one. + * + * @param transaction transaction + * @return the request itself + * @throws ClickHouseException when failed to set transaction + */ + @SuppressWarnings("unchecked") + public SelfT transaction(ClickHouseTransaction transaction) throws ClickHouseException { + checkSealed(); + + try { + txRef.updateAndGet(x -> { + if (changeProperty(PROP_TRANSACTION, x, transaction) != null) { + final ClickHouseNode currentServer = getServer(); + final ClickHouseNode txServer = transaction.getServer(); + // there's no global transaction and ReplicateMergeTree is not supported + if (!currentServer.isSameEndpoint(txServer) && changeServer(currentServer, txServer) != txServer) { + throw new IllegalStateException(ClickHouseUtils + .format("Failed to change current server from %s to %s", currentServer, txServer)); + } + // skip the check in session method + option(ClickHouseClientOption.SESSION_ID, transaction.getSessionId()); + option(ClickHouseClientOption.SESSION_CHECK, true); + option(ClickHouseClientOption.SESSION_TIMEOUT, + transaction.getTimeout() < 1 ? null : transaction.getTimeout()); + removeSetting(ClickHouseTransaction.SETTING_IMPLICIT_TRANSACTION); + } else if (x != null) { + clearSession(); + } + return transaction; + }); + } catch (IllegalStateException e) { + throw ClickHouseException.of(e.getMessage(), getServer()); + } + return (SelfT) this; + } + /** * Changes current database. * @@ -1675,6 +1875,10 @@ public SelfT reset() { checkSealed(); this.externalTables.clear(); + this.namedParameters.clear(); + + this.serverListener = null; + if (changeListener == null) { this.options.clear(); this.settings.clear(); @@ -1685,17 +1889,24 @@ public SelfT reset() { for (String s : this.settings.keySet().toArray(new String[0])) { removeSetting(s); } - this.changeListener = null; } - this.serverListener = null; - this.namedParameters.clear(); - this.input = changeProperty(PROP_DATA, this.input, null); this.output = changeProperty(PROP_OUTPUT, this.output, null); this.sql = changeProperty(PROP_QUERY, this.sql, null); this.preparedQuery = changeProperty(PROP_PREPARED_QUERY, this.preparedQuery, null); this.queryId = changeProperty(PROP_QUERY_ID, this.queryId, null); + ClickHouseRequestManager current = managerRef.get(); + if (current != null && managerRef.compareAndSet(current, null)) { + changeProperty(PROP_MANAGER, current, null); + } + ClickHouseTransaction tx = txRef.get(); + if (tx != null && txRef.compareAndSet(tx, null)) { + changeProperty(PROP_TRANSACTION, tx, null); + } + + this.changeListener = null; + resetCache(); return (SelfT) this; @@ -1703,6 +1914,7 @@ public SelfT reset() { /** * Creates a sealed request, which is an immutable copy of the current request. + * Listeners won't be copied to the sealed instance, because it's immutable. * * @return sealed request, an immutable copy of the current request */ @@ -1722,6 +1934,8 @@ public ClickHouseRequest seal() { req.queryId = queryId; req.sql = sql; req.preparedQuery = preparedQuery; + req.managerRef.set(managerRef.get()); + req.txRef.set(txRef.get()); } return req; @@ -1745,7 +1959,7 @@ public Mutation write() { * @throws CompletionException when error occurred during execution */ public CompletableFuture execute() { - return getClient().execute(isSealed() ? this : seal()); + return getClient().execute(this); } /** @@ -1755,6 +1969,45 @@ public CompletableFuture execute() { * @throws ClickHouseException when error occurred during execution */ public ClickHouseResponse executeAndWait() throws ClickHouseException { - return getClient().executeAndWait(isSealed() ? this : seal()); + return getClient().executeAndWait(this); + } + + /** + * Executes the request within a transaction, wait until it's completed and + * the transaction being committed or rolled back. The transaction here is + * either an implicit transaction(using {@code implicit_transaction} server + * setting, with less overhead but requiring 22.7+) or auto-commit + * transaction(using clone of this request), depending on argument + * {@code useImplicitTransaction}. + * + * @param useImplicitTransaction use {@code implicit_transaction} server setting + * with minimum overhead(no session on server side + * and no additional objects on client side), or + * an auto-commit {@link ClickHouseTransaction} + * @return non-null response + * @throws ClickHouseException when error occurred during execution + */ + public ClickHouseResponse executeWithinTransaction(boolean useImplicitTransaction) throws ClickHouseException { + if (useImplicitTransaction) { + return set(ClickHouseTransaction.SETTING_IMPLICIT_TRANSACTION, 1).transaction(null).executeAndWait(); + } + + ClickHouseTransaction tx = null; + try { + tx = getManager().createImplicitTransaction(this); + // transaction will be committed only when the response is fully consumed + return getClient().executeAndWait(transaction(tx)); + } catch (Exception e) { + if (tx != null) { + try { + tx.rollback(); + } catch (Exception ex) { + // ignore + } + } + throw ClickHouseException.of(e, getServer()); + } finally { + transaction(null); + } } } diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseRequestManager.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseRequestManager.java new file mode 100644 index 000000000..3980d7f08 --- /dev/null +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseRequestManager.java @@ -0,0 +1,142 @@ +package com.clickhouse.client; + +import java.util.UUID; + +/** + * Request manager is responsible for generating query and session ID, as well + * as transaction creation. {@link java.util.ServiceLoader} will search and + * instantiate customized request manager first, and then fall back to default + * implementation if no luck. + */ +public class ClickHouseRequestManager { + /** + * Inner class for static initialization. + */ + static final class InstanceHolder { + private static final ClickHouseRequestManager instance = ClickHouseUtils + .getService(ClickHouseRequestManager.class, ClickHouseRequestManager::new); + + private InstanceHolder() { + } + } + + /** + * Gets instance of request manager. + * + * @return non-null request manager + */ + public static ClickHouseRequestManager getInstance() { + return InstanceHolder.instance; + } + + /** + * Creates a new query ID. + * + * @return non-null query ID + */ + public String createQueryId() { + return UUID.randomUUID().toString(); + } + + /** + * Creates a new session ID. + * + * @return non-null session ID + */ + public String createSessionId() { + return UUID.randomUUID().toString(); + } + + /** + * Creates an implicit transaction. + * + * @param request non-null request + * @return non-null new transaction + * @throws ClickHouseException when failed to create implicit transaction + */ + public ClickHouseTransaction createImplicitTransaction(ClickHouseRequest request) throws ClickHouseException { + return new ClickHouseTransaction(ClickHouseChecker.nonNull(request, "Request").getServer(), + request.getConfig().getTransactionTimeout(), true); + } + + /** + * Creates a new transaction. Same as {@code createTransaction(request, 0)}. + * + * @param request non-null request + * @return non-null new transaction + * @throws ClickHouseException when failed to create transaction + */ + public ClickHouseTransaction createTransaction(ClickHouseRequest request) throws ClickHouseException { + return createTransaction(request, 0); + } + + /** + * Creates a new transaction. Unlike + * {@link #getOrStartTransaction(ClickHouseRequest, int)}, the transaction's + * state is {@link ClickHouseTransaction#NEW} and it's not bounded with the + * request. + * + * @param request non-null request + * @param timeout transaction timeout in seconds, zero or negative number + * means {@code request.getConfig().getTransactionTimeout()} + * @return non-null new transaction + * @throws ClickHouseException when failed to create transaction + */ + public ClickHouseTransaction createTransaction(ClickHouseRequest request, int timeout) + throws ClickHouseException { + return createTransaction(ClickHouseChecker.nonNull(request, "Request").getServer(), + request.getConfig().getTransactionTimeout()); + } + + /** + * Creates a new transaction. {@link #createSessionId()} will be called + * to start a new session just for the transaction. + * + * @param server non-null server + * @param timeout transaction timeout in seconds + * @return non-null new transaction + * @throws ClickHouseException when failed to create transaction + */ + public ClickHouseTransaction createTransaction(ClickHouseNode server, int timeout) throws ClickHouseException { + return new ClickHouseTransaction(ClickHouseChecker.nonNull(server, "Server"), createSessionId(), + timeout > 0 ? timeout : server.config.getTransactionTimeout(), null); + } + + /** + * Gets or starts a new transaction. Same as + * {@code getOrStartTransaction(request, 0)}. + * + * @param request non-null request + * @return non-null transaction + * @throws ClickHouseException when failed to get or start transaction + */ + public ClickHouseTransaction getOrStartTransaction(ClickHouseRequest request) throws ClickHouseException { + return getOrStartTransaction(request, 0); + } + + /** + * Gets or starts a new transaction. {@link #createSessionId()} will be called + * to when a new transaction is created. + * + * @param request non-null request + * @param timeout transaction timeout in seconds, zero or negative number + * means {@code request.getConfig().getTransactionTimeout()} + * @return non-null transaction in {@link ClickHouseTransaction#ACTIVE} state + * @throws ClickHouseException when failed to get or start transaction + */ + public ClickHouseTransaction getOrStartTransaction(ClickHouseRequest request, int timeout) + throws ClickHouseException { + if (timeout < 1) { + timeout = request.getConfig().getTransactionTimeout(); + } + ClickHouseTransaction tx = ClickHouseChecker.nonNull(request, "Request").getTransaction(); + if (tx != null && tx.getTimeout() == timeout) { + return tx; + } + + tx = createTransaction(request.getServer(), timeout); + tx.begin(); + request.transaction(tx); + return tx; + } +} 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 ee9f108df..de8c6749b 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseResponse.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseResponse.java @@ -110,6 +110,7 @@ default ClickHouseRecord firstRecord() { * supposed to be called for more than once. * * @return non-null iterable collection + * @throws UncheckedIOException when failed to read data(e.g. deserialization) */ Iterable records(); diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseTransaction.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseTransaction.java new file mode 100644 index 000000000..30d4d7b27 --- /dev/null +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseTransaction.java @@ -0,0 +1,686 @@ +package com.clickhouse.client; + +import java.io.Serializable; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +import com.clickhouse.client.logging.Logger; +import com.clickhouse.client.logging.LoggerFactory; + +/** + * This class represents a transaction in ClickHouse. Besides transaction ID + * {@code Tuple(snapshotVersion UInt64, localTxCounter UInt64, hostId UUID)}, it + * also contains session ID and references to the connected server and client + * for issuing queries. + */ +public final class ClickHouseTransaction implements Serializable { + /** + * This class encapsulates transaction ID, which is defined as + * {@code Tuple(snapshotVersion UInt64, localTxCounter UInt64, hostId UUID)}. + */ + public static class XID implements Serializable { + private static final long serialVersionUID = 4907177669971332404L; + + public static final XID EMPTY = new XID(0L, 0L, new UUID(0L, 0L).toString()); + + /** + * Creates transaction ID from the given tuple. + * + * @param list non-null tuple with 3 elements + * @return non-null transaction ID + */ + public static XID of(List list) { + if (list == null || list.size() != 3) { + throw new IllegalArgumentException( + "Non-null tuple with 3 elements(long, long, String) is required"); + } + long snapshotVersion = (Long) list.get(0); + long localTxCounter = (Long) list.get(1); + String hostId = String.valueOf(list.get(2)); + if (EMPTY.snapshotVersion == snapshotVersion && EMPTY.localTxCounter == localTxCounter + && EMPTY.hostId.equals(hostId)) { + return EMPTY; + } + return new XID(snapshotVersion, localTxCounter, hostId); + } + + private final long snapshotVersion; + private final long localTxCounter; + private final String hostId; + + protected XID(long snapshotVersion, long localTxCounter, String hostId) { + this.snapshotVersion = snapshotVersion; + this.localTxCounter = localTxCounter; + this.hostId = hostId; + } + + public long getSnapshotVersion() { + return snapshotVersion; + } + + public long getLocalTransactionCounter() { + return localTxCounter; + } + + public String getHostId() { + return hostId; + } + + public String asTupleString() { + return new StringBuilder().append('(').append(snapshotVersion).append(',').append(localTxCounter) + .append(",'").append(hostId).append("')").toString(); + } + + @Override + public int hashCode() { + final int prime = 31; + int result = prime + (int) (snapshotVersion ^ (snapshotVersion >>> 32)); + result = prime * result + (int) (localTxCounter ^ (localTxCounter >>> 32)); + result = prime * result + hostId.hashCode(); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } else if (obj == null || getClass() != obj.getClass()) { + return false; + } + + XID other = (XID) obj; + return snapshotVersion == other.snapshotVersion && localTxCounter == other.localTxCounter + && hostId.equals(other.hostId); + } + + @Override + public String toString() { + return new StringBuilder().append("TransactionId [snapshotVersion=").append(snapshotVersion) + .append(", localTxCounter=").append(localTxCounter).append(", hostId=").append(hostId).append("]@") + .append(hashCode()).toString(); + } + } + + private static final Logger log = LoggerFactory.getLogger(ClickHouseTransaction.class); + private static final long serialVersionUID = -4618710299106666829L; + + private static final String[] NAMES = new String[] { + "New", "Active", "Failed", "Commited", "RolledBack" + }; + + static final String QUERY_SELECT_TX_ID = "SELECT transactionID()"; + + // transaction state + public static final int NEW = 0; + public static final int ACTIVE = 1; + public static final int FAILED = 2; + public static final int COMMITTED = 3; + public static final int ROLLED_BACK = 4; + + // reserved CSN - see + // https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/TransactionID.h + public static final long CSN_UNKNOWN = 0L; // For transactions that are probably not committed (yet) + public static final long CSN_PREHISTORIC = 1L; // For changes were made without creating a transaction + // Special reserved values + public static final long CSN_COMMITTING = 2L; + public static final long CSN_EVERYTHING_VISIBLE = 3L; + public static final long CSN_MAX_RESERVED = 32L; + + public static final String SETTING_IMPLICIT_TRANSACTION = "implicit_transaction"; + public static final String SETTING_THROW_ON_UNSUPPORTED_QUERY_INSIDE_TRANSACTION = "throw_on_unsupported_query_inside_transaction"; + public static final String SETTING_WAIT_CHANGES_BECOME_VISIBLE_AFTER_COMMIT_MODE = "wait_changes_become_visible_after_commit_mode"; + + /** + * Updates the given request by enabling or disabling implicit transaction. + * + * @param request non-null request to update + * @param enable whether enable implicit transaction or not + * @throws ClickHouseException when failed to enable or disable implicit + * transaction + */ + static void setImplicitTransaction(ClickHouseRequest request, boolean enable) throws ClickHouseException { + if (enable) { + request.set(SETTING_IMPLICIT_TRANSACTION, 1).transaction(null); + } else { + request.removeSetting(SETTING_IMPLICIT_TRANSACTION); + } + } + + private final ClickHouseNode server; + private final String sessionId; + private final int timeout; + private final boolean implicit; + private final AtomicReference id; + private final AtomicInteger state; + + /** + * Constructs a unique transaction in {@link #ACTIVE} state. + * {@link ClickHouseRequestManager#createSessionId()} will be used to ensure + * uniquness of the transaction. + * + * @param server non-null server of the transaction + * @param timeout transaction timeout + * @param implicit whether it's an implicit transaction or not + */ + protected ClickHouseTransaction(ClickHouseNode server, int timeout, boolean implicit) throws ClickHouseException { + this.server = server; + this.sessionId = ClickHouseRequestManager.getInstance().createSessionId(); + this.timeout = timeout < 1 ? 0 : timeout; + this.implicit = implicit; + this.id = new AtomicReference<>(XID.EMPTY); + this.state = new AtomicInteger(NEW); + + try { + id.updateAndGet(x -> { + boolean success = false; + try { + issue("BEGIN TRANSACTION", false, Collections.emptyMap()); + XID txId = XID.of(issue(QUERY_SELECT_TX_ID).getValue(0).asTuple()); + + if (XID.EMPTY.equals(txId)) { + throw new ClickHouseTransactionException( + ClickHouseTransactionException.ERROR_UNKNOWN_STATUS_OF_TRANSACTION, + ClickHouseUtils.format("Failed to start transaction(implicit=%s)", implicit), this); + } + success = state.compareAndSet(NEW, ACTIVE); + return txId; + } catch (ClickHouseException e) { + throw new IllegalStateException(e); + } finally { + if (!success) { + state.compareAndSet(NEW, FAILED); + } + } + }); + log.debug("Began transaction(implicit=%s): %s", this.implicit, this); + } catch (IllegalStateException e) { + if (e.getCause() instanceof ClickHouseException) { + throw (ClickHouseException) e.getCause(); + } else { + throw e; + } + } + } + + /** + * Constructs a transaction in {@link #NEW} state, hence {@link #begin()} or + * {@link #begin(Map)} must be called before commit/rollback and + * {@link #isImplicit()} is always {@code false}. + * + * @param server non-null server of the transaction + * @param sessionId non-empty session ID for the transaction + * @param timeout transaction timeout + * @param id optional transaction ID + */ + protected ClickHouseTransaction(ClickHouseNode server, String sessionId, int timeout, XID id) { + this.server = server; + this.sessionId = sessionId; + this.timeout = timeout < 1 ? 0 : timeout; + this.implicit = false; + if (id == null || XID.EMPTY.equals(id)) { + this.id = new AtomicReference<>(XID.EMPTY); + this.state = new AtomicInteger(NEW); + } else { + this.id = new AtomicReference<>(id); + this.state = new AtomicInteger(ACTIVE); + } + } + + /** + * Ensures client and server are using the exact same transaction ID. + * + * @throws ClickHouseException when transaction ID is inconsistent between + * client and server + */ + protected void ensureTransactionId() throws ClickHouseException { + if (!implicit) { + XID serverTxId = XID.of(issue(QUERY_SELECT_TX_ID).getValue(0).asTuple()); + if (!serverTxId.equals(id.get())) { + throw new ClickHouseTransactionException( + ClickHouseUtils.format( + "Inconsistent transaction ID - client expected %s but found %s on server.", + id.get(), serverTxId), + this); + } + } + } + + /** + * Issues transaction related query. Same as + * {@code issue(command, true, Collections.emptyMap())}. + * + * @param command non-empty transaction related query + * @return non-null record + * @throws ClickHouseException when failed to issue the query + */ + protected final ClickHouseRecord issue(String command) throws ClickHouseException { + return issue(command, true, Collections.emptyMap()); + } + + /** + * Issues transaction related query. + * + * @param command non-empty transaction related query + * @param sessionCheck whether to enable session check + * @param settings optional server settings + * @return non-null record + * @throws ClickHouseException when failed to issue the query + */ + protected ClickHouseRecord issue(String command, boolean sessionCheck, Map settings) + throws ClickHouseException { + ClickHouseRecord result = ClickHouseRecord.EMPTY; + try (ClickHouseResponse response = ClickHouseClient.newInstance(server.getProtocol()).connect(server) + .format(ClickHouseFormat.RowBinaryWithNamesAndTypes) + .settings(settings).session(sessionId, sessionCheck, timeout > 0 ? timeout : null) + .query(command).executeAndWait()) { + Iterator records = response.records().iterator(); + if (records.hasNext()) { + result = records.next(); + } + } catch (ClickHouseException e) { + switch (e.getErrorCode()) { + case ClickHouseException.ERROR_SESSION_NOT_FOUND: + throw new ClickHouseTransactionException( + "Invalid transaction due to session not found or timeed out", e.getCause(), this); + case ClickHouseTransactionException.ERROR_INVALID_TRANSACTION: + case ClickHouseTransactionException.ERROR_UNKNOWN_STATUS_OF_TRANSACTION: + throw new ClickHouseTransactionException(e.getErrorCode(), e.getMessage(), e.getCause(), this); + default: + break; + } + throw e; + } + return result; + } + + /** + * Gets current transaction ID. + * + * @return non-null transaction ID + */ + public XID getId() { + return id.get(); + } + + /** + * Gets server of the transaction. + * + * @return non-null server + */ + public ClickHouseNode getServer() { + return server; + } + + /** + * Gets session id of the transaction. + * + * @return non-empty session id + */ + public String getSessionId() { + return sessionId; + } + + /** + * Gets transaction state, one of {@link #NEW}, {@link #ACTIVE}, + * {@link #COMMITTED}, {@link #ROLLED_BACK}, or {@link #FAILED}. + * + * @return transaction state + */ + public int getState() { + return state.get(); + } + + /** + * Gets transaction timeout in seconds. + * + * @return transaction timeout in seconds, zero or negative number means + * {@code default_session_timeout} as defined on server + */ + public int getTimeout() { + return timeout; + } + + /** + * Checks if the transaction is implicit or not. + * + * @return true if it's an implicit transaction; false otherwise + */ + public boolean isImplicit() { + return implicit; + } + + /** + * Checks whether the transation's state is {@link #NEW}. + * + * @return true if the state is {@link #NEW}; false otherwise + */ + public boolean isNew() { + return state.get() == NEW; + } + + /** + * Checks whether the transation's state is {@link #ACTIVE}. + * + * @return true if the state is {@link #ACTIVE}; false otherwise + */ + public boolean isActive() { + return state.get() == ACTIVE; + } + + /** + * Checks whether the transation's state is {@link #COMMITTED}. + * + * @return true if the state is {@link #COMMITTED}; false otherwise + */ + public boolean isCommitted() { + return state.get() == COMMITTED; + } + + /** + * Checks whether the transation's state is {@link #ROLLED_BACK}. + * + * @return true if the state is {@link #ROLLED_BACK}; false otherwise + */ + public boolean isRolledBack() { + return state.get() == ROLLED_BACK; + } + + /** + * Checks whether the transation's state is {@link #FAILED}. + * + * @return true if the state is {@link #FAILED}; false otherwise + */ + public boolean isFailed() { + return state.get() == FAILED; + } + + /** + * Aborts the transaction. + */ + public void abort() { + log.debug("Abort %s", this); + int currentState = state.get(); + if (currentState == NEW) { + log.debug("Skip since it's a new transaction which hasn't started yet"); + return; + } + + id.updateAndGet(x -> { + try (ClickHouseResponse response = ClickHouseClient.newInstance(server.getProtocol()).connect(server) + .query("KILL TRANSACTION WHERE tid=" + x.asTupleString()).executeAndWait()) { + // ignore + } catch (ClickHouseException e) { + log.warn("Failed to abort transaction %s", x.asTupleString()); + } finally { + state.compareAndSet(currentState, FAILED); + } + return x; + }); + log.debug("Aborted transaction: %s", this); + } + + /** + * Starts a new transaction. Same as {@code begin(Collections.emptyMap())}. + * + * @throws ClickHouseException when failed to begin new transaction + */ + public void begin() throws ClickHouseException { + begin(Collections.emptyMap()); + } + + /** + * Starts a new transaction with optional server settings. It's a no-op when + * calling against an {@link #ACTIVE} transaction. + * + * @param settings optional server settings + * @throws ClickHouseException when failed to begin new transaction + */ + public void begin(Map settings) throws ClickHouseException { + log.debug("Begin %s", this); + int currentState = state.get(); + if (currentState == ACTIVE) { + log.debug("Skip since the transaction has been started already"); + return; + } else if (currentState == FAILED) { + throw new ClickHouseTransactionException( + "Cannot restart a failed transaction - please roll back or create a new transaction", + this); + } + + try { + id.updateAndGet(x -> { + boolean success = false; + XID txId = null; + try { + // reuse existing transaction if any + txId = XID.of(issue(QUERY_SELECT_TX_ID, false, Collections.emptyMap()).getValue(0).asTuple()); + if (XID.EMPTY.equals(txId)) { + issue("BEGIN TRANSACTION", true, settings); + txId = XID.of(issue(QUERY_SELECT_TX_ID).getValue(0).asTuple()); + } + + if (XID.EMPTY.equals(txId)) { + throw new ClickHouseTransactionException( + ClickHouseTransactionException.ERROR_UNKNOWN_STATUS_OF_TRANSACTION, + "Failed to start new transaction", this); + } + success = state.compareAndSet(currentState, ACTIVE); + return txId; + } catch (ClickHouseException e) { + throw new IllegalStateException(e); + } finally { + if (txId != null && !success) { + state.compareAndSet(currentState, FAILED); + } + } + }); + log.debug("Began new transaction: %s", this); + } catch (IllegalStateException e) { + if (e.getCause() instanceof ClickHouseException) { + throw (ClickHouseException) e.getCause(); + } else { + throw e; + } + } + } + + /** + * Commits the transaction. Same as {@code commit(Collections.emptyMap())}. + * + * @throws ClickHouseException when failed to commit the transaction + */ + public void commit() throws ClickHouseException { + commit(Collections.emptyMap()); + } + + /** + * Commits the transaction with optional server settings. It's a no-op when + * calling against a {@link #COMMITTED} transaction. + * + * @param settings optional server settings + * @throws ClickHouseException when failed to commit the transaction + */ + public void commit(Map settings) throws ClickHouseException { + log.debug("Commit %s", this); + int currentState = state.get(); + if (currentState == COMMITTED) { + log.debug("Skip since the transaction has been committed already"); + return; + } else if (currentState != ACTIVE) { + throw new ClickHouseTransactionException( + ClickHouseUtils.format("Cannot commit inactive transaction(state=%s)", NAMES[currentState]), this); + } + + try { + id.updateAndGet(x -> { + boolean success = false; + try { + ensureTransactionId(); + issue("COMMIT", true, settings); + success = state.compareAndSet(currentState, COMMITTED); + return x; + } catch (ClickHouseException e) { + throw new IllegalStateException(e); + } finally { + if (!success) { + state.compareAndSet(currentState, FAILED); + } + } + }); + } catch (IllegalStateException e) { + if (e.getCause() instanceof ClickHouseException) { + throw (ClickHouseException) e.getCause(); + } else { + throw e; + } + } + } + + /** + * Rolls back the transaction. Same as {@code rollback(Collections.emptyMap())}. + * + * @throws ClickHouseException when failed to roll back the transaction + */ + public void rollback() throws ClickHouseException { + rollback(Collections.emptyMap()); + } + + /** + * Rolls back the transaction with optional server settings. It's a no-op when + * calling against a {@link #NEW} or {@link #ROLLED_BACK} transaction. + * + * @param settings optional server settings + * @throws ClickHouseException when failed to roll back the transaction + */ + public void rollback(Map settings) throws ClickHouseException { + log.debug("Roll back %s", this); + int currentState = state.get(); + if (currentState == NEW) { + log.debug("Skip since the transaction has not started yet"); + return; + } else if (currentState == ROLLED_BACK) { + log.debug("Skip since the transaction has been rolled back already"); + return; + } else if (currentState != ACTIVE && currentState != FAILED) { + throw new ClickHouseTransactionException( + ClickHouseUtils.format("Cannot roll back inactive transaction(state=%s)", NAMES[currentState]), + this); + } + + try { + id.updateAndGet(x -> { + boolean success = false; + try { + ensureTransactionId(); + issue("ROLLBACK", true, settings); + success = state.compareAndSet(currentState, ROLLED_BACK); + return x; + } catch (ClickHouseException e) { + throw new IllegalStateException(e); + } finally { + if (!success) { + state.compareAndSet(currentState, FAILED); + } + } + }); + } catch (IllegalStateException e) { + if (e.getCause() instanceof ClickHouseException) { + throw (ClickHouseException) e.getCause(); + } else { + throw e; + } + } + } + + /** + * Sets transaction snapshot with optional server settings. Same as + * {@code snapshot(snapshotVersion, Collections.emptyMap())}. + * + * @param snapshotVersion snapshot version + * @throws ClickHouseException when failed to set transaction snapshot + */ + public void snapshot(long snapshotVersion) throws ClickHouseException { + snapshot(snapshotVersion, Collections.emptyMap()); + } + + /** + * Sets transaction snapshot with optional server settings, only works for + * {@link #ACTIVE} transaction. Use {@code snapshot(CSN_EVERYTHING_VISIBLE)} to + * read uncommitted data. + * + * @param snapshotVersion snapshot version + * @param settings optional server settings + * @throws ClickHouseException when failed to set transaction snapshot + */ + public void snapshot(long snapshotVersion, Map settings) throws ClickHouseException { + log.debug("Set snapshot %d for %s", snapshotVersion, this); + int currentState = state.get(); + if (currentState != ACTIVE) { + throw new ClickHouseTransactionException( + ClickHouseUtils.format("Cannot set snapshot version for inactive transaction(state=%s)", + NAMES[currentState]), + this); + } + + try { + id.updateAndGet(x -> { + boolean success = false; + try { + ensureTransactionId(); + issue("SET TRANSACTION SNAPSHOT " + snapshotVersion, true, settings); + success = true; + return x; + } catch (ClickHouseException e) { + throw new IllegalStateException(e); + } finally { + if (!success) { + state.compareAndSet(currentState, FAILED); + } + } + }); + } catch (IllegalStateException e) { + if (e.getCause() instanceof ClickHouseException) { + throw (ClickHouseException) e.getCause(); + } else { + throw e; + } + } + } + + @Override + public int hashCode() { + final int prime = 31; + int result = prime + server.getBaseUri().hashCode(); + result = prime * result + sessionId.hashCode(); + result = prime * result + timeout; + result = prime * result + id.get().hashCode(); + result = prime * result + state.get(); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } else if (obj == null || getClass() != obj.getClass()) { + return false; + } + + ClickHouseTransaction other = (ClickHouseTransaction) obj; + return server.isSameEndpoint(other.server) && sessionId.equals(other.sessionId) + && timeout == other.timeout && id.get().equals(other.id.get()) && state.get() == other.state.get(); + } + + @Override + public String toString() { + return new StringBuilder().append("ClickHouseTransaction [id=").append(id.get().asTupleString()) + .append(", session=").append(sessionId).append(", timeout=").append(timeout).append(", state=") + .append(NAMES[state.get()]).append(", server=").append(server.getBaseUri()).append("]@") + .append(hashCode()).toString(); + } +} diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseTransactionException.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseTransactionException.java new file mode 100644 index 000000000..f71fae940 --- /dev/null +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseTransactionException.java @@ -0,0 +1,30 @@ +package com.clickhouse.client; + +public class ClickHouseTransactionException extends ClickHouseException { + public static final int ERROR_INVALID_TRANSACTION = 649; + public static final int ERROR_UNKNOWN_STATUS_OF_TRANSACTION = 659; + + private final ClickHouseTransaction tx; + + public ClickHouseTransactionException(String message, ClickHouseTransaction tx) { + this(ERROR_INVALID_TRANSACTION, message, tx); + } + + public ClickHouseTransactionException(String message, Throwable cause, ClickHouseTransaction tx) { + this(ERROR_INVALID_TRANSACTION, message, cause, tx); + } + + public ClickHouseTransactionException(int code, String message, Throwable cause, ClickHouseTransaction tx) { + super(code, message, cause); + this.tx = tx; + } + + public ClickHouseTransactionException(int code, String message, ClickHouseTransaction tx) { + super(code, message, tx.getServer()); + this.tx = tx; + } + + public ClickHouseTransaction getTransaction() { + return tx; + } +} diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/config/ClickHouseClientOption.java b/clickhouse-client/src/main/java/com/clickhouse/client/config/ClickHouseClientOption.java index e9d06cbc6..623bc8e2b 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/config/ClickHouseClientOption.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/config/ClickHouseClientOption.java @@ -215,6 +215,12 @@ public enum ClickHouseClientOption implements ClickHouseOption { */ RETRY("retry", 0, "Maximum number of times retry can happen for a request, zero or negative value means no retry."), + /** + * Whether to repeat execution when session is locked, until timed out(according + * to {@link #SESSION_TIMEOUT} or {@link #CONNECTION_TIMEOUT}). + */ + REPEAT_ON_SESSION_LOCK("repeat_on_session_lock", true, + "Whether to repeat execution when session is locked, until timed out(according to 'session_timeout' or 'connect_timeout')."), /** * Whether to reuse wrapper of value(e.g. ClickHouseValue or * ClickHouseRecord) for memory efficiency. @@ -249,10 +255,10 @@ public enum ClickHouseClientOption implements ClickHouseOption { */ SESSION_CHECK("session_check", false, "Whether to check if existence of session id."), /** - * Session timeout in milliseconds. + * Session timeout in seconds. */ SESSION_TIMEOUT("session_timeout", 0, - "Session timeout in milliseconds. 0 or negative number means same as server default."), + "Session timeout in seconds. 0 or negative number means same as server default."), /** * Socket timeout in milliseconds. */ @@ -278,6 +284,11 @@ public enum ClickHouseClientOption implements ClickHouseOption { * SSL key. */ SSL_KEY("sslkey", "", "RSA key in PKCS#8 format."), + /** + * Transaction timeout in seconds. + */ + TRANSACTION_TIMEOUT("transaction_timeout", 0, + "Transaction timeout in seconds. 0 or negative number means same as session_timeout."), /** * Whether to use blocking queue for buffering. */ 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 a1f5b94e1..4b2a2fabe 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 @@ -2,6 +2,7 @@ import java.io.FileNotFoundException; import java.io.InputStream; +import java.io.Serializable; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -22,7 +23,7 @@ /** * "Attached" temporary table. */ -public class ClickHouseExternalTable { +public class ClickHouseExternalTable implements Serializable { public static class Builder { private String name; private ClickHouseFile file; @@ -164,6 +165,8 @@ public static Builder builder() { return new Builder(); } + private static final long serialVersionUID = -5395148151046691946L; + private final String name; private final ClickHouseFile file; private final ClickHouseDeferredValue content; diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseStreamResponse.java b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseStreamResponse.java index 2f6e2a85c..582fa108c 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseStreamResponse.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseStreamResponse.java @@ -84,13 +84,13 @@ protected ClickHouseStreamResponse(ClickHouseConfig config, ClickHouseInputStrea log.error("Failed to create stream response, closing input stream"); try { input.close(); - } catch (Exception e) { + } catch (IOException e) { // ignore } } } - this.summary = summary != null ? summary : ClickHouseResponseSummary.EMPTY; this.closed = hasError; + this.summary = summary != null ? summary : ClickHouseResponseSummary.EMPTY; } @Override @@ -115,10 +115,11 @@ public void close() { } finally { try { input.close(); - } catch (Exception e) { + } catch (IOException e) { log.warn("Failed to close input stream", e); + } finally { + closed = true; } - closed = true; } } diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseExceptionTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseExceptionTest.java index eb29819b3..daca27c01 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseExceptionTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseExceptionTest.java @@ -35,7 +35,7 @@ public void testConstructorWithCause() { @Test(groups = { "unit" }) public void testConstructorWithoutCause() { - ClickHouseException e = new ClickHouseException(-1, (String) null, null); + ClickHouseException e = new ClickHouseException(-1, (String) null, (ClickHouseNode) null); Assert.assertEquals(e.getErrorCode(), -1); Assert.assertNull(e.getCause()); Assert.assertEquals(e.getMessage(), "Unknown error -1"); 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 fd502595a..b825b51d4 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseRequestTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseRequestTest.java @@ -73,7 +73,7 @@ public void testBuild() { } @Test(groups = { "unit" }) - public void testChangeListener() { + public void testConfigChangeListener() { final ClickHouseConfig config = new ClickHouseConfig(); final List changedOptions = new ArrayList<>(); final List changedProperties = new ArrayList<>(); @@ -108,7 +108,9 @@ public void settingChanged(ClickHouseRequest source, String setting, Serializ request.option(ClickHouseClientOption.FORMAT, ClickHouseFormat.Avro); request.removeOption(ClickHouseClientOption.BUFFER_SIZE); request.removeOption(ClickHouseClientOption.ASYNC); - request.query("select 1").query("select 2", "id=2").query(select3); + request.query("select 1"); + request.query("select 2", "id=2"); + request.query(select3); request.reset(); request.format(ClickHouseFormat.TSV); Assert.assertEquals(changedOptions.toArray(new Object[0]), @@ -126,6 +128,8 @@ public void settingChanged(ClickHouseRequest source, String setting, Serializ { request, ClickHouseRequest.PROP_PREPARED_QUERY, null, select3 }, { request, ClickHouseRequest.PROP_QUERY, "select 2", "select 3" }, { request, ClickHouseRequest.PROP_QUERY_ID, "id=2", null }, + { request, ClickHouseRequest.PROP_QUERY, "select 3", null }, + { request, ClickHouseRequest.PROP_PREPARED_QUERY, select3, null }, }); changedOptions.clear(); @@ -147,10 +151,33 @@ public void settingChanged(ClickHouseRequest source, String setting, Serializ changedSettings.clear(); request.setChangeListener(listener); - Assert.assertNull(request.copy().changeListener, "Listener should never be copied"); + Assert.assertEquals(request.copy().changeListener, request.changeListener); Assert.assertNull(request.seal().changeListener, "Listener should never be copied"); } + @Test(groups = { "unit" }) + public void testServerListener() { + ClickHouseRequest request = ClickHouseClient.newInstance().connect(ClickHouseNode.builder().build()); + final List serverChanges = new ArrayList<>(); + request.setServerListener( + (currentServer, newServer) -> serverChanges.add(new Object[] { currentServer, newServer })); + ClickHouseNode s11 = ClickHouseNode.of("http://node1"); + ClickHouseNode s12 = ClickHouseNode.of("grpc://node1/system"); + ClickHouseNode s21 = ClickHouseNode.of("tcp://node2"); + ClickHouseNode s22 = ClickHouseNode.of("https://node2"); + request.changeServer(request.getServer(), s11); + Assert.assertEquals(serverChanges.toArray(new Object[0]), new Object[][] { { ClickHouseNode.DEFAULT, s11 } }); + request.changeServer(ClickHouseNode.DEFAULT, s12); + Assert.assertEquals(serverChanges.toArray(new Object[0]), new Object[][] { { ClickHouseNode.DEFAULT, s11 } }); + request.changeServer(s11, s21); + Assert.assertEquals(serverChanges.toArray(new Object[0]), + new Object[][] { { ClickHouseNode.DEFAULT, s11 }, { s11, s21 } }); + request.reset(); + request.changeServer(s21, s22); + Assert.assertEquals(serverChanges.toArray(new Object[0]), + new Object[][] { { ClickHouseNode.DEFAULT, s11 }, { s11, s21 } }); + } + @Test(groups = { "unit" }) public void testCopy() { ClickHouseRequest request = ClickHouseClient.newInstance().connect(ClickHouseNode.builder().build()); diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/ClientIntegrationTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/ClientIntegrationTest.java index 4635b3381..c5330ae78 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/ClientIntegrationTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/ClientIntegrationTest.java @@ -33,6 +33,7 @@ import java.util.zip.GZIPOutputStream; import com.clickhouse.client.ClickHouseClientBuilder.Agent; +import com.clickhouse.client.ClickHouseTransaction.XID; import com.clickhouse.client.config.ClickHouseBufferingMode; import com.clickhouse.client.config.ClickHouseClientOption; import com.clickhouse.client.config.ClickHouseRenameMethod; @@ -57,6 +58,29 @@ import org.testng.annotations.Test; public abstract class ClientIntegrationTest extends BaseIntegrationTest { + protected void checkRowCount(String queryOrTableName, int expectedRowCount) throws ClickHouseException { + try (ClickHouseClient client = getClient()) { + checkRowCount(client.connect(getServer()).format(ClickHouseFormat.RowBinaryWithNamesAndTypes), + queryOrTableName, expectedRowCount); + } + } + + protected void checkRowCount(ClickHouseRequest request, String queryOrTableName, int expectedRowCount) + throws ClickHouseException { + String sql = queryOrTableName.indexOf(' ') > 0 ? queryOrTableName + : "select count(1) from ".concat(queryOrTableName); + try (ClickHouseResponse response = request.query(sql).executeAndWait()) { + int count = 0; + for (ClickHouseRecord r : response.records()) { + if (count == 0) { + Assert.assertEquals(r.getValue(0).asInteger(), expectedRowCount); + } + count++; + } + Assert.assertEquals(count, 1); + } + } + protected ClickHouseResponseSummary execute(ClickHouseRequest request, String sql) throws ClickHouseException { try (ClickHouseResponse response = request.query(sql).executeAndWait()) { for (ClickHouseRecord record : response.records()) { @@ -242,12 +266,23 @@ public void testCompression(ClickHouseFormat format, ClickHouseBufferingMode buf .executeAndWait()) { Assert.assertNotNull(resp); } + int expectedRows = 1; + if (server.getProtocol() != ClickHouseProtocol.GRPC) { + try (ClickHouseResponse resp = request.write().table("test_compress_decompress") + .format(ClickHouseFormat.CSV).data(ClickHouseInputStream.of("'" + uuid + "'\n'" + uuid + "'")) + .executeAndWait()) { + Assert.assertNotNull(resp); + } + expectedRows += 2; + } boolean hasResult = false; try (ClickHouseResponse resp = request - .query("select id from test_compress_decompress where id = :uuid") + .query("select id, count(1) n from test_compress_decompress where id = :uuid group by id") .params(ClickHouseStringValue.of(uuid)).executeAndWait()) { - Assert.assertEquals(resp.firstRecord().getValue(0).asString(), uuid); + ClickHouseRecord r = resp.firstRecord(); + Assert.assertEquals(r.getValue(0).asString(), uuid); + Assert.assertEquals(r.getValue(1).asInteger(), expectedRows); hasResult = true; } Assert.assertTrue(hasResult, "Should have at least one result"); @@ -1382,20 +1417,22 @@ public void testTempTable() throws Exception { @Test(groups = "integration") public void testErrorDuringInsert() throws Exception { ClickHouseNode server = getServer(); - if (server.getProtocol() != ClickHouseProtocol.HTTP) { - throw new SkipException("Skip as only http implementation works well"); - } ClickHouseClient.send(server, "drop table if exists error_during_insert", "create table error_during_insert(n UInt64, flag UInt8)engine=Null").get(); boolean success = true; try (ClickHouseClient client = getClient(); - ClickHouseResponse resp = client.connect(getServer()).write().format(ClickHouseFormat.RowBinary) + ClickHouseResponse resp = client.connect(server).write().format(ClickHouseFormat.RowBinary) .query("insert into error_during_insert select number, throwIf(number>=100000000) from numbers(500000000)") .executeAndWait()) { for (ClickHouseRecord r : resp.records()) { Assert.fail("Should have no record"); } Assert.fail("Insert should be aborted"); + } catch (UncheckedIOException e) { + ClickHouseException ex = ClickHouseException.of(e, server); + Assert.assertEquals(ex.getErrorCode(), 395); + Assert.assertTrue(ex.getCause() instanceof IOException, "Should end up with IOException"); + success = false; } catch (ClickHouseException e) { Assert.assertEquals(e.getErrorCode(), 395); Assert.assertTrue(e.getCause() instanceof IOException, "Should end up with IOException"); @@ -1408,9 +1445,6 @@ public void testErrorDuringInsert() throws Exception { @Test(groups = "integration") public void testErrorDuringQuery() throws Exception { ClickHouseNode server = getServer(); - if (server.getProtocol() != ClickHouseProtocol.HTTP) { - throw new SkipException("Skip as only http implementation works well"); - } String query = "select number, throwIf(number>=100000000) from numbers(500000000)"; long count = 0L; try (ClickHouseClient client = getClient(); @@ -1430,4 +1464,403 @@ public void testErrorDuringQuery() throws Exception { Assert.assertNotEquals(count, 0L, "Should have read at least one record"); } + + @Test(groups = "integration") + public void testSessionLock() throws Exception { + ClickHouseNode server = getServer(); + String sessionId = ClickHouseRequestManager.getInstance().createSessionId(); + try (ClickHouseClient client = getClient()) { + ClickHouseRequest req1 = client.connect(server).session(sessionId) + .query("select * from numbers(10000000)"); + ClickHouseRequest req2 = client.connect(server) + .option(ClickHouseClientOption.REPEAT_ON_SESSION_LOCK, true) + .option(ClickHouseClientOption.CONNECTION_TIMEOUT, 500) + .session(sessionId).query("select 1"); + + ClickHouseResponse resp1 = req1.executeAndWait(); + try (ClickHouseResponse resp = req2.executeAndWait()) { + Assert.fail("Should fail due to session is locked by previous query"); + } catch (ClickHouseException e) { + Assert.assertEquals(e.getErrorCode(), ClickHouseException.ERROR_SESSION_IS_LOCKED); + } + new Thread(() -> { + try { + Thread.sleep(1000); + resp1.close(); + } catch (InterruptedException e) { + // ignore + } + }).start(); + + try (ClickHouseResponse resp = req2.option(ClickHouseClientOption.CONNECTION_TIMEOUT, 30000) + .executeAndWait()) { + Assert.assertNotNull(resp); + } + } + } + + @Test // (groups = "integration") + public void testAbortTransaction() throws Exception { + ClickHouseNode server = getServer(); + String tableName = "test_abort_transaction"; + ClickHouseClient.send(server, "drop table if exists " + tableName, + "create table " + tableName + " (id Int64)engine=MergeTree order by id").get(); + try (ClickHouseClient client = getClient()) { + ClickHouseRequest txRequest = client.connect(server).transaction(); + try (ClickHouseResponse response = txRequest.query("insert into " + tableName + " values(1)(2)(3)") + .executeAndWait()) { + // ignore + } + checkRowCount(txRequest, tableName, 3); + checkRowCount(tableName, 3); + Assert.assertEquals(txRequest.getTransaction().getState(), ClickHouseTransaction.ACTIVE); + + txRequest.getTransaction().abort(); + Assert.assertEquals(txRequest.getTransaction().getState(), ClickHouseTransaction.FAILED); + checkRowCount(tableName, 0); + + try { + checkRowCount(txRequest, tableName, 0); + Assert.fail("Should fail as the transaction is invalid"); + } catch (ClickHouseException e) { + Assert.assertEquals(e.getErrorCode(), ClickHouseTransactionException.ERROR_INVALID_TRANSACTION); + } + } + } + + @Test // (groups = "integration") + public void testNewTransaction() throws ClickHouseException { + ClickHouseNode server = getServer(); + try (ClickHouseClient client = getClient()) { + ClickHouseRequest request = client.connect(server); + Assert.assertNull(request.getSessionId().orElse(null), "Should have no session"); + Assert.assertNull(request.getTransaction(), "Should have no transaction"); + + request.transaction(); + Assert.assertNotNull(request.getSessionId().orElse(null), "Should have session now"); + ClickHouseTransaction tx = request.getTransaction(); + Assert.assertNotNull(tx, "Should have transaction now"); + Assert.assertEquals(tx.getSessionId(), request.getSessionId().orElse(null)); + Assert.assertEquals(tx.getServer(), server); + Assert.assertEquals(tx.getState(), ClickHouseTransaction.ACTIVE); + Assert.assertNotEquals(tx.getId(), XID.EMPTY); + + request.transaction(0); // current transaction should be reused + Assert.assertEquals(request.getTransaction(), tx); + Assert.assertEquals(ClickHouseRequestManager.getInstance().getOrStartTransaction(request, 0), tx); + Assert.assertNotEquals(ClickHouseRequestManager.getInstance().createTransaction(server, 0), tx); + + request.transaction(30); // same transaction ID but with different timeout settings + Assert.assertNotEquals(request.getTransaction(), tx); + Assert.assertEquals(request.getTransaction().getId().getSnapshotVersion(), tx.getId().getSnapshotVersion()); + Assert.assertEquals(request.getTransaction().getId().getHostId(), tx.getId().getHostId()); + Assert.assertNotEquals(request.getTransaction().getId().getLocalTransactionCounter(), + tx.getId().getLocalTransactionCounter()); + Assert.assertNotEquals(request.getTransaction().getSessionId(), tx.getSessionId()); + + request.transaction(0); + Assert.assertNotEquals(request.getTransaction(), tx); + + ClickHouseRequest otherRequest = client.connect(server).transaction(tx); + Assert.assertEquals(otherRequest.getSessionId().orElse(null), tx.getSessionId()); + Assert.assertEquals(otherRequest.getTransaction(), tx); + } + } + + @Test // (groups = "integration") + public void testJoinTransaction() throws ClickHouseException { + ClickHouseNode server = getServer(); + try (ClickHouseClient client = getClient()) { + ClickHouseRequest request = client.connect(server).transaction(); + ClickHouseTransaction tx = request.getTransaction(); + + ClickHouseRequest otherRequest = client.connect(server).transaction(tx); + Assert.assertEquals(otherRequest.getSessionId().orElse(null), request.getSessionId().orElse(null)); + Assert.assertEquals(otherRequest.getTransaction(), request.getTransaction()); + + ClickHouseTransaction newTx = ClickHouseRequestManager.getInstance().createTransaction(server, 0); + Assert.assertNotEquals(newTx, XID.EMPTY); + Assert.assertNotEquals(tx, newTx); + Assert.assertEquals(newTx.getState(), ClickHouseTransaction.NEW); + + // now replace the existing transaction to the new one + request.transaction(newTx); + Assert.assertEquals(request.getTransaction(), newTx); + Assert.assertNotEquals(request.getSessionId().orElse(null), otherRequest.getSessionId().orElse(null)); + Assert.assertNotEquals(request.getTransaction(), otherRequest.getTransaction()); + } + } + + @Test // (groups = "integration") + public void testCommitTransaction() throws Exception { + ClickHouseNode server = getServer(); + ClickHouseClient.send(server, "drop table if exists test_tx_commit", + "create table test_tx_commit(a Int64, b String)engine=MergeTree order by a").get(); + try (ClickHouseClient client = getClient()) { + ClickHouseRequest request = client.connect(server).transaction(); + ClickHouseTransaction tx = request.getTransaction(); + + ClickHouseRequest otherRequest = client.connect(server).transaction(tx); + Assert.assertEquals(otherRequest.getSessionId().orElse(null), request.getSessionId().orElse(null)); + Assert.assertEquals(otherRequest.getTransaction(), request.getTransaction()); + + ClickHouseTransaction newTx = ClickHouseRequestManager.getInstance().createTransaction(server, 0); + Assert.assertNotEquals(newTx, XID.EMPTY); + Assert.assertNotEquals(tx, newTx); + Assert.assertEquals(newTx.getState(), ClickHouseTransaction.NEW); + + // now replace the existing transaction to the new one + request.transaction(newTx); + Assert.assertEquals(request.getTransaction(), newTx); + Assert.assertNotEquals(request.getSessionId().orElse(null), otherRequest.getSessionId().orElse(null)); + Assert.assertNotEquals(request.getTransaction(), otherRequest.getTransaction()); + } + } + + @Test // (groups = "integration") + public void testRollbackTransaction() throws Exception { + String tableName = "test_tx_rollback"; + ClickHouseNode server = getServer(); + ClickHouseClient.send(server, "drop table if exists " + tableName, + "create table " + tableName + "(a Int64, b String)engine=MergeTree order by a").get(); + + checkRowCount(tableName, 0); + try (ClickHouseClient client = getClient()) { + ClickHouseRequest request = client.connect(server).transaction(); + ClickHouseTransaction tx = request.getTransaction(); + try (ClickHouseResponse response = client.connect(server) + .query("insert into " + tableName + " values(0, '?')").executeAndWait()) { + // ignore + } + int rows = 1; + checkRowCount(tableName, rows); + checkRowCount(request, tableName, rows); + + try (ClickHouseResponse response = request + .query("insert into " + tableName + " values(1,'x')(2,'y')(3,'z')") + .executeAndWait()) { + // ignore + } + rows += 3; + + checkRowCount(request, tableName, rows); + ClickHouseRequest otherRequest = client.connect(server).transaction(tx); + checkRowCount(otherRequest, tableName, rows); + checkRowCount(tableName, rows); + + try (ClickHouseResponse response = client.connect(server) + .query("insert into " + tableName + " values(-1, '?')").executeAndWait()) { + // ignore + } + rows++; + + checkRowCount(request, tableName, rows); + checkRowCount(otherRequest, tableName, rows); + checkRowCount(tableName, rows); + + try (ClickHouseResponse response = otherRequest.query("insert into " + tableName + " values(4,'.')") + .executeAndWait()) { + // ignore + } + rows++; + + checkRowCount(request, tableName, rows); + checkRowCount(otherRequest, tableName, rows); + checkRowCount(tableName, rows); + + rows -= 4; + for (int i = 0; i < 10; i++) { + tx.rollback(); + checkRowCount(tableName, rows); + checkRowCount(otherRequest, tableName, rows); + checkRowCount(request, tableName, rows); + } + } + } + + @Test // (groups = "integration") + public void testTransactionSnapshot() throws Exception { + String tableName = "test_tx_snapshots"; + ClickHouseNode server = getServer(); + ClickHouseClient.send(server, "drop table if exists " + tableName, + "create table " + tableName + "(a Int64)engine=MergeTree order by a").get(); + try (ClickHouseClient client = getClient()) { + ClickHouseRequest req1 = client.connect(server).transaction(); + ClickHouseRequest req2 = client.connect(server).transaction(); + try (ClickHouseResponse response = req1.query("insert into " + tableName + " values(1)").executeAndWait()) { + // ignore + } + req2.getTransaction().snapshot(1); + checkRowCount(tableName, 1); + checkRowCount(req1, tableName, 1); + checkRowCount(req2, tableName, 0); + try (ClickHouseResponse response = req2.query("insert into " + tableName + " values(2)").executeAndWait()) { + // ignore + } + checkRowCount(tableName, 2); + checkRowCount(req1, tableName, 1); + checkRowCount(req2, tableName, 1); + + req1.getTransaction().snapshot(1); + try (ClickHouseResponse response = req1.query("insert into " + tableName + " values(3)").executeAndWait()) { + // ignore + } + checkRowCount(tableName, 3); + checkRowCount(req1, tableName, 2); + checkRowCount(req2, tableName, 1); + + try (ClickHouseResponse response = req2.query("insert into " + tableName + " values(4)").executeAndWait()) { + // ignore + } + checkRowCount(tableName, 4); + checkRowCount(req1, tableName, 2); + checkRowCount(req2, tableName, 2); + + req2.getTransaction().snapshot(3); + checkRowCount(tableName, 4); + checkRowCount(req1, tableName, 2); + checkRowCount(req2, tableName, 4); + + req1.getTransaction().snapshot(3); + checkRowCount(tableName, 4); + checkRowCount(req1, tableName, 4); + checkRowCount(req2, tableName, 4); + + req1.getTransaction().snapshot(1); + try (ClickHouseResponse response = req1.query("insert into " + tableName + " values(5)").executeAndWait()) { + // ignore + } + checkRowCount(tableName, 5); + checkRowCount(req1, tableName, 3); + checkRowCount(req2, tableName, 5); + + req2.getTransaction().commit(); + checkRowCount(tableName, 5); + checkRowCount(req1, tableName, 3); + checkRowCount(req2, tableName, 5); + try { + req2.getTransaction().snapshot(5); + } catch (ClickHouseTransactionException e) { + Assert.assertEquals(e.getErrorCode(), ClickHouseTransactionException.ERROR_INVALID_TRANSACTION); + } + + req1.getTransaction().commit(); + checkRowCount(tableName, 5); + checkRowCount(req1, tableName, 5); + checkRowCount(req2, tableName, 5); + try { + req1.getTransaction().snapshot(5); + } catch (ClickHouseTransactionException e) { + Assert.assertEquals(e.getErrorCode(), ClickHouseTransactionException.ERROR_INVALID_TRANSACTION); + } + } + } + + @Test // (groups = "integration") + public void testTransactionTimeout() throws Exception { + String tableName = "test_tx_timeout"; + ClickHouseNode server = getServer(); + ClickHouseClient.send(server, "drop table if exists " + tableName, + "create table " + tableName + "(a UInt64)engine=MergeTree order by a").get(); + try (ClickHouseClient client = getClient()) { + ClickHouseRequest request = client.connect(server).transaction(1); + ClickHouseTransaction tx = request.getTransaction(); + Assert.assertEquals(tx.getState(), ClickHouseTransaction.ACTIVE); + tx.commit(); + Assert.assertEquals(tx.getState(), ClickHouseTransaction.COMMITTED); + + tx.begin(); + Assert.assertEquals(tx.getState(), ClickHouseTransaction.ACTIVE); + tx.rollback(); + Assert.assertEquals(tx.getState(), ClickHouseTransaction.ROLLED_BACK); + + tx.begin(); + Thread.sleep(3000L); + try (ClickHouseResponse response = client.connect(server).transaction(tx).query("select 1") + .executeAndWait()) { + Assert.fail("Query should fail due to session timed out"); + } catch (ClickHouseException e) { + // session not found(since it's timed out) + Assert.assertEquals(e.getErrorCode(), ClickHouseException.ERROR_SESSION_NOT_FOUND); + } + Assert.assertEquals(tx.getState(), ClickHouseTransaction.ACTIVE); + + try { + tx.commit(); + Assert.fail("Should fail to commit due to session timed out"); + } catch (ClickHouseTransactionException e) { + Assert.assertEquals(e.getErrorCode(), ClickHouseTransactionException.ERROR_INVALID_TRANSACTION); + } + Assert.assertEquals(tx.getState(), ClickHouseTransaction.FAILED); + + try { + tx.rollback(); + Assert.fail("Should fail to roll back due to session timed out"); + } catch (ClickHouseTransactionException e) { + Assert.assertEquals(e.getErrorCode(), ClickHouseTransactionException.ERROR_INVALID_TRANSACTION); + } + Assert.assertEquals(tx.getState(), ClickHouseTransaction.FAILED); + + try { + tx.begin(); + Assert.fail("Should fail to restart due to session timed out"); + } catch (ClickHouseTransactionException e) { + Assert.assertEquals(e.getErrorCode(), ClickHouseTransactionException.ERROR_INVALID_TRANSACTION); + } + Assert.assertEquals(tx.getState(), ClickHouseTransaction.FAILED); + + request.transaction(null); + Assert.assertNull(request.getTransaction(), "Should have no transaction"); + checkRowCount(tableName, 0); + request.transaction(1); + try (ClickHouseResponse response = request.write().query("insert into " + tableName + " values(1)(2)(3)") + .executeAndWait()) { + // ignore + } + Assert.assertEquals(request.getTransaction().getState(), ClickHouseTransaction.ACTIVE); + checkRowCount(tableName, 3); + checkRowCount(request, tableName, 3); + Thread.sleep(3000L); + checkRowCount(tableName, 0); + try { + checkRowCount(request, tableName, 3); + Assert.fail("Should fail to query due to session timed out"); + } catch (ClickHouseException e) { + Assert.assertEquals(e.getErrorCode(), ClickHouseException.ERROR_SESSION_NOT_FOUND); + } + Assert.assertEquals(request.getTransaction().getState(), ClickHouseTransaction.ACTIVE); + } + } + + @Test // (groups = "integration") + public void testImplicitTransaction() throws Exception { + ClickHouseNode server = getServer(); + String tableName = "test_implicit_transaction"; + ClickHouseClient.send(server, "drop table if exists " + tableName, + "create table " + tableName + " (id Int64)engine=MergeTree order by id").get(); + try (ClickHouseClient client = getClient()) { + ClickHouseRequest request = client.connect(server); + ClickHouseTransaction.setImplicitTransaction(request, true); + try (ClickHouseResponse response = request.query("insert into " + tableName + " values(1)") + .executeAndWait()) { + // ignore + } + checkRowCount(tableName, 1); + ClickHouseTransaction.setImplicitTransaction(request, false); + try (ClickHouseResponse response = request.query("insert into " + tableName + " values(2)") + .executeAndWait()) { + // ignore + } + checkRowCount(tableName, 2); + + ClickHouseTransaction.setImplicitTransaction(request, true); + try (ClickHouseResponse response = request.transaction().query("insert into " + tableName + " values(3)") + .executeAndWait()) { + // ignore + } + checkRowCount(tableName, 3); + request.getTransaction().rollback(); + checkRowCount(tableName, 2); + } + } } diff --git a/clickhouse-grpc-client/src/main/java/com/clickhouse/client/grpc/ClickHouseGrpcResponse.java b/clickhouse-grpc-client/src/main/java/com/clickhouse/client/grpc/ClickHouseGrpcResponse.java index 54185fcac..e800dd191 100644 --- a/clickhouse-grpc-client/src/main/java/com/clickhouse/client/grpc/ClickHouseGrpcResponse.java +++ b/clickhouse-grpc-client/src/main/java/com/clickhouse/client/grpc/ClickHouseGrpcResponse.java @@ -2,11 +2,13 @@ import java.io.IOException; import java.io.InputStream; +import java.io.UncheckedIOException; import java.util.Map; import com.clickhouse.client.ClickHouseCompression; import com.clickhouse.client.ClickHouseConfig; import com.clickhouse.client.ClickHouseDeferredValue; +import com.clickhouse.client.ClickHouseException; import com.clickhouse.client.ClickHouseInputStream; import com.clickhouse.client.ClickHouseResponseSummary; import com.clickhouse.client.data.ClickHouseStreamResponse; @@ -20,7 +22,15 @@ public class ClickHouseGrpcResponse extends ClickHouseStreamResponse { private final ClickHouseStreamObserver observer; private final Result result; - static ClickHouseInputStream getInput(ClickHouseConfig config, InputStream input) { + static void checkError(Result result) { + if (result != null && result.hasException()) { + throw new UncheckedIOException(new IOException(ClickHouseException + .buildErrorMessage(result.getException().getCode(), + result.getException().getDisplayText()))); + } + } + + static ClickHouseInputStream getInput(ClickHouseConfig config, InputStream input, Runnable postCloseAction) { final ClickHouseInputStream in; if (config.getResponseCompressAlgorithm() == ClickHouseCompression.LZ4) { in = ClickHouseInputStream.of( @@ -31,10 +41,10 @@ static ClickHouseInputStream getInput(ClickHouseConfig config, InputStream input return input; } }), - config.getReadBufferSize(), null); + config.getReadBufferSize(), postCloseAction); } else { in = ClickHouseInputStream.of(input, config.getReadBufferSize(), config.getResponseCompressAlgorithm(), - null); + postCloseAction); } return in; @@ -51,8 +61,10 @@ protected ClickHouseGrpcResponse(ClickHouseConfig config, Map se protected ClickHouseGrpcResponse(ClickHouseConfig config, Map settings, Result result) throws IOException { super(config, - result.getOutput().isEmpty() ? ClickHouseInputStream.of(result.getOutput().newInput()) - : getInput(config, result.getOutput().newInput()), + result.getOutput().isEmpty() + ? ClickHouseInputStream.of(result.getOutput().newInput(), config.getReadBufferSize(), + () -> checkError(result)) + : getInput(config, result.getOutput().newInput(), () -> checkError(result)), settings, null, new ClickHouseResponseSummary(null, null)); diff --git a/clickhouse-grpc-client/src/main/java/com/clickhouse/client/grpc/ClickHouseStreamObserver.java b/clickhouse-grpc-client/src/main/java/com/clickhouse/client/grpc/ClickHouseStreamObserver.java index f1b19f9dd..fedb8a124 100644 --- a/clickhouse-grpc-client/src/main/java/com/clickhouse/client/grpc/ClickHouseStreamObserver.java +++ b/clickhouse-grpc-client/src/main/java/com/clickhouse/client/grpc/ClickHouseStreamObserver.java @@ -1,11 +1,16 @@ package com.clickhouse.client.grpc; import java.io.IOException; +import java.io.UncheckedIOException; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + import io.grpc.Status; import io.grpc.StatusException; import io.grpc.stub.StreamObserver; + +import com.clickhouse.client.ClickHouseCompression; import com.clickhouse.client.ClickHouseConfig; import com.clickhouse.client.ClickHouseDataStreamFactory; import com.clickhouse.client.ClickHouseException; @@ -36,7 +41,7 @@ public class ClickHouseStreamObserver implements StreamObserver { private final ClickHouseResponseSummary summary; - private Throwable error; + private final AtomicReference errorRef; protected ClickHouseStreamObserver(ClickHouseConfig config, ClickHouseNode server, ClickHouseOutputStream output) { this.server = server; @@ -44,19 +49,26 @@ protected ClickHouseStreamObserver(ClickHouseConfig config, ClickHouseNode serve this.startLatch = new CountDownLatch(1); this.finishLatch = new CountDownLatch(1); + Runnable postCloseAction = () -> { + IOException exp = getError(); + if (exp != null) { + throw new UncheckedIOException(exp); + } + }; if (output != null) { this.stream = output; - this.input = ClickHouseInputStream.empty(); + this.input = ClickHouseInputStream.wrap(null, ClickHouseInputStream.empty(), + config.getReadBufferSize(), postCloseAction, ClickHouseCompression.NONE, 0); } else { ClickHousePipedOutputStream pipedStream = ClickHouseDataStreamFactory.getInstance() .createPipedOutputStream(config, null); this.stream = pipedStream; - this.input = ClickHouseGrpcResponse.getInput(config, pipedStream.getInputStream()); + this.input = ClickHouseGrpcResponse.getInput(config, pipedStream.getInputStream(), postCloseAction); } this.summary = new ClickHouseResponseSummary(null, null); - this.error = null; + this.errorRef = new AtomicReference<>(null); } protected void checkClosed() { @@ -65,12 +77,6 @@ protected void checkClosed() { } } - protected void setError(Throwable error) { - if (this.error == null) { - this.error = error; - } - } - protected boolean updateStatus(Result result) { summary.update(); @@ -111,9 +117,10 @@ protected boolean updateStatus(Result result) { log.error("Server error: Code=%s, %s", e.getCode(), e.getDisplayText()); log.error(e.getStackTrace()); + Throwable error = errorRef.get(); if (error == null) { - error = new ClickHouseException(result.getException().getCode(), result.getException().getDisplayText(), - this.server); + errorRef.compareAndSet(null, new IOException(ClickHouseException + .buildErrorMessage(result.getException().getCode(), result.getException().getDisplayText()))); } } @@ -125,15 +132,15 @@ public boolean isCompleted() { } public boolean isCancelled() { - return isCompleted() && error != null; + return isCompleted() && errorRef.get() != null; } public ClickHouseResponseSummary getSummary() { return summary; } - public Throwable getError() { - return error; + public IOException getError() { + return errorRef.get(); } @Override @@ -162,7 +169,7 @@ public void onError(Throwable t) { try { log.error("Query failed", t); - setError(t); + errorRef.compareAndSet(null, new IOException(t)); try { stream.close(); } catch (IOException e) { @@ -183,9 +190,7 @@ public void onCompleted() { try { stream.flush(); } catch (IOException e) { - if (error == null) { - error = e; - } + errorRef.compareAndSet(null, e); log.error("Failed to flush output", e); } finally { startLatch.countDown(); 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 81452ba21..37e136f79 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 @@ -75,4 +75,10 @@ public void testLZ4FrameStream() throws IOException { expected); } + + @Test(groups = { "integration" }) + @Override + public void testSessionLock() { + throw new SkipException("Skip due to session is not supported"); + } } diff --git a/clickhouse-http-client/src/main/java/com/clickhouse/client/http/ClickHouseHttpClient.java b/clickhouse-http-client/src/main/java/com/clickhouse/client/http/ClickHouseHttpClient.java index 1fd8d4b07..f9aaaf64a 100644 --- a/clickhouse-http-client/src/main/java/com/clickhouse/client/http/ClickHouseHttpClient.java +++ b/clickhouse-http-client/src/main/java/com/clickhouse/client/http/ClickHouseHttpClient.java @@ -1,6 +1,7 @@ package com.clickhouse.client.http; import java.io.IOException; +import java.io.UncheckedIOException; import java.nio.charset.StandardCharsets; import java.util.Collection; import java.util.Collections; @@ -15,6 +16,7 @@ import com.clickhouse.client.ClickHouseProtocol; import com.clickhouse.client.ClickHouseRequest; import com.clickhouse.client.ClickHouseResponse; +import com.clickhouse.client.ClickHouseTransaction; import com.clickhouse.client.config.ClickHouseOption; import com.clickhouse.client.data.ClickHouseStreamResponse; import com.clickhouse.client.http.config.ClickHouseHttpOption; @@ -101,15 +103,25 @@ protected ClickHouseResponse send(ClickHouseRequest sealedRequest) throws Cli log.debug("Query: %s", sql); ClickHouseConfig config = sealedRequest.getConfig(); final ClickHouseHttpResponse httpResponse; + final ClickHouseTransaction tx = sealedRequest.getTransaction(); + final Runnable postAction = tx != null && tx.isImplicit() + ? () -> { + try { + tx.commit(); + } catch (ClickHouseException e) { + throw new UncheckedIOException(new IOException(e.getMessage())); + } + } + : null; if (conn.isReusable()) { ClickHouseNode server = sealedRequest.getServer(); httpResponse = conn.post(sql, sealedRequest.getInputStream().orElse(null), sealedRequest.getExternalTables(), ClickHouseHttpConnection.buildUrl(server.getBaseUri(), sealedRequest), - ClickHouseHttpConnection.createDefaultHeaders(config, server), config); + ClickHouseHttpConnection.createDefaultHeaders(config, server), config, postAction); } else { httpResponse = conn.post(sql, sealedRequest.getInputStream().orElse(null), - sealedRequest.getExternalTables(), null, null, config); + sealedRequest.getExternalTables(), null, null, config, postAction); } return ClickHouseStreamResponse.of(httpResponse.getConfig(sealedRequest), httpResponse.getInputStream(), sealedRequest.getSettings(), null, httpResponse.summary); diff --git a/clickhouse-http-client/src/main/java/com/clickhouse/client/http/ClickHouseHttpConnection.java b/clickhouse-http-client/src/main/java/com/clickhouse/client/http/ClickHouseHttpConnection.java index 436e93c53..3207971cc 100644 --- a/clickhouse-http-client/src/main/java/com/clickhouse/client/http/ClickHouseHttpConnection.java +++ b/clickhouse-http-client/src/main/java/com/clickhouse/client/http/ClickHouseHttpConnection.java @@ -260,19 +260,20 @@ protected Map mergeHeaders(Map requestHeaders) { /** * Posts query and data to server. * - * @param query non-blank query - * @param data optionally input stream for batch updating - * @param tables optionally external tables for query - * @param url optionally url - * @param headers optionally request headers - * @param config optionally configuration + * @param query non-blank query + * @param data optionally input stream for batch updating + * @param tables optionally external tables for query + * @param url optionally url + * @param headers optionally request headers + * @param config optionally configuration + * @param postCloseAction optionally post action * @return response * @throws IOException when error occured posting request and/or server failed * to respond */ protected abstract ClickHouseHttpResponse post(String query, ClickHouseInputStream data, - List tables, String url, Map headers, ClickHouseConfig config) - throws IOException; + List tables, String url, Map headers, ClickHouseConfig config, + Runnable postCloseAction) throws IOException; /** * Checks whether the connection is reusable or not. This method will be called @@ -297,36 +298,36 @@ protected boolean isReusable() { public abstract boolean ping(int timeout); public ClickHouseHttpResponse update(String query) throws IOException { - return post(query, null, null, null, null, null); + return post(query, null, null, null, null, null, null); } public ClickHouseHttpResponse update(String query, Map headers) throws IOException { - return post(query, null, null, null, headers, null); + return post(query, null, null, null, headers, null, null); } public ClickHouseHttpResponse update(String query, ClickHouseInputStream data) throws IOException { - return post(query, data, null, null, null, null); + return post(query, data, null, null, null, null, null); } public ClickHouseHttpResponse update(String query, ClickHouseInputStream data, Map headers) throws IOException { - return post(query, data, null, null, headers, null); + return post(query, data, null, null, headers, null, null); } public ClickHouseHttpResponse query(String query) throws IOException { - return post(query, null, null, null, null, null); + return post(query, null, null, null, null, null, null); } public ClickHouseHttpResponse query(String query, Map headers) throws IOException { - return post(query, null, null, null, headers, null); + return post(query, null, null, null, headers, null, null); } public ClickHouseHttpResponse query(String query, List tables) throws IOException { - return post(query, null, tables, null, null, null); + return post(query, null, tables, null, null, null, null); } public ClickHouseHttpResponse query(String query, List tables, Map headers) throws IOException { - return post(query, null, tables, null, headers, null); + return post(query, null, tables, null, headers, null, null); } } diff --git a/clickhouse-http-client/src/main/java/com/clickhouse/client/http/HttpUrlConnectionImpl.java b/clickhouse-http-client/src/main/java/com/clickhouse/client/http/HttpUrlConnectionImpl.java index 41ef9a7b7..e490ff506 100644 --- a/clickhouse-http-client/src/main/java/com/clickhouse/client/http/HttpUrlConnectionImpl.java +++ b/clickhouse-http-client/src/main/java/com/clickhouse/client/http/HttpUrlConnectionImpl.java @@ -59,7 +59,7 @@ public class HttpUrlConnectionImpl extends ClickHouseHttpConnection { private final HttpURLConnection conn; - private ClickHouseHttpResponse buildResponse() throws IOException { + private ClickHouseHttpResponse buildResponse(Runnable postCloseAction) throws IOException { // X-ClickHouse-Server-Display-Name: xxx // X-ClickHouse-Query-Id: xxx // X-ClickHouse-Format: RowBinaryWithNamesAndTypes @@ -94,13 +94,16 @@ private ClickHouseHttpResponse buildResponse() throws IOException { action = () -> { try (OutputStream o = output) { ClickHouseInputStream.pipe(conn.getInputStream(), o, c.getWriteBufferSize()); + if (postCloseAction != null) { + postCloseAction.run(); + } } catch (IOException e) { throw new UncheckedIOException("Failed to redirect response to given output stream", e); } }; } else { source = conn.getInputStream(); - action = null; + action = postCloseAction; } return new ClickHouseHttpResponse(this, hasOutputFile ? ClickHouseInputStream.of(source, c.getReadBufferSize(), action) @@ -183,7 +186,8 @@ private void checkResponse(HttpURLConnection conn) throws IOException { } errorMsg = builder.toString(); } catch (IOException e) { - log.warn("Error while reading error message[code=%s] from server [%s]", errorCode, serverName, e); + log.debug("Failed to read error message[code=%s] from server [%s] due to: %s", errorCode, serverName, + e.getMessage()); errorMsg = new String(bytes, StandardCharsets.UTF_8); } @@ -205,7 +209,8 @@ protected boolean isReusable() { @Override protected ClickHouseHttpResponse post(String sql, ClickHouseInputStream data, List tables, - String url, Map headers, ClickHouseConfig config) throws IOException { + String url, Map headers, ClickHouseConfig config, Runnable postCloseAction) + throws IOException { Charset charset = StandardCharsets.US_ASCII; byte[] boundary = null; if (tables != null && !tables.isEmpty()) { @@ -283,7 +288,7 @@ protected ClickHouseHttpResponse post(String sql, ClickHouseInputStream data, Li checkResponse(conn); - return buildResponse(); + return buildResponse(postCloseAction); } @Override diff --git a/clickhouse-http-client/src/main/java11/com/clickhouse/client/http/HttpClientConnectionImpl.java b/clickhouse-http-client/src/main/java11/com/clickhouse/client/http/HttpClientConnectionImpl.java index 5418fd0c1..1ca013be4 100644 --- a/clickhouse-http-client/src/main/java11/com/clickhouse/client/http/HttpClientConnectionImpl.java +++ b/clickhouse-http-client/src/main/java11/com/clickhouse/client/http/HttpClientConnectionImpl.java @@ -80,8 +80,8 @@ public List select(URI uri) { private final HttpClient httpClient; private final HttpRequest pingRequest; - private ClickHouseHttpResponse buildResponse(ClickHouseConfig config, HttpResponse r) - throws IOException { + private ClickHouseHttpResponse buildResponse(ClickHouseConfig config, HttpResponse r, + Runnable postAction) throws IOException { HttpHeaders headers = r.headers(); String displayName = headers.firstValue("X-ClickHouse-Server-Display-Name").orElse(server.getHost()); String queryId = headers.firstValue("X-ClickHouse-Query-Id").orElse(""); @@ -107,6 +107,9 @@ private ClickHouseHttpResponse buildResponse(ClickHouseConfig config, HttpRespon action = () -> { try (OutputStream o = output) { ClickHouseInputStream.pipe(checkResponse(config, r).body(), o, config.getWriteBufferSize()); + if (postAction != null) { + postAction.run(); + } } catch (IOException e) { throw new UncheckedIOException("Failed to redirect response to given output stream", e); } finally { @@ -115,7 +118,12 @@ private ClickHouseHttpResponse buildResponse(ClickHouseConfig config, HttpRespon }; } else { source = checkResponse(config, r).body(); - action = this::closeQuietly; + action = () -> { + if (postAction != null) { + postAction.run(); + } + closeQuietly(); + }; } return new ClickHouseHttpResponse(this, @@ -148,7 +156,8 @@ private HttpResponse checkResponse(ClickHouseConfig config, HttpRes } errorMsg = builder.toString(); } catch (IOException e) { - log.warn("Error while reading error message[code=%s] from server [%s]", errorCode, serverName, e); + log.debug("Failed to read error message[code=%s] from server [%s] due to: %s", errorCode, serverName, + e.getMessage()); errorMsg = new String(bytes, StandardCharsets.UTF_8); } @@ -197,7 +206,8 @@ private CompletableFuture> postRequest(HttpRequest req } private ClickHouseHttpResponse postStream(ClickHouseConfig config, HttpRequest.Builder reqBuilder, String boundary, - String sql, ClickHouseInputStream data, List tables) throws IOException { + String sql, ClickHouseInputStream data, List tables, Runnable postAction) + throws IOException { final boolean hasFile = data != null && data.getUnderlyingFile().isAvailable(); ClickHousePipedOutputStream stream = ClickHouseDataStreamFactory.getInstance().createPipedOutputStream(config, null); @@ -263,11 +273,11 @@ private ClickHouseHttpResponse postStream(ClickHouseConfig config, HttpRequest.B } } - return buildResponse(config, r); + return buildResponse(config, r, postAction); } - private ClickHouseHttpResponse postString(ClickHouseConfig config, HttpRequest.Builder reqBuilder, String sql) - throws IOException { + private ClickHouseHttpResponse postString(ClickHouseConfig config, HttpRequest.Builder reqBuilder, String sql, + Runnable postAction) throws IOException { reqBuilder.POST(HttpRequest.BodyPublishers.ofString(sql)); HttpResponse r; try { @@ -283,12 +293,12 @@ private ClickHouseHttpResponse postString(ClickHouseConfig config, HttpRequest.B throw new IOException("Failed to post query", cause); } } - return buildResponse(config, r); + return buildResponse(config, r, postAction); } @Override protected ClickHouseHttpResponse post(String sql, ClickHouseInputStream data, List tables, - String url, Map headers, ClickHouseConfig config) throws IOException { + String url, Map headers, ClickHouseConfig config, Runnable postAction) throws IOException { ClickHouseConfig c = config == null ? this.config : config; HttpRequest.Builder reqBuilder = HttpRequest.newBuilder() .uri(URI.create(ClickHouseChecker.isNullOrEmpty(url) ? this.url : url)) @@ -308,8 +318,8 @@ protected ClickHouseHttpResponse post(String sql, ClickHouseInputStream data, Li } } - return boundary != null || data != null ? postStream(c, reqBuilder, boundary, sql, data, tables) - : postString(c, reqBuilder, sql); + return boundary != null || data != null ? postStream(c, reqBuilder, boundary, sql, data, tables, postAction) + : postString(c, reqBuilder, sql, postAction); } @Override diff --git a/clickhouse-http-client/src/test/java/com/clickhouse/client/http/ClickHouseHttpClientTest.java b/clickhouse-http-client/src/test/java/com/clickhouse/client/http/ClickHouseHttpClientTest.java index d6e9780f5..a9cd4fef1 100644 --- a/clickhouse-http-client/src/test/java/com/clickhouse/client/http/ClickHouseHttpClientTest.java +++ b/clickhouse-http-client/src/test/java/com/clickhouse/client/http/ClickHouseHttpClientTest.java @@ -68,6 +68,18 @@ public void testPing() throws Exception { } } + @Test // (groups = "integration") + public void testTransaction() throws Exception { + testAbortTransaction(); + testNewTransaction(); + testJoinTransaction(); + testCommitTransaction(); + testRollbackTransaction(); + testTransactionSnapshot(); + testTransactionTimeout(); + testImplicitTransaction(); + } + @Test // (groups = "integration") public void testSslClientAuth() throws Exception { // NPE on JDK 8: diff --git a/clickhouse-http-client/src/test/java/com/clickhouse/client/http/ClickHouseHttpConnectionTest.java b/clickhouse-http-client/src/test/java/com/clickhouse/client/http/ClickHouseHttpConnectionTest.java index 8bb67d6ff..18a47d381 100644 --- a/clickhouse-http-client/src/test/java/com/clickhouse/client/http/ClickHouseHttpConnectionTest.java +++ b/clickhouse-http-client/src/test/java/com/clickhouse/client/http/ClickHouseHttpConnectionTest.java @@ -25,8 +25,8 @@ protected SimpleHttpConnection(ClickHouseNode server, ClickHouseRequest reque @Override protected ClickHouseHttpResponse post(String query, ClickHouseInputStream data, - List tables, String url, Map headers, ClickHouseConfig config) - throws IOException { + List tables, String url, Map headers, ClickHouseConfig config, + Runnable postAction) throws IOException { return null; } diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseConnection.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseConnection.java index 6dc6d69f1..d89f63181 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseConnection.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseConnection.java @@ -15,6 +15,7 @@ import com.clickhouse.client.ClickHouseColumn; import com.clickhouse.client.ClickHouseConfig; import com.clickhouse.client.ClickHouseDataType; +import com.clickhouse.client.ClickHouseTransaction; import com.clickhouse.client.ClickHouseValue; import com.clickhouse.client.ClickHouseValues; import com.clickhouse.client.ClickHouseVersion; @@ -205,6 +206,13 @@ default PreparedStatement prepareStatement(String sql, int resultSetType, int re */ ClickHouseVersion getServerVersion(); + /** + * Gets current transaction. + * + * @return current transaction, which could be null + */ + ClickHouseTransaction getTransaction(); + /** * Gets URI of the connection. * @@ -219,6 +227,20 @@ default PreparedStatement prepareStatement(String sql, int resultSetType, int re */ JdbcConfig getJdbcConfig(); + /** + * Checks whether transaction is supported. + * + * @return true if transaction is supported; false otherwise + */ + boolean isTransactionSupported(); + + /** + * Checks whether implicit transaction is supported. + * + * @return true if implicit transaction is supported; false otherwise + */ + boolean isImplicitTransactionSupported(); + /** * Creates a new query ID. * diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseDatabaseMetaData.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseDatabaseMetaData.java index 743bf5678..c6b32d506 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseDatabaseMetaData.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseDatabaseMetaData.java @@ -65,18 +65,14 @@ protected ResultSet query(String sql, ClickHouseRecordTransformer func) throws S protected ResultSet query(String sql, ClickHouseRecordTransformer func, boolean ignoreError) throws SQLException { SQLException error = null; - try { - ClickHouseStatement stmt = connection.createStatement(); + try (ClickHouseStatement stmt = connection.createStatement()) { stmt.setLargeMaxRows(0L); return new ClickHouseResultSet("", "", stmt, // load everything into memory ClickHouseSimpleResponse.of(stmt.getRequest() .format(ClickHouseFormat.RowBinaryWithNamesAndTypes) .option(ClickHouseClientOption.RENAME_RESPONSE_COLUMN, ClickHouseRenameMethod.NONE) - .query(sql).execute().get(), func)); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw SqlExceptionUtils.forCancellation(e); + .query(sql).executeAndWait(), func)); } catch (Exception e) { error = SqlExceptionUtils.handle(e); } @@ -669,13 +665,13 @@ public int getMaxUserNameLength() throws SQLException { @Override public int getDefaultTransactionIsolation() throws SQLException { - return connection.getJdbcConfig().isJdbcCompliant() ? Connection.TRANSACTION_READ_COMMITTED + return connection.getJdbcConfig().isJdbcCompliant() ? Connection.TRANSACTION_REPEATABLE_READ : Connection.TRANSACTION_NONE; } @Override public boolean supportsTransactions() throws SQLException { - return connection.getJdbcConfig().isJdbcCompliant(); + return connection.isTransactionSupported() || connection.getJdbcConfig().isJdbcCompliant(); } @Override @@ -687,7 +683,8 @@ public boolean supportsTransactionIsolationLevel(int level) throws SQLException throw SqlExceptionUtils.clientError("Unknown isolation level: " + level); } - return connection.getJdbcConfig().isJdbcCompliant(); + return (connection.isTransactionSupported() && Connection.TRANSACTION_REPEATABLE_READ == level) + || connection.getJdbcConfig().isJdbcCompliant(); } @Override diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcConfig.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcConfig.java index 2bbe5713f..39283ffe0 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcConfig.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcConfig.java @@ -27,6 +27,7 @@ public class JdbcConfig { public static final String PROP_JDBC_COMPLIANT = "jdbcCompliant"; public static final String PROP_NAMED_PARAM = "namedParameter"; public static final String PROP_NULL_AS_DEFAULT = "nullAsDefault"; + public static final String PROP_TX_SUPPORT = "transactionSupport"; public static final String PROP_TYPE_MAP = "typeMappings"; public static final String PROP_WRAPPER_OBJ = "wrapperObject"; @@ -40,6 +41,7 @@ public class JdbcConfig { private static final String DEFAULT_JDBC_COMPLIANT = BOOLEAN_TRUE; private static final String DEFAULT_NAMED_PARAM = BOOLEAN_FALSE; private static final String DEFAULT_NULL_AS_DEFAULT = "0"; + private static final String DEFAULT_TX_SUPPORT = BOOLEAN_FALSE; private static final String DEFAULT_TYPE_MAP = ""; private static final String DEFAULT_WRAPPER_OBJ = BOOLEAN_FALSE; @@ -126,6 +128,11 @@ public static List getDriverProperties() { info.description = "Default approach to handle null value, sets to 0 or negative number to throw exception when target column is not nullable, 1 to disable the null-check, and 2 or higher to replace null to default value of corresponding data type."; list.add(info); + info = new DriverPropertyInfo(PROP_TX_SUPPORT, DEFAULT_TX_SUPPORT); + info.choices = new String[] { BOOLEAN_TRUE, BOOLEAN_FALSE }; + info.description = "Whether to enable transaction support or not."; + list.add(info); + info = new DriverPropertyInfo(PROP_TYPE_MAP, DEFAULT_TYPE_MAP); info.description = "Default type mappings between ClickHouse data type and Java class. You can define multiple mappings using comma as separator."; list.add(info); @@ -145,6 +152,7 @@ public static List getDriverProperties() { private final boolean jdbcCompliant; private final boolean namedParameter; private final int nullAsDefault; + private final boolean txSupport; private final Map> typeMap; private final boolean wrapperObject; @@ -164,6 +172,7 @@ public JdbcConfig(Properties props) { this.jdbcCompliant = extractBooleanValue(props, PROP_JDBC_COMPLIANT, DEFAULT_JDBC_COMPLIANT); this.namedParameter = extractBooleanValue(props, PROP_NAMED_PARAM, DEFAULT_NAMED_PARAM); this.nullAsDefault = extractIntValue(props, PROP_NULL_AS_DEFAULT, DEFAULT_NULL_AS_DEFAULT); + this.txSupport = extractBooleanValue(props, PROP_TX_SUPPORT, DEFAULT_TX_SUPPORT); this.typeMap = extractTypeMapValue(props, PROP_TYPE_MAP, DEFAULT_TYPE_MAP); this.wrapperObject = extractBooleanValue(props, PROP_WRAPPER_OBJ, DEFAULT_WRAPPER_OBJ); } @@ -224,6 +233,15 @@ public boolean isJdbcCompliant() { return jdbcCompliant; } + /** + * Checks whether transaction support is enabled or not. + * + * @return true if transaction support is enabled; false otherwise + */ + public boolean isTransactionSupported() { + return txSupport; + } + /** * Gets default approach to handle null value. * diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/SqlExceptionUtils.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/SqlExceptionUtils.java index 25b3ffe3f..eb13dc55b 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/SqlExceptionUtils.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/SqlExceptionUtils.java @@ -12,6 +12,7 @@ */ public final class SqlExceptionUtils { public static final String SQL_STATE_CLIENT_ERROR = "HY000"; + public static final String SQL_STATE_OPERATION_CANCELLED = "HY008"; public static final String SQL_STATE_CONNECTION_EXCEPTION = "08000"; public static final String SQL_STATE_SQL_ERROR = "07000"; public static final String SQL_STATE_NO_DATA = "02000"; @@ -23,19 +24,48 @@ public final class SqlExceptionUtils { private SqlExceptionUtils() { } + private static SQLException create(Throwable e) { + if (e == null) { + return unknownError(); + } else if (e instanceof ClickHouseException) { + return handle((ClickHouseException) e); + } else if (e instanceof SQLException) { + return (SQLException) e; + } + + Throwable cause = e.getCause(); + if (cause instanceof ClickHouseException) { + return handle((ClickHouseException) cause); + } else if (e instanceof SQLException) { + return (SQLException) cause; + } else if (cause == null) { + cause = e; + } + + return new SQLException(cause); + } + // https://en.wikipedia.org/wiki/SQLSTATE private static String toSqlState(ClickHouseException e) { - String sqlState; - if (e.getErrorCode() == ClickHouseException.ERROR_NETWORK - || e.getErrorCode() == ClickHouseException.ERROR_POCO) { - sqlState = SQL_STATE_CONNECTION_EXCEPTION; - } else if (e.getErrorCode() == 0) { - sqlState = e.getCause() instanceof ConnectException ? SQL_STATE_CONNECTION_EXCEPTION - : SQL_STATE_CLIENT_ERROR; - } else { - sqlState = e.getCause() instanceof ConnectException ? SQL_STATE_CONNECTION_EXCEPTION : SQL_STATE_SQL_ERROR; + final String sqlState; + switch (e.getErrorCode()) { + case ClickHouseException.ERROR_ABORTED: + case ClickHouseException.ERROR_CANCELLED: + sqlState = SQL_STATE_OPERATION_CANCELLED; + break; + case ClickHouseException.ERROR_NETWORK: + case ClickHouseException.ERROR_POCO: + sqlState = SQL_STATE_CONNECTION_EXCEPTION; + break; + case 0: + sqlState = e.getCause() instanceof ConnectException ? SQL_STATE_CONNECTION_EXCEPTION + : SQL_STATE_CLIENT_ERROR; + break; + default: + sqlState = e.getCause() instanceof ConnectException ? SQL_STATE_CONNECTION_EXCEPTION + : SQL_STATE_SQL_ERROR; + break; } - return sqlState; } @@ -56,25 +86,14 @@ public static SQLException handle(ClickHouseException e) { : unknownError(); } - public static SQLException handle(Throwable e) { - if (e == null) { - return unknownError(); - } else if (e instanceof ClickHouseException) { - return handle((ClickHouseException) e); - } else if (e instanceof SQLException) { - return (SQLException) e; + public static SQLException handle(Throwable e, Throwable... more) { + SQLException rootEx = create(e); + if (more != null) { + for (Throwable t : more) { + rootEx.setNextException(create(t)); + } } - - Throwable cause = e.getCause(); - if (cause instanceof ClickHouseException) { - return handle((ClickHouseException) cause); - } else if (e instanceof SQLException) { - return (SQLException) cause; - } else if (cause == null) { - cause = e; - } - - return new SQLException(cause); + return rootEx; } public static BatchUpdateException batchUpdateError(Throwable e, long[] updateCounts) { @@ -127,7 +146,8 @@ public static SQLException forCancellation(Exception e) { } // operation canceled - return new SQLException(e.getMessage(), "HY008", ClickHouseException.ERROR_ABORTED, cause); + return new SQLException(e.getMessage(), SQL_STATE_OPERATION_CANCELLED, ClickHouseException.ERROR_ABORTED, + cause); } public static SQLFeatureNotSupportedException unsupportedError(String message) { diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseConnectionImpl.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseConnectionImpl.java index 4e6af77b5..e4ba1c48e 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseConnectionImpl.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseConnectionImpl.java @@ -19,8 +19,6 @@ import java.util.Optional; import java.util.Properties; import java.util.TimeZone; -import java.util.UUID; -import java.util.concurrent.CancellationException; import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; @@ -30,6 +28,7 @@ import com.clickhouse.client.ClickHouseClientBuilder; import com.clickhouse.client.ClickHouseColumn; import com.clickhouse.client.ClickHouseConfig; +import com.clickhouse.client.ClickHouseException; import com.clickhouse.client.ClickHouseFormat; import com.clickhouse.client.ClickHouseNode; import com.clickhouse.client.ClickHouseNodeSelector; @@ -38,6 +37,7 @@ import com.clickhouse.client.ClickHouseRecord; import com.clickhouse.client.ClickHouseRequest; import com.clickhouse.client.ClickHouseResponse; +import com.clickhouse.client.ClickHouseTransaction; import com.clickhouse.client.ClickHouseUtils; import com.clickhouse.client.ClickHouseValues; import com.clickhouse.client.ClickHouseVersion; @@ -56,7 +56,6 @@ import com.clickhouse.jdbc.SqlExceptionUtils; import com.clickhouse.jdbc.JdbcWrapper; import com.clickhouse.jdbc.internal.ClickHouseJdbcUrlParser.ConnectionInfo; -import com.clickhouse.jdbc.internal.FakeTransaction.FakeSavepoint; import com.clickhouse.jdbc.parser.ClickHouseSqlParser; import com.clickhouse.jdbc.parser.ClickHouseSqlStatement; import com.clickhouse.jdbc.parser.StatementType; @@ -64,7 +63,19 @@ public class ClickHouseConnectionImpl extends JdbcWrapper implements ClickHouseConnection { private static final Logger log = LoggerFactory.getLogger(ClickHouseConnectionImpl.class); - private static final String CREATE_DB = "create database if not exists `"; + private static final String SETTING_READONLY = "readonly"; + + private static final String SQL_GET_SERVER_INFO = "select currentUser() user, timezone() timezone, version() version, " + + "toInt8(ifnull((select value from system.settings where name = 'readonly'), '0')) as readonly, " + + "toInt8(ifnull((select value from system.settings where name = '" + + ClickHouseTransaction.SETTING_THROW_ON_UNSUPPORTED_QUERY_INSIDE_TRANSACTION + + "'), '-1')) as non_transational_query, " + + "lower(ifnull((select value from system.settings where name = '" + + ClickHouseTransaction.SETTING_WAIT_CHANGES_BECOME_VISIBLE_AFTER_COMMIT_MODE + + "'), '')) as commit_wait_mode, " + + "toInt8(ifnull((select value from system.settings where name = '" + + ClickHouseTransaction.SETTING_IMPLICIT_TRANSACTION + "'), '-1')) as implicit_transaction " + + "FORMAT RowBinaryWithNamesAndTypes"; protected static ClickHouseRecord getServerInfo(ClickHouseNode node, ClickHouseRequest request, boolean createDbIfNotExist) throws SQLException { @@ -76,27 +87,17 @@ protected static ClickHouseRecord getServerInfo(ClickHouseNode node, ClickHouseR try (ClickHouseResponse response = newReq.option(ClickHouseClientOption.ASYNC, false) .option(ClickHouseClientOption.COMPRESS, false).option(ClickHouseClientOption.DECOMPRESS, false) .option(ClickHouseClientOption.FORMAT, ClickHouseFormat.RowBinaryWithNamesAndTypes) - .query("select currentUser(), timezone(), version(), getSetting('readonly') readonly " - + "FORMAT RowBinaryWithNamesAndTypes") - .execute().get()) { + .query(SQL_GET_SERVER_INFO).executeAndWait()) { return response.firstRecord(); - } catch (InterruptedException | CancellationException e) { - // not going to happen as it's synchronous call - Thread.currentThread().interrupt(); - throw SqlExceptionUtils.forCancellation(e); } catch (Exception e) { SQLException sqlExp = SqlExceptionUtils.handle(e); if (createDbIfNotExist && sqlExp.getErrorCode() == 81) { String db = node.getDatabase(request.getConfig()); try (ClickHouseResponse resp = newReq.use("") - .query(new StringBuilder(CREATE_DB.length() + 1 + db.length()).append(CREATE_DB).append(db) - .append('`').toString()) - .execute().get()) { + .query(new StringBuilder("create database if not exists `") + .append(ClickHouseUtils.escape(db, '`')).append('`').toString()) + .executeAndWait()) { return getServerInfo(node, request, false); - } catch (InterruptedException | CancellationException ex) { - // not going to happen as it's synchronous call - Thread.currentThread().interrupt(); - throw SqlExceptionUtils.forCancellation(ex); } catch (SQLException ex) { throw ex; } catch (Exception ex) { @@ -128,10 +129,35 @@ protected static ClickHouseRecord getServerInfo(ClickHouseNode node, ClickHouseR private final ClickHouseVersion serverVersion; private final String user; private final int initialReadOnly; + private final int initialNonTxQuerySupport; + private final String initialTxCommitWaitMode; + private final int initialImplicitTx; private final Map> typeMap; - private final AtomicReference fakeTransaction; + private final AtomicReference txRef; + + protected JdbcTransaction createTransaction() throws SQLException { + if (!isTransactionSupported()) { + return new JdbcTransaction(null); + } + + try { + ClickHouseTransaction tx = clientRequest.getManager().createTransaction(clientRequest); + tx.begin(); + // if (txIsolation == Connection.TRANSACTION_READ_UNCOMMITTED) { + // tx.snapshot(ClickHouseTransaction.CSN_EVERYTHING_VISIBLE); + // } + clientRequest.transaction(tx); + return new JdbcTransaction(tx); + } catch (ClickHouseException e) { + throw SqlExceptionUtils.handle(e); + } + } + + protected JdbcSavepoint createSavepoint() { + return new JdbcSavepoint(1, "name"); + } /** * Checks if the connection is open or not. @@ -169,7 +195,9 @@ protected void ensureSupport(String feature, boolean silent) throws SQLException } protected void ensureTransactionSupport() throws SQLException { - ensureSupport("Transaction", false); + if (!isTransactionSupported()) { + ensureSupport("Transaction", false); + } } protected List getTableColumns(String dbName, String tableName, String columns) @@ -192,20 +220,17 @@ protected List getTableColumns(String dbName, String tableName List list; try (ClickHouseResponse resp = clientRequest.copy().format(ClickHouseFormat.RowBinaryWithNamesAndTypes) .option(ClickHouseClientOption.RENAME_RESPONSE_COLUMN, ClickHouseRenameMethod.NONE) - .query(builder.toString()).execute().get()) { + .query(builder.toString()).executeAndWait()) { list = resp.getColumns(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw SqlExceptionUtils.forCancellation(e); } catch (Exception e) { throw SqlExceptionUtils.handle(e); } return list; } - // for testing only - final FakeTransaction getTransaction() { - return fakeTransaction.get(); + // for testing purpose + final JdbcTransaction getJdbcTrasaction() { + return txRef.get(); } public ClickHouseConnectionImpl(String url) throws SQLException { @@ -219,7 +244,6 @@ public ClickHouseConnectionImpl(String url, Properties properties) throws SQLExc public ClickHouseConnectionImpl(ConnectionInfo connInfo) throws SQLException { jdbcConf = connInfo.getJdbcConfig(); - autoCommit = !jdbcConf.isJdbcCompliant() || jdbcConf.isAutoCommit(); jvmTimeZone = TimeZone.getDefault(); ClickHouseClientBuilder clientBuilder = ClickHouseClient.builder() @@ -246,7 +270,7 @@ public ClickHouseConnectionImpl(ConnectionInfo connInfo) throws SQLException { } } - log.warn("Connecting to: %s", node); + log.debug("Connecting to: %s", node); ClickHouseConfig config = clientRequest.getConfig(); String currentUser = null; TimeZone timeZone = null; @@ -255,9 +279,20 @@ public ClickHouseConnectionImpl(ConnectionInfo connInfo) throws SQLException { timeZone = config.getServerTimeZone(); version = config.getServerVersion(); if (jdbcConf.isCreateDbIfNotExist()) { - initialReadOnly = getServerInfo(node, clientRequest, true).getValue(3).asInteger(); + ClickHouseRecord r = getServerInfo(node, clientRequest, true); + initialReadOnly = r.getValue(3).asInteger(); + initialNonTxQuerySupport = r.getValue(4).asInteger(); + initialTxCommitWaitMode = r.getValue(5).asString(); + initialImplicitTx = r.getValue(6).asInteger(); } else { - initialReadOnly = (int) clientRequest.getSettings().getOrDefault("readonly", 0); + initialReadOnly = (int) clientRequest.getSettings().getOrDefault(SETTING_READONLY, 0); + initialNonTxQuerySupport = (int) clientRequest.getSettings() + .getOrDefault(ClickHouseTransaction.SETTING_THROW_ON_UNSUPPORTED_QUERY_INSIDE_TRANSACTION, 1); + initialTxCommitWaitMode = (String) clientRequest.getSettings() + .getOrDefault(ClickHouseTransaction.SETTING_WAIT_CHANGES_BECOME_VISIBLE_AFTER_COMMIT_MODE, + "wait_unknown"); + initialImplicitTx = (int) clientRequest.getSettings() + .getOrDefault(ClickHouseTransaction.SETTING_IMPLICIT_TRANSACTION, 0); } } else { ClickHouseRecord r = getServerInfo(node, clientRequest, jdbcConf.isCreateDbIfNotExist()); @@ -276,21 +311,33 @@ public ClickHouseConnectionImpl(ConnectionInfo connInfo) throws SQLException { // tsTimeZone.hasSameRules(ClickHouseValues.UTC_TIMEZONE) timeZone = "UTC".equals(tz) ? ClickHouseValues.UTC_TIMEZONE : TimeZone.getTimeZone(tz); initialReadOnly = r.getValue(3).asInteger(); + initialNonTxQuerySupport = r.getValue(4).asInteger(); + initialTxCommitWaitMode = r.getValue(5).asString(); + initialImplicitTx = r.getValue(6).asInteger(); // update request and corresponding config clientRequest.option(ClickHouseClientOption.SERVER_TIME_ZONE, tz) .option(ClickHouseClientOption.SERVER_VERSION, ver); } - this.autoCommit = true; + this.autoCommit = !jdbcConf.isJdbcCompliant() || jdbcConf.isAutoCommit(); this.closed = false; this.database = config.getDatabase(); this.clientRequest.use(this.database); this.readOnly = initialReadOnly != 0; this.networkTimeout = 0; this.rsHoldability = ResultSet.HOLD_CURSORS_OVER_COMMIT; - this.txIsolation = jdbcConf.isJdbcCompliant() ? Connection.TRANSACTION_READ_COMMITTED - : Connection.TRANSACTION_NONE; + if (isTransactionSupported()) { + this.txIsolation = Connection.TRANSACTION_REPEATABLE_READ; + if (jdbcConf.isJdbcCompliant()) { + this.clientRequest.set(ClickHouseTransaction.SETTING_THROW_ON_UNSUPPORTED_QUERY_INSIDE_TRANSACTION, 0); + // .set(ClickHouseTransaction.SETTING_WAIT_CHANGES_BECOME_VISIBLE_AFTER_COMMIT_MODE, + // "wait_unknown"); + } + } else { + this.txIsolation = jdbcConf.isJdbcCompliant() ? Connection.TRANSACTION_READ_COMMITTED + : Connection.TRANSACTION_NONE; + } this.user = currentUser != null ? currentUser : node.getCredentials(config).getUserName(); this.serverTimeZone = timeZone; @@ -304,7 +351,7 @@ public ClickHouseConnectionImpl(ConnectionInfo connInfo) throws SQLException { } this.serverVersion = version; this.typeMap = new HashMap<>(jdbcConf.getTypeMap()); - this.fakeTransaction = new AtomicReference<>(); + this.txRef = new AtomicReference<>(this.autoCommit ? null : createTransaction()); } @Override @@ -325,13 +372,12 @@ public void setAutoCommit(boolean autoCommit) throws SQLException { ensureTransactionSupport(); if (this.autoCommit = autoCommit) { // commit - FakeTransaction tx = fakeTransaction.getAndSet(null); + JdbcTransaction tx = txRef.getAndSet(null); if (tx != null) { - tx.logTransactionDetails(log, FakeTransaction.ACTION_COMMITTED); - tx.clear(); + tx.commit(log); } } else { // start new transaction - if (!fakeTransaction.compareAndSet(null, new FakeTransaction())) { + if (!txRef.compareAndSet(null, createTransaction())) { log.warn("[JDBC Compliant Mode] not able to start a new transaction, reuse the exist one"); } } @@ -354,13 +400,18 @@ public void commit() throws SQLException { ensureTransactionSupport(); - FakeTransaction tx = fakeTransaction.getAndSet(new FakeTransaction()); + JdbcTransaction tx = txRef.get(); if (tx == null) { // invalid transaction state - throw new SQLException(FakeTransaction.ERROR_TX_NOT_STARTED, SqlExceptionUtils.SQL_STATE_INVALID_TX_STATE); + throw new SQLException(JdbcTransaction.ERROR_TX_NOT_STARTED, SqlExceptionUtils.SQL_STATE_INVALID_TX_STATE); } else { - tx.logTransactionDetails(log, FakeTransaction.ACTION_COMMITTED); - tx.clear(); + try { + tx.commit(log); + } finally { + if (!txRef.compareAndSet(tx, createTransaction())) { + log.warn("Transaction was set to %s unexpectedly", txRef.get()); + } + } } } @@ -374,13 +425,18 @@ public void rollback() throws SQLException { ensureTransactionSupport(); - FakeTransaction tx = fakeTransaction.getAndSet(new FakeTransaction()); + JdbcTransaction tx = txRef.get(); if (tx == null) { // invalid transaction state - throw new SQLException(FakeTransaction.ERROR_TX_NOT_STARTED, SqlExceptionUtils.SQL_STATE_INVALID_TX_STATE); + throw new SQLException(JdbcTransaction.ERROR_TX_NOT_STARTED, SqlExceptionUtils.SQL_STATE_INVALID_TX_STATE); } else { - tx.logTransactionDetails(log, FakeTransaction.ACTION_ROLLBACK); - tx.clear(); + try { + tx.rollback(log); + } finally { + if (!txRef.compareAndSet(tx, createTransaction())) { + log.warn("Transaction was set to %s unexpectedly", txRef.get()); + } + } } } @@ -395,10 +451,15 @@ public void close() throws SQLException { this.closed = true; } - FakeTransaction tx = fakeTransaction.getAndSet(null); + JdbcTransaction tx = txRef.get(); if (tx != null) { - tx.logTransactionDetails(log, FakeTransaction.ACTION_COMMITTED); - tx.clear(); + try { + tx.commit(log); + } finally { + if (!txRef.compareAndSet(tx, null)) { + log.warn("Transaction was set to %s unexpectedly", txRef.get()); + } + } } } @@ -422,9 +483,9 @@ public void setReadOnly(boolean readOnly) throws SQLException { } } else { if (readOnly) { - clientRequest.set("readonly", 2); + clientRequest.set(SETTING_READONLY, 2); } else { - clientRequest.removeSetting("readonly"); + clientRequest.removeSetting(SETTING_READONLY); } this.readOnly = readOnly; } @@ -455,11 +516,16 @@ public String getCatalog() throws SQLException { public void setTransactionIsolation(int level) throws SQLException { ensureOpen(); - if (level == Connection.TRANSACTION_READ_UNCOMMITTED || level == Connection.TRANSACTION_READ_COMMITTED - || level == Connection.TRANSACTION_REPEATABLE_READ || level == Connection.TRANSACTION_SERIALIZABLE) { + if (Connection.TRANSACTION_NONE != level && Connection.TRANSACTION_READ_UNCOMMITTED != level + && Connection.TRANSACTION_READ_COMMITTED != level && Connection.TRANSACTION_REPEATABLE_READ != level + && Connection.TRANSACTION_SERIALIZABLE != level) { + throw new SQLException("Invalid transaction isolation level: " + level); + } else if (isTransactionSupported()) { + txIsolation = Connection.TRANSACTION_REPEATABLE_READ; + } else if (jdbcConf.isJdbcCompliant()) { txIsolation = level; } else { - throw new SQLException("Invalid transaction isolation level: " + level); + txIsolation = Connection.TRANSACTION_NONE; } } @@ -533,7 +599,13 @@ public Savepoint setSavepoint(String name) throws SQLException { throw SqlExceptionUtils.unsupportedError("setSavepoint not implemented"); } - FakeTransaction tx = fakeTransaction.updateAndGet(current -> current != null ? current : new FakeTransaction()); + JdbcTransaction tx = txRef.get(); + if (tx == null) { + tx = createTransaction(); + if (!txRef.compareAndSet(null, tx)) { + tx = txRef.get(); + } + } return tx.newSavepoint(name); } @@ -549,17 +621,17 @@ public void rollback(Savepoint savepoint) throws SQLException { throw SqlExceptionUtils.unsupportedError("rollback not implemented"); } - if (!(savepoint instanceof FakeSavepoint)) { + if (!(savepoint instanceof JdbcSavepoint)) { throw SqlExceptionUtils.clientError("Unsupported type of savepoint: " + savepoint); } - FakeTransaction tx = fakeTransaction.get(); + JdbcTransaction tx = txRef.get(); if (tx == null) { // invalid transaction state - throw new SQLException(FakeTransaction.ERROR_TX_NOT_STARTED, SqlExceptionUtils.SQL_STATE_INVALID_TX_STATE); + throw new SQLException(JdbcTransaction.ERROR_TX_NOT_STARTED, SqlExceptionUtils.SQL_STATE_INVALID_TX_STATE); } else { - FakeSavepoint s = (FakeSavepoint) savepoint; - tx.logSavepointDetails(log, s, FakeTransaction.ACTION_ROLLBACK); + JdbcSavepoint s = (JdbcSavepoint) savepoint; + tx.logSavepointDetails(log, s, JdbcTransaction.ACTION_ROLLBACK); tx.toSavepoint(s); } } @@ -576,16 +648,16 @@ public void releaseSavepoint(Savepoint savepoint) throws SQLException { throw SqlExceptionUtils.unsupportedError("rollback not implemented"); } - if (!(savepoint instanceof FakeSavepoint)) { + if (!(savepoint instanceof JdbcSavepoint)) { throw SqlExceptionUtils.clientError("Unsupported type of savepoint: " + savepoint); } - FakeTransaction tx = fakeTransaction.get(); + JdbcTransaction tx = txRef.get(); if (tx == null) { // invalid transaction state - throw new SQLException(FakeTransaction.ERROR_TX_NOT_STARTED, SqlExceptionUtils.SQL_STATE_INVALID_TX_STATE); + throw new SQLException(JdbcTransaction.ERROR_TX_NOT_STARTED, SqlExceptionUtils.SQL_STATE_INVALID_TX_STATE); } else { - FakeSavepoint s = (FakeSavepoint) savepoint; + JdbcSavepoint s = (JdbcSavepoint) savepoint; tx.logSavepointDetails(log, s, "released"); tx.toSavepoint(s); } @@ -916,6 +988,11 @@ public ClickHouseVersion getServerVersion() { return serverVersion; } + @Override + public ClickHouseTransaction getTransaction() { + return clientRequest.getTransaction(); + } + @Override public URI getUri() { return clientRequest.getServer().toUri(ClickHouseJdbcUrlParser.JDBC_CLICKHOUSE_PREFIX); @@ -926,10 +1003,22 @@ public JdbcConfig getJdbcConfig() { return jdbcConf; } + @Override + public boolean isTransactionSupported() { + return jdbcConf.isTransactionSupported() && initialNonTxQuerySupport >= 0 + && !ClickHouseChecker.isNullOrEmpty(initialTxCommitWaitMode); + } + + @Override + public boolean isImplicitTransactionSupported() { + return jdbcConf.isTransactionSupported() && initialImplicitTx >= 0; + } + @Override public String newQueryId() { - FakeTransaction tx = fakeTransaction.get(); - return tx != null ? tx.newQuery(null) : UUID.randomUUID().toString(); + String queryId = clientRequest.getManager().createQueryId(); + JdbcTransaction tx = txRef.get(); + return tx != null ? tx.newQuery(queryId) : queryId; } @Override diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseStatementImpl.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseStatementImpl.java index 1b0be66e0..82e20603d 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseStatementImpl.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseStatementImpl.java @@ -10,7 +10,6 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; -import java.util.UUID; import java.util.Map.Entry; import com.clickhouse.client.ClickHouseChecker; @@ -26,6 +25,7 @@ import com.clickhouse.client.ClickHouseUtils; import com.clickhouse.client.ClickHouseValue; import com.clickhouse.client.ClickHouseValues; +import com.clickhouse.client.ClickHouseRequest.Mutation; import com.clickhouse.client.config.ClickHouseClientOption; import com.clickhouse.client.config.ClickHouseConfigChangeListener; import com.clickhouse.client.config.ClickHouseOption; @@ -75,9 +75,11 @@ public class ClickHouseStatementImpl extends JdbcWrapper private ClickHouseResponse getLastResponse(Map options, List tables, Map settings) throws SQLException { + boolean autoTx = connection.getAutoCommit() && connection.isTransactionSupported(); + // disable extremes - if (parsedStmts.length > 1) { - request.session(UUID.randomUUID().toString()); + if (parsedStmts.length > 1 && !request.getSessionId().isPresent()) { + request.session(request.getManager().createSessionId()); } ClickHouseResponse response = null; for (int i = 0, len = parsedStmts.length; i < len; i++) { @@ -85,12 +87,11 @@ private ClickHouseResponse getLastResponse(Map o if (stmt.hasFormat()) { request.format(ClickHouseFormat.valueOf(stmt.getFormat())); } + request.query(stmt.getSQL(), queryId = connection.newQueryId()); // TODO skip useless queries to reduce network calls and server load try { - response = request.query(stmt.getSQL(), queryId = connection.newQueryId()).execute().get(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw SqlExceptionUtils.forCancellation(e); + response = autoTx ? request.executeWithinTransaction(connection.isImplicitTransactionSupported()) + : request.transaction(connection.getTransaction()).executeAndWait(); } catch (Exception e) { throw SqlExceptionUtils.handle(e); } finally { @@ -109,16 +110,16 @@ protected void ensureOpen() throws SQLException { } } - protected ClickHouseResponse executeStatement(String stmt, - Map options, List tables, - Map settings) throws SQLException { + protected ClickHouseResponse executeStatement(String stmt, Map options, + List tables, Map settings) throws SQLException { + boolean autoTx = connection.getAutoCommit() && connection.isTransactionSupported(); try { if (options != null) { request.options(options); } if (settings != null && !settings.isEmpty()) { if (!request.getSessionId().isPresent()) { - request.session(UUID.randomUUID().toString()); + request.session(request.getManager().createSessionId()); } for (Entry e : settings.entrySet()) { request.set(e.getKey(), e.getValue()); @@ -130,28 +131,28 @@ protected ClickHouseResponse executeStatement(String stmt, for (ClickHouseExternalTable t : tables) { if (t.isTempTable()) { if (!request.getSessionId().isPresent()) { - request.session(UUID.randomUUID().toString()); + request.session(request.getManager().createSessionId()); } String tableName = new StringBuilder().append(quote) .append(ClickHouseUtils.escape(t.getName(), quote)).append(quote).toString(); - request.query("drop temporary table if exists ".concat(tableName)).executeAndWait(); - request.query("create temporary table " + tableName + "(" + t.getStructure() + ")") - .executeAndWait(); - request.write().table(tableName) - // .format(t.getFormat() != null ? t.getFormat() : ClickHouseFormat.RowBinary) - .data(t.getContent()).send().get(); + try (ClickHouseResponse dropResp = request + .query("DROP TEMPORARY TABLE IF EXISTS ".concat(tableName)).executeAndWait(); + ClickHouseResponse createResp = request + .query("CREATE TEMPORARY TABLE " + tableName + "(" + t.getStructure() + ")") + .executeAndWait(); + ClickHouseResponse writeResp = request.write().table(tableName).data(t.getContent()) + .sendAndWait()) { + // ignore + } } else { list.add(t); } } request.external(list); } - - return request.query(stmt, queryId = connection.newQueryId()).execute().get(); - } catch (InterruptedException e) { - log.error("can not close stream: %s", e.getMessage()); - Thread.currentThread().interrupt(); - throw SqlExceptionUtils.forCancellation(e); + request.query(stmt, queryId = connection.newQueryId()); + return autoTx ? request.executeWithinTransaction(connection.isImplicitTransactionSupported()) + : request.transaction(connection.getTransaction()).executeAndWait(); } catch (Exception e) { throw SqlExceptionUtils.handle(e); } @@ -164,15 +165,14 @@ protected ClickHouseResponse executeStatement(ClickHouseSqlStatement stmt, } protected int executeInsert(String sql, InputStream input) throws SQLException { + boolean autoTx = connection.getAutoCommit() && connection.isTransactionSupported(); ClickHouseResponseSummary summary = null; - try (ClickHouseResponse resp = request.write().query(sql, queryId = connection.newQueryId()).data(input) - .execute().get(); + Mutation req = request.write().query(sql, queryId = connection.newQueryId()).data(input); + try (ClickHouseResponse resp = autoTx + ? req.executeWithinTransaction(connection.isImplicitTransactionSupported()) + : req.transaction(connection.getTransaction()).sendAndWait(); ResultSet rs = updateResult(new ClickHouseSqlStatement(sql, StatementType.INSERT), resp)) { summary = resp.getSummary(); - } catch (InterruptedException e) { - log.error("can not close stream: %s", e.getMessage()); - Thread.currentThread().interrupt(); - throw SqlExceptionUtils.forCancellation(e); } catch (Exception e) { throw SqlExceptionUtils.handle(e); } @@ -436,19 +436,24 @@ public void setQueryTimeout(int seconds) throws SQLException { @Override public void cancel() throws SQLException { - final String qid; - if ((qid = this.queryId) == null || isClosed()) { + if (isClosed()) { return; } - ClickHouseClient.send(request.getServer(), String.format("KILL QUERY WHERE query_id='%s'", qid)) - .whenComplete((summary, exception) -> { - if (exception != null) { - log.warn("Failed to kill query [%s] due to: %s", qid, exception.getMessage()); - } else if (summary != null) { - log.debug("Killed query [%s]", qid); - } - }); + final String qid; + if ((qid = this.queryId) != null) { + ClickHouseClient.send(request.getServer(), String.format("KILL QUERY WHERE query_id='%s'", qid)) + .whenComplete((summary, exception) -> { + if (exception != null) { + log.warn("Failed to kill query [%s] due to: %s", qid, exception.getMessage()); + } else if (summary != null) { + log.debug("Killed query [%s]", qid); + } + }); + } + if (request.getTransaction() != null) { + request.getTransaction().abort(); + } } @Override diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/FakeTransaction.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/FakeTransaction.java deleted file mode 100644 index 4a46be48a..000000000 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/FakeTransaction.java +++ /dev/null @@ -1,145 +0,0 @@ -package com.clickhouse.jdbc.internal; - -import java.sql.Connection; -import java.sql.SQLException; -import java.sql.Savepoint; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.UUID; - -import com.clickhouse.client.ClickHouseUtils; -import com.clickhouse.client.logging.Logger; -import com.clickhouse.jdbc.SqlExceptionUtils; - -public final class FakeTransaction { - static final String ACTION_COMMITTED = "committed"; - static final String ACTION_ROLLBACK = "rolled back"; - - static final String ERROR_TX_NOT_STARTED = "Transaction not started"; - - static final int DEFAULT_TX_ISOLATION_LEVEL = Connection.TRANSACTION_READ_UNCOMMITTED; - - static final class FakeSavepoint implements Savepoint { - final int id; - final String name; - - FakeSavepoint(int id, String name) { - this.id = id; - this.name = name; - } - - @Override - public int getSavepointId() throws SQLException { - if (name != null) { - throw SqlExceptionUtils - .clientError("Cannot get ID from a named savepoint, please use getSavepointName() instead"); - } - - return id; - } - - @Override - public String getSavepointName() throws SQLException { - if (name == null) { - throw SqlExceptionUtils - .clientError("Cannot get name from an un-named savepoint, please use getSavepointId() instead"); - } - - return name; - } - - @Override - public String toString() { - return new StringBuilder().append("FakeSavepoint [id=").append(id).append(", name=").append(name) - .append(']').toString(); - } - } - - final String id; - - private final List queries; - private final List savepoints; - - FakeTransaction() { - this.id = UUID.randomUUID().toString(); - this.queries = new LinkedList<>(); - this.savepoints = new ArrayList<>(); - } - - synchronized List getQueries() { - return new ArrayList<>(queries); - } - - synchronized List getSavepoints() { - return new ArrayList<>(savepoints); - } - - synchronized void logSavepointDetails(Logger log, FakeSavepoint s, String action) { - log.warn( - "[JDBC Compliant Mode] Savepoint(id=%d, name=%s) of transaction [%s](%d queries & %d savepoints) is %s.", - s.id, s.name, id, queries.size(), savepoints.size(), action); - } - - synchronized void logTransactionDetails(Logger log, String action) { - log.warn("[JDBC Compliant Mode] Transaction [%s](%d queries & %d savepoints) is %s.", id, queries.size(), - savepoints.size(), action); - - log.debug(() -> { - log.debug("[JDBC Compliant Mode] Transaction [%s] is %s - begin", id, action); - int total = queries.size(); - int counter = 1; - for (String queryId : queries) { - log.debug(" '%s', -- query (%d of %d) in transaction [%s]", queryId, counter++, total, id); - } - - total = savepoints.size(); - counter = 1; - for (FakeSavepoint savepoint : savepoints) { - log.debug(" %s (%d of %d) in transaction [%s]", savepoint, counter++, total, id); - } - return ClickHouseUtils.format("[JDBC Compliant Mode] Transaction [%s] is %s - end", id, action); - }); - } - - synchronized String newQuery(String queryId) { - if (queryId == null || queries.contains(queryId)) { - queryId = UUID.randomUUID().toString(); - } - - queries.add(queryId); - - return queryId; - } - - synchronized FakeSavepoint newSavepoint(String name) { - FakeSavepoint savepoint = new FakeSavepoint(queries.size(), name); - this.savepoints.add(savepoint); - return savepoint; - } - - synchronized void toSavepoint(FakeSavepoint savepoint) throws SQLException { - boolean found = false; - Iterator it = savepoints.iterator(); - while (it.hasNext()) { - FakeSavepoint s = it.next(); - if (found) { - it.remove(); - } else if (s == savepoint) { - found = true; - it.remove(); - } - } - - if (!found) { - throw SqlExceptionUtils.clientError("Invalid savepoint: " + savepoint); - } - queries.subList(savepoint.id, queries.size()).clear(); - } - - synchronized void clear() { - this.queries.clear(); - this.savepoints.clear(); - } -} diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/InputBasedPreparedStatement.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/InputBasedPreparedStatement.java index 9cbc8f1bd..14e5d261a 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/InputBasedPreparedStatement.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/InputBasedPreparedStatement.java @@ -316,7 +316,8 @@ public void addBatch() throws SQLException { int nullAsDefault = getNullAsDefault(); for (int i = 0, len = values.length; i < len; i++) { if (!flags[i]) { - throw SqlExceptionUtils.clientError(ClickHouseUtils.format("Missing value for parameter #%d", i + 1)); + throw SqlExceptionUtils + .clientError(ClickHouseUtils.format("Missing value for parameter #%d [%s]", i + 1, columns[i])); } ClickHouseColumn col = columns[i]; ClickHouseValue val = values[i]; diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/JdbcSavepoint.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/JdbcSavepoint.java new file mode 100644 index 000000000..ce35d0ff7 --- /dev/null +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/JdbcSavepoint.java @@ -0,0 +1,42 @@ +package com.clickhouse.jdbc.internal; + +import java.sql.SQLException; +import java.sql.Savepoint; + +import com.clickhouse.jdbc.SqlExceptionUtils; + +public class JdbcSavepoint implements Savepoint { + final int id; + final String name; + + JdbcSavepoint(int id, String name) { + this.id = id; + this.name = name; + } + + @Override + public int getSavepointId() throws SQLException { + if (name != null) { + throw SqlExceptionUtils + .clientError("Cannot get ID from a named savepoint, please use getSavepointName() instead"); + } + + return id; + } + + @Override + public String getSavepointName() throws SQLException { + if (name == null) { + throw SqlExceptionUtils + .clientError("Cannot get name from an un-named savepoint, please use getSavepointId() instead"); + } + + return name; + } + + @Override + public String toString() { + return new StringBuilder().append("JdbcSavepoint [id=").append(id).append(", name=").append(name) + .append(']').toString(); + } +} \ No newline at end of file diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/JdbcTransaction.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/JdbcTransaction.java new file mode 100644 index 000000000..4318d6df3 --- /dev/null +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/JdbcTransaction.java @@ -0,0 +1,152 @@ +package com.clickhouse.jdbc.internal; + +import java.sql.SQLException; +import java.util.Collections; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.UUID; + +import com.clickhouse.client.ClickHouseChecker; +import com.clickhouse.client.ClickHouseException; +import com.clickhouse.client.ClickHouseRequestManager; +import com.clickhouse.client.ClickHouseTransaction; +import com.clickhouse.client.ClickHouseUtils; +import com.clickhouse.client.logging.Logger; +import com.clickhouse.jdbc.SqlExceptionUtils; + +public class JdbcTransaction { + static final String ACTION_COMMITTED = "committed"; + static final String ACTION_ROLLBACK = "rolled back"; + + static final String ERROR_TX_NOT_STARTED = "Transaction not started"; + + protected final ClickHouseTransaction tx; + protected final String id; + protected final List queries; + protected final List savepoints; + + JdbcTransaction() { + this(null); + } + + public JdbcTransaction(ClickHouseTransaction tx) { + this.tx = tx; + this.id = tx != null ? tx.getId().asTupleString() : UUID.randomUUID().toString(); + this.queries = new LinkedList<>(); + this.savepoints = new LinkedList<>(); + } + + public void commit(Logger log) throws SQLException { + if (this.tx != null) { + try { + this.tx.commit(); + } catch (ClickHouseException e) { + throw SqlExceptionUtils.handle(e); + } + } else { + logTransactionDetails(log, ACTION_COMMITTED); + } + clear(); + } + + public void rollback(Logger log) throws SQLException { + if (this.tx != null) { + try { + this.tx.rollback(); + } catch (ClickHouseException e) { + throw SqlExceptionUtils.handle(e); + } + } else { + logTransactionDetails(log, JdbcTransaction.ACTION_ROLLBACK); + } + clear(); + } + + synchronized List getQueries() { + return Collections.unmodifiableList(queries); + } + + synchronized List getSavepoints() { + return Collections.unmodifiableList(savepoints); + } + + synchronized void logSavepointDetails(Logger log, JdbcSavepoint s, String action) { + log.warn( + "[JDBC Compliant Mode] Savepoint(id=%d, name=%s) of transaction [%s](%d queries & %d savepoints) is %s.", + s.id, s.name, id, queries.size(), savepoints.size(), action); + } + + synchronized void logTransactionDetails(Logger log, String action) { + if (tx != null) { + log.debug("%s (%d queries & %d savepoints) is %s", tx, queries.size(), + savepoints.size(), action); + } else { + log.warn("[JDBC Compliant Mode] Transaction [%s] (%d queries & %d savepoints) is %s.", id, queries.size(), + savepoints.size(), action); + } + + log.debug(() -> { + log.debug("[JDBC Compliant Mode] Transaction [%s] is %s - begin", id, action); + int total = queries.size(); + int counter = 1; + for (String queryId : queries) { + log.debug(" '%s', -- query (%d of %d) in transaction [%s]", queryId, counter++, total, id); + } + + total = savepoints.size(); + counter = 1; + for (JdbcSavepoint savepoint : savepoints) { + log.debug(" %s (%d of %d) in transaction [%s]", savepoint, counter++, total, id); + } + return ClickHouseUtils.format("[JDBC Compliant Mode] Transaction [%s] is %s - end", id, action); + }); + } + + synchronized String newQuery(String queryId) { + if (ClickHouseChecker.isNullOrEmpty(queryId) || queries.contains(queryId)) { + queryId = ClickHouseRequestManager.getInstance().createQueryId(); + } + + queries.add(queryId); + + return queryId; + } + + synchronized JdbcSavepoint newSavepoint(String name) { + JdbcSavepoint savepoint = new JdbcSavepoint(queries.size(), name); + this.savepoints.add(savepoint); + return savepoint; + } + + synchronized void toSavepoint(JdbcSavepoint savepoint) throws SQLException { + if (tx != null) { + try { + tx.rollback(); + } catch (ClickHouseException e) { + throw SqlExceptionUtils.handle(e); + } + } + boolean found = false; + Iterator it = savepoints.iterator(); + while (it.hasNext()) { + JdbcSavepoint s = it.next(); + if (found) { + it.remove(); + } else if (s == savepoint) { + found = true; + it.remove(); + } + } + + if (!found) { + throw SqlExceptionUtils.clientError("Invalid savepoint: " + savepoint); + } + queries.subList(savepoint.id, queries.size()).clear(); + } + + synchronized void clear() { + this.queries.clear(); + this.savepoints.clear(); + } +} diff --git a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseConnectionTest.java b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseConnectionTest.java index 24057fff2..ec8eb0ead 100644 --- a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseConnectionTest.java +++ b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseConnectionTest.java @@ -2,6 +2,7 @@ import java.sql.Array; import java.sql.Connection; +import java.sql.PreparedStatement; import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; @@ -28,6 +29,26 @@ public void testCreateArray() throws SQLException { } } + @Test // (groups = "integration") + public void testAutoCommitMode() throws Exception { + Properties props = new Properties(); + props.setProperty("transactionSupport", "true"); + + for (int i = 0; i < 10; i++) { + try (Connection conn = newConnection(props); Statement stmt = conn.createStatement()) { + stmt.execute("select 1, throwIf(" + i + " % 3 = 0)"); + stmt.executeQuery("select number, toDateTime(number), toString(number), throwIf(" + i + " % 5 = 0)" + + " from numbers(100000)"); + } catch (SQLException e) { + if (i % 3 == 0 || i % 5 == 0) { + Assert.assertEquals(e.getErrorCode(), 395); + } else { + Assert.fail("Should not have exception"); + } + } + } + } + @Test(groups = "integration") public void testNonExistDatabase() throws Exception { String database = UUID.randomUUID().toString(); @@ -176,4 +197,309 @@ public void testReadOnly() throws SQLException { Assert.assertEquals(exp.getErrorCode(), 164); } } + + @Test // (groups = "integration") + public void testTransaction() throws Exception { + testAutoCommit(); + testManualCommit(); + testNestedTransactions(); + testParallelTransactions(); + } + + @Test // (groups = "integration") + public void testAutoCommit() throws Exception { + Properties props = new Properties(); + props.setProperty("transactionSupport", "true"); + String tableName = "test_jdbc_tx_auto_commit"; + try (Connection c = newConnection(props); Statement s = c.createStatement()) { + s.execute("drop table if exists " + tableName + "; " + + "create table " + tableName + "(id UInt64) engine=MergeTree order by id"); + } + + try (ClickHouseConnection conn = newConnection(); + ClickHouseConnection txConn = newConnection(props); + Statement stmt = conn.createStatement(); + Statement txStmt = txConn.createStatement(); + PreparedStatement ps = conn.prepareStatement("insert into " + tableName); + PreparedStatement txPs = txConn.prepareStatement("insert into " + tableName)) { + Assert.assertTrue(conn.getAutoCommit()); + Assert.assertTrue(txConn.getAutoCommit()); + Assert.assertFalse(conn.isTransactionSupported()); + Assert.assertTrue(txConn.isTransactionSupported()); + Assert.assertFalse(conn.isImplicitTransactionSupported()); + if (txConn.getServerVersion().check("[22.7,)")) { + Assert.assertTrue(txConn.isImplicitTransactionSupported(), + "Implicit transaction is supported since 22.7"); + } else { + Assert.assertFalse(txConn.isImplicitTransactionSupported(), + "Implicit transaction is NOT supported before 22.7"); + } + + checkRowCount(stmt, "select 1", 1); + checkRowCount(txStmt, "select 1", 1); + + txStmt.execute("drop table if exists " + tableName + "; " + + "create table " + tableName + "(id UInt64) engine=MergeTree order by id"); + checkRowCount(stmt, tableName, 0); + checkRowCount(txStmt, tableName, 0); + + stmt.executeUpdate("insert into " + tableName + " values(1)"); + checkRowCount(stmt, tableName, 1); + checkRowCount(txStmt, tableName, 1); + + txStmt.executeUpdate("insert into " + tableName + " values(2)"); + checkRowCount(stmt, tableName, 2); + checkRowCount(txStmt, tableName, 2); + + try (Connection c = newConnection(props); Statement s = c.createStatement()) { + c.setAutoCommit(false); + s.executeUpdate("insert into " + tableName + " values(-1)"); + checkRowCount(stmt, tableName, 3); + checkRowCount(txStmt, tableName, 2); + checkRowCount(s, tableName, 3); + c.rollback(); + checkRowCount(stmt, tableName, 2); + checkRowCount(txStmt, tableName, 2); + checkRowCount(s, tableName, 2); + } + checkRowCount(stmt, tableName, 2); + checkRowCount(txStmt, tableName, 2); + + try (Connection c = newConnection(props); Statement s = c.createStatement()) { + c.setAutoCommit(false); + s.executeUpdate("insert into " + tableName + " values(-2)"); + checkRowCount(stmt, tableName, 3); + checkRowCount(txStmt, tableName, 2); + checkRowCount(s, tableName, 3); + } + checkRowCount(stmt, tableName, 3); + checkRowCount(txStmt, tableName, 3); + + ps.setInt(1, 3); + ps.addBatch(); + ps.setInt(1, 4); + ps.addBatch(); + ps.executeBatch(); + checkRowCount(stmt, tableName, 5); + checkRowCount(txStmt, tableName, 5); + + txPs.setInt(1, 5); + txPs.addBatch(); + txPs.setInt(1, 6); + txPs.addBatch(); + txPs.executeBatch(); + checkRowCount(stmt, tableName, 7); + checkRowCount(txStmt, tableName, 7); + } + } + + @Test // (groups = "integration") + public void testManualCommit() throws Exception { + Properties props = new Properties(); + props.setProperty("autoCommit", "false"); + Properties txProps = new Properties(); + txProps.putAll(props); + txProps.setProperty("transactionSupport", "true"); + String tableName = "test_jdbc_manual_tx"; + try (Connection c = newConnection(txProps); Statement s = c.createStatement()) { + s.execute("drop table if exists " + tableName + "; " + + "create table " + tableName + "(id UInt64, value String) engine=MergeTree order by id"); + } + + try (ClickHouseConnection conn = newConnection(props); + ClickHouseConnection txConn = newConnection(txProps); + Statement stmt = conn.createStatement(); + Statement txStmt = txConn.createStatement(); + PreparedStatement ps = conn.prepareStatement("insert into " + tableName); + PreparedStatement txPs = txConn.prepareStatement("insert into " + tableName)) { + Assert.assertFalse(conn.getAutoCommit()); + Assert.assertFalse(txConn.getAutoCommit()); + Assert.assertFalse(conn.isTransactionSupported()); + Assert.assertTrue(txConn.isTransactionSupported()); + Assert.assertFalse(conn.isImplicitTransactionSupported()); + if (txConn.getServerVersion().check("[22.7,)")) { + Assert.assertTrue(txConn.isImplicitTransactionSupported(), + "Implicit transaction is supported since 22.7"); + } else { + Assert.assertFalse(txConn.isImplicitTransactionSupported(), + "Implicit transaction is NOT supported before 22.7"); + } + + checkRowCount(stmt, "select 1", 1); + checkRowCount(txStmt, "select 1", 1); + txConn.commit(); + + checkRowCount(stmt, "select 1", 1); + checkRowCount(txStmt, "select 1", 1); + txConn.rollback(); + + checkRowCount(stmt, tableName, 0); + checkRowCount(txStmt, tableName, 0); + + txStmt.executeUpdate("insert into " + tableName + " values(0, '0')"); + checkRowCount(stmt, tableName, 1); + checkRowCount(txStmt, tableName, 1); + txConn.rollback(); + checkRowCount(stmt, tableName, 0); + checkRowCount(txStmt, tableName, 0); + + stmt.executeUpdate("insert into " + tableName + " values(1, 'a')"); + checkRowCount(stmt, tableName, 1); + checkRowCount(txStmt, tableName, 1); + + txStmt.executeUpdate("insert into " + tableName + " values(2, 'b')"); + checkRowCount(stmt, tableName, 2); + checkRowCount(txStmt, tableName, 2); + + try (Connection c = newConnection(txProps); Statement s = c.createStatement()) { + s.executeUpdate("insert into " + tableName + " values(-1, '-1')"); + checkRowCount(stmt, tableName, 3); + checkRowCount(txStmt, tableName, 2); + checkRowCount(s, tableName, 2); + c.rollback(); + checkRowCount(stmt, tableName, 2); + checkRowCount(txStmt, tableName, 2); + checkRowCount(s, tableName, 1); + } + checkRowCount(stmt, tableName, 2); + checkRowCount(txStmt, tableName, 2); + + try (Connection c = newConnection(txProps); Statement s = c.createStatement()) { + s.executeUpdate("insert into " + tableName + " values(3, 'c')"); + checkRowCount(stmt, tableName, 3); + checkRowCount(txStmt, tableName, 2); + checkRowCount(s, tableName, 2); + txConn.commit(); + checkRowCount(stmt, tableName, 3); + checkRowCount(txStmt, tableName, 2); + checkRowCount(s, tableName, 2); + } + checkRowCount(stmt, tableName, 3); + checkRowCount(txStmt, tableName, 2); + txConn.commit(); + checkRowCount(txStmt, tableName, 3); + + txConn.setAutoCommit(true); + Assert.assertTrue(txConn.getAutoCommit()); + try (Statement s = txConn.createStatement()) { + s.executeUpdate("insert into " + tableName + " values(4, 'd')"); + checkRowCount(stmt, tableName, 4); + checkRowCount(txStmt, tableName, 4); + checkRowCount(s, tableName, 4); + } + + try (Statement s = txConn.createStatement()) { + checkRowCount(stmt, tableName, 4); + checkRowCount(txStmt, tableName, 4); + checkRowCount(s, tableName, 4); + } + } + } + + @Test // (groups = "integration") + public void testNestedTransactions() throws Exception { + Properties props = new Properties(); + props.setProperty("autoCommit", "false"); + props.setProperty("transactionSupport", "true"); + String tableName = "test_jdbc_nested_tx"; + try (Connection c = newConnection(props); Statement s = c.createStatement()) { + s.execute("drop table if exists " + tableName + "; " + + "create table " + tableName + "(id UInt64) engine=MergeTree order by id"); + } + + try (Connection conn = newConnection(props); + Statement stmt = conn.createStatement(); + PreparedStatement ps = conn.prepareStatement("insert into " + tableName)) { + checkRowCount(stmt, tableName, 0); + stmt.executeQuery("insert into " + tableName + " values(1)"); + checkRowCount(stmt, tableName, 1); + ps.setInt(1, 2); + ps.executeUpdate(); + checkRowCount(stmt, tableName, 2); + ps.setInt(1, 3); + ps.executeBatch(); + checkRowCount(stmt, tableName, 2); + ps.setInt(1, 3); + ps.addBatch(); + ps.executeBatch(); + checkRowCount(stmt, tableName, 3); + try (Connection c = newConnection(); Statement s = c.createStatement()) { + checkRowCount(s, tableName, 3); + } + + conn.rollback(); + checkRowCount(stmt, tableName, 0); + try (Connection c = newConnection(); Statement s = c.createStatement()) { + checkRowCount(s, tableName, 0); + } + } + } + + @Test // (groups = "integration") + public void testParallelTransactions() throws Exception { + Properties props = new Properties(); + props.setProperty("autoCommit", "false"); + props.setProperty("transactionSupport", "true"); + String tableName = "test_jdbc_parallel_tx"; + try (Connection c = newConnection(props); Statement s = c.createStatement()) { + s.execute("drop table if exists " + tableName + "; " + + "create table " + tableName + "(id UInt64) engine=MergeTree order by id"); + } + + try (Connection conn1 = newConnection(props); + Connection conn2 = newConnection(props); + Statement stmt1 = conn1.createStatement(); + Statement stmt2 = conn2.createStatement(); + PreparedStatement ps1 = conn1.prepareStatement("insert into " + tableName); + PreparedStatement ps2 = conn2.prepareStatement("insert into " + tableName)) { + stmt1.executeUpdate("insert into " + tableName + " values(-1)"); + checkRowCount(stmt1, tableName, 1); + checkRowCount(stmt2, tableName, 0); + conn1.rollback(); + checkRowCount(stmt1, tableName, 0); + checkRowCount(stmt2, tableName, 0); + + stmt2.executeUpdate("insert into " + tableName + " values(-2)"); + checkRowCount(stmt1, tableName, 0); + checkRowCount(stmt2, tableName, 1); + conn2.commit(); + checkRowCount(stmt1, tableName, 0); + checkRowCount(stmt2, tableName, 1); + conn1.commit(); + checkRowCount(stmt1, tableName, 1); + checkRowCount(stmt2, tableName, 1); + + ps1.setInt(1, 1); + ps1.addBatch(); + ps1.setInt(1, 2); + ps1.addBatch(); + ps1.setInt(1, 3); + ps1.addBatch(); + ps1.executeBatch(); + checkRowCount(stmt1, tableName, 4); + checkRowCount(stmt2, tableName, 1); + conn1.commit(); + checkRowCount(stmt1, tableName, 4); + checkRowCount(stmt2, tableName, 1); + try (Connection c = newConnection(props); Statement s = c.createStatement()) { + checkRowCount(s, tableName, 4); + } + + ps2.setInt(1, 4); + ps2.addBatch(); + ps2.setInt(1, 5); + ps2.addBatch(); + ps2.setInt(1, 6); + // ps2.addBatch(); + ps2.executeBatch(); + checkRowCount(stmt1, tableName, 4); + checkRowCount(stmt2, tableName, 3); + conn2.commit(); + checkRowCount(stmt1, tableName, 4); + checkRowCount(stmt2, tableName, 6); + try (Connection c = newConnection(props); Statement s = c.createStatement()) { + checkRowCount(s, tableName, 6); + } + } + } } diff --git a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/JdbcIntegrationTest.java b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/JdbcIntegrationTest.java index d77a47747..59547fb3b 100644 --- a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/JdbcIntegrationTest.java +++ b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/JdbcIntegrationTest.java @@ -2,11 +2,14 @@ import java.sql.Connection; import java.sql.DriverManager; +import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; import java.util.Locale; import java.util.Properties; +import org.testng.Assert; + import com.clickhouse.client.BaseIntegrationTest; import com.clickhouse.client.ClickHouseNode; import com.clickhouse.client.ClickHouseProtocol; @@ -56,6 +59,16 @@ protected String buildJdbcUrl(ClickHouseProtocol protocol, String prefix, String return builder.toString(); } + protected void checkRowCount(Statement stmt, String queryOrTableName, int expectedRowCount) throws SQLException { + String sql = queryOrTableName.indexOf(' ') > 0 ? queryOrTableName + : "select count(1) from ".concat(queryOrTableName); + try (ResultSet rs = stmt.executeQuery(sql)) { + Assert.assertTrue(rs.next(), "Should have at least one record"); + Assert.assertEquals(rs.getInt(1), expectedRowCount); + Assert.assertFalse(rs.next(), "Should have only one record"); + } + } + public JdbcIntegrationTest() { String className = getClass().getSimpleName(); if (className.startsWith(CLASS_PREFIX)) { diff --git a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/internal/ClickHouseConnectionImplTest.java b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/internal/ClickHouseConnectionImplTest.java index 527cce12f..c84ace208 100644 --- a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/internal/ClickHouseConnectionImplTest.java +++ b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/internal/ClickHouseConnectionImplTest.java @@ -18,10 +18,11 @@ public void testManualCommit() throws SQLException { Assert.assertNull(conn.getTransaction(), "Should NOT have any transaction"); conn.setAutoCommit(false); Assert.assertEquals(conn.getAutoCommit(), false); - FakeTransaction tx = conn.getTransaction(); + JdbcTransaction tx = conn.getJdbcTrasaction(); Assert.assertNotNull(tx, "Should have transaction"); Assert.assertEquals(tx.getQueries().size(), 0); Assert.assertEquals(tx.getSavepoints().size(), 0); + Assert.assertEquals(tx.tx, conn.getTransaction()); try (ClickHouseStatement stmt = conn.createStatement()) { stmt.execute("select 1; select 2"); Assert.assertEquals(tx.getQueries().size(), 2); @@ -72,7 +73,7 @@ public void testManualCommit() throws SQLException { Assert.assertEquals(tx.getSavepoints().size(), 2); } conn.commit(); - FakeTransaction newTx = conn.getTransaction(); + JdbcTransaction newTx = conn.getJdbcTrasaction(); Assert.assertNotEquals(newTx, tx); Assert.assertNotNull(tx, "Should have transaction"); Assert.assertEquals(tx.getQueries().size(), 0); @@ -80,6 +81,7 @@ public void testManualCommit() throws SQLException { Assert.assertNotNull(newTx, "Should have transaction"); Assert.assertEquals(newTx.getQueries().size(), 0); Assert.assertEquals(newTx.getSavepoints().size(), 0); + Assert.assertEquals(newTx.tx, conn.getTransaction()); tx = newTx; try (ClickHouseStatement stmt = conn.createStatement()) { @@ -89,7 +91,7 @@ public void testManualCommit() throws SQLException { Assert.assertEquals(tx.getSavepoints().size(), 1); } conn.commit(); - newTx = conn.getTransaction(); + newTx = conn.getJdbcTrasaction(); Assert.assertNotEquals(newTx, tx); Assert.assertNotNull(tx, "Should have transaction"); Assert.assertEquals(tx.getQueries().size(), 0); @@ -97,6 +99,7 @@ public void testManualCommit() throws SQLException { Assert.assertNotNull(newTx, "Should have transaction"); Assert.assertEquals(newTx.getQueries().size(), 0); Assert.assertEquals(newTx.getSavepoints().size(), 0); + Assert.assertEquals(newTx.tx, conn.getTransaction()); } } @@ -107,10 +110,11 @@ public void testManualRollback() throws SQLException { Assert.assertNull(conn.getTransaction(), "Should NOT have any transaction"); conn.setAutoCommit(false); Assert.assertEquals(conn.getAutoCommit(), false); - FakeTransaction tx = conn.getTransaction(); + JdbcTransaction tx = conn.getJdbcTrasaction(); Assert.assertNotNull(tx, "Should have transaction"); Assert.assertEquals(tx.getQueries().size(), 0); Assert.assertEquals(tx.getSavepoints().size(), 0); + Assert.assertEquals(tx.tx, conn.getTransaction()); try (ClickHouseStatement stmt = conn.createStatement()) { stmt.execute("select 1; select 2"); Assert.assertEquals(tx.getQueries().size(), 2); @@ -161,7 +165,7 @@ public void testManualRollback() throws SQLException { Assert.assertEquals(tx.getSavepoints().size(), 2); } conn.rollback(); - FakeTransaction newTx = conn.getTransaction(); + JdbcTransaction newTx = conn.getJdbcTrasaction(); Assert.assertNotEquals(newTx, tx); Assert.assertNotNull(tx, "Should have transaction"); Assert.assertEquals(tx.getQueries().size(), 0); @@ -169,6 +173,7 @@ public void testManualRollback() throws SQLException { Assert.assertNotNull(newTx, "Should have transaction"); Assert.assertEquals(newTx.getQueries().size(), 0); Assert.assertEquals(newTx.getSavepoints().size(), 0); + Assert.assertEquals(newTx.tx, conn.getTransaction()); tx = newTx; try (ClickHouseStatement stmt = conn.createStatement()) { @@ -178,7 +183,7 @@ public void testManualRollback() throws SQLException { Assert.assertEquals(tx.getSavepoints().size(), 1); } conn.rollback(); - newTx = conn.getTransaction(); + newTx = conn.getJdbcTrasaction(); Assert.assertNotEquals(newTx, tx); Assert.assertNotNull(tx, "Should have transaction"); Assert.assertEquals(tx.getQueries().size(), 0); @@ -186,6 +191,7 @@ public void testManualRollback() throws SQLException { Assert.assertNotNull(newTx, "Should have transaction"); Assert.assertEquals(newTx.getQueries().size(), 0); Assert.assertEquals(newTx.getSavepoints().size(), 0); + Assert.assertEquals(newTx.tx, conn.getTransaction()); } } diff --git a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/internal/FakeTransactionTest.java b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/internal/JdbcTransactionTest.java similarity index 88% rename from clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/internal/FakeTransactionTest.java rename to clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/internal/JdbcTransactionTest.java index 27fc6c13d..adb1f4c75 100644 --- a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/internal/FakeTransactionTest.java +++ b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/internal/JdbcTransactionTest.java @@ -4,15 +4,13 @@ import java.util.Arrays; import java.util.Collections; -import com.clickhouse.jdbc.internal.FakeTransaction.FakeSavepoint; - import org.testng.Assert; import org.testng.annotations.Test; -public class FakeTransactionTest { +public class JdbcTransactionTest { @Test(groups = "unit") public void testQuery() { - FakeTransaction tx = new FakeTransaction(); + JdbcTransaction tx = new JdbcTransaction(); Assert.assertNotNull(tx.id); Assert.assertEquals(tx.getQueries(), Collections.emptyList()); Assert.assertEquals(tx.getSavepoints(), Collections.emptyList()); @@ -40,13 +38,13 @@ public void testQuery() { @Test(groups = "unit") public void testSavepoint() throws SQLException { - FakeTransaction tx = new FakeTransaction(); + JdbcTransaction tx = new JdbcTransaction(); Assert.assertNotNull(tx.id); Assert.assertEquals(tx.getQueries(), Collections.emptyList()); Assert.assertEquals(tx.getSavepoints(), Collections.emptyList()); - FakeSavepoint unnamedSavepoint = tx.newSavepoint(null); - FakeSavepoint s1 = unnamedSavepoint; + JdbcSavepoint unnamedSavepoint = tx.newSavepoint(null); + JdbcSavepoint s1 = unnamedSavepoint; Assert.assertEquals(unnamedSavepoint.id, 0); Assert.assertEquals(unnamedSavepoint.getSavepointId(), 0); Assert.assertNull(unnamedSavepoint.name, "Un-named savepoint should not have name"); @@ -54,8 +52,8 @@ public void testSavepoint() throws SQLException { Assert.assertEquals(tx.getQueries(), Collections.emptyList()); Assert.assertEquals(tx.getSavepoints(), Collections.singleton(unnamedSavepoint)); - FakeSavepoint namedSavepoint = tx.newSavepoint("tmp"); - FakeSavepoint s2 = namedSavepoint; + JdbcSavepoint namedSavepoint = tx.newSavepoint("tmp"); + JdbcSavepoint s2 = namedSavepoint; Assert.assertEquals(namedSavepoint.id, 0); Assert.assertThrows(SQLException.class, () -> s2.getSavepointId()); Assert.assertEquals(namedSavepoint.name, "tmp"); @@ -76,7 +74,7 @@ public void testSavepoint() throws SQLException { Assert.assertEquals(tx.getSavepoints(), Collections.emptyList()); String queryId = tx.newQuery(null); - FakeSavepoint s3 = unnamedSavepoint = tx.newSavepoint(null); + JdbcSavepoint s3 = unnamedSavepoint = tx.newSavepoint(null); Assert.assertEquals(unnamedSavepoint.id, 1); Assert.assertEquals(unnamedSavepoint.getSavepointId(), 1); Assert.assertNull(unnamedSavepoint.name, "Un-named savepoint should not have name"); @@ -85,7 +83,7 @@ public void testSavepoint() throws SQLException { Assert.assertEquals(tx.getSavepoints(), Collections.singleton(unnamedSavepoint)); tx.newQuery(null); - FakeSavepoint s4 = namedSavepoint = tx.newSavepoint("tmp"); + JdbcSavepoint s4 = namedSavepoint = tx.newSavepoint("tmp"); Assert.assertEquals(namedSavepoint.id, 2); Assert.assertThrows(SQLException.class, () -> s4.getSavepointId()); Assert.assertEquals(namedSavepoint.name, "tmp"); diff --git a/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/integration/BatchInsertsTest.java b/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/integration/BatchInsertsTest.java index 978e9dc1c..f402c12c7 100644 --- a/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/integration/BatchInsertsTest.java +++ b/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/integration/BatchInsertsTest.java @@ -82,7 +82,7 @@ public void testBatchInsert2() throws Exception { "string String," + "int32 Int32," + "float64 Float64" + - ") ENGINE = MergeTree(date, (date), 8192)" + ") ENGINE = MergeTree order by date" ); Date date = new Date(dateFormat.parse("1989-01-30").getTime()); @@ -194,7 +194,7 @@ public void testSimpleInsert() throws Exception { "string String," + "int32 Int32," + "float64 Float64" + - ") ENGINE = MergeTree(date, (date), 8192)" + ") ENGINE = MergeTree order by date" ); Date date = new Date(dateFormat.parse("1989-01-30").getTime()); @@ -237,7 +237,7 @@ public void batchInsertNulls() throws Exception { "string Nullable(String)," + "int32 Nullable(Int32)," + "float64 Nullable(Float64)" + - ") ENGINE = MergeTree(date, (date), 8192)" + ") ENGINE = MergeTree order by date" ); ClickHousePreparedStatement statement = (ClickHousePreparedStatement) connection.prepareStatement( diff --git a/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/integration/ClickHouseDatabaseMetadataTest.java b/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/integration/ClickHouseDatabaseMetadataTest.java index 6dabf6b29..013834c91 100644 --- a/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/integration/ClickHouseDatabaseMetadataTest.java +++ b/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/integration/ClickHouseDatabaseMetadataTest.java @@ -173,7 +173,7 @@ private Object[][] getTableEngines() { new String[] {"TinyLog"}, new String[] {"Log"}, new String[] {"Memory"}, - new String[] {"MergeTree(foo, (foo), 8192)"} + new String[] {"MergeTree order by foo"} }; // unfortunately this is hard to test // new String[] {"Dictionary(myDict)"}, diff --git a/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/integration/RowBinaryStreamTest.java b/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/integration/RowBinaryStreamTest.java index 4253d4ed9..54e881675 100644 --- a/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/integration/RowBinaryStreamTest.java +++ b/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/integration/RowBinaryStreamTest.java @@ -548,7 +548,7 @@ public void testTimeZone() throws Exception { final ClickHouseStatement statement = connection.createStatement(); connection.createStatement().execute("DROP TABLE IF EXISTS binary_tz"); connection.createStatement().execute( - "CREATE TABLE binary_tz (date Date, dateTime DateTime) ENGINE = MergeTree(date, (date), 8192)"); + "CREATE TABLE binary_tz (date Date, dateTime DateTime) ENGINE = MergeTree order by date"); // final Date date1 = new Date(1497474018000L);