From b8a2e1ce9456096104c90340b07d1ae6ccc9e8f7 Mon Sep 17 00:00:00 2001 From: Zhichun Wu Date: Mon, 13 Dec 2021 21:48:33 +0800 Subject: [PATCH 1/6] Separate thread pool for I/O, add more tests for streaming, along with other minor changes for code clean up --- .../com/clickhouse/client/AbstractClient.java | 26 +-- .../client/ClickHouseClientBuilder.java | 25 +-- .../client/ClickHouseException.java | 2 +- .../client/ClickHouseInputStream.java | 84 +++++--- .../client/ClickHouseThreadFactory.java | 49 +++++ .../clickhouse/client/ClickHouseUtils.java | 28 +-- .../client/config/ClickHouseClientOption.java | 7 +- .../client/config/ClickHouseDefaults.java | 5 + .../client/data/BinaryStreamUtils.java | 22 +-- .../client/data/ClickHousePipedStream.java | 2 +- .../clickhouse/client/AbstractClientTest.java | 14 +- .../client/ClickHouseExceptionTest.java | 26 +-- .../client/ClickHouseInputStreamTest.java | 183 ++++++++++++++++++ .../data/ClickHousePipedStreamTest.java | 6 +- .../data/ClickhouseLZ4InputStreamTest.java | 91 +++++++-- .../client/http/ClickHouseHttpClient.java | 3 +- .../client/http/DefaultHttpConnection.java | 4 +- .../http/ClickHouseResponseHandler.java | 11 +- .../client/http/DefaultHttpConnection.java | 73 +++++-- .../http/DefaultHttpConnectionTest.java | 2 +- .../jdbc/ClickHouseStatementTest.java | 20 ++ 21 files changed, 528 insertions(+), 155 deletions(-) create mode 100644 clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseThreadFactory.java create mode 100644 clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseInputStreamTest.java diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/AbstractClient.java b/clickhouse-client/src/main/java/com/clickhouse/client/AbstractClient.java index aa95a8f4e..cf3c49d5b 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/AbstractClient.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/AbstractClient.java @@ -170,9 +170,8 @@ public void init(ClickHouseConfig config) { this.config = config; if (this.executor == null) { // only initialize once int threads = config.getMaxThreadsPerClient(); - this.executor = threads <= 0 ? ClickHouseClient.getExecutorService() - : ClickHouseUtils.newThreadPool(getClass().getSimpleName(), threads, - config.getMaxQueuedRequests()); + this.executor = threads < 1 ? ClickHouseClient.getExecutorService() + : ClickHouseUtils.newThreadPool(this, threads, config.getMaxQueuedRequests()); } initialized = true; @@ -196,27 +195,16 @@ public final void close() { try { server = null; - if (executor != null) { - executor.shutdown(); - } - if (connection != null) { closeConnection(connection, false); + connection = null; } - // shutdown* won't shutdown commonPool, so awaitTermination will always time out - // on the other hand, for a client-specific thread pool, we'd better shut it - // down for real - if (executor != null && config.getMaxThreadsPerClient() > 0 - && !executor.awaitTermination(config.getConnectionTimeout(), TimeUnit.MILLISECONDS)) { - executor.shutdownNow(); + // avoid shutting down shared thread pool + if (executor != null && config.getMaxThreadsPerClient() > 0 && !executor.isTerminated()) { + executor.shutdown(); } - executor = null; - connection = null; - } catch (InterruptedException e) { - log.warn("Got interrupted when closing client", e); - Thread.currentThread().interrupt(); } catch (Exception e) { log.warn("Exception occurred when closing client", e); } finally { @@ -226,7 +214,7 @@ public final void close() { closeConnection(connection, true); } - if (executor != null) { + if (executor != null && config.getMaxThreadsPerClient() > 0) { executor.shutdownNow(); } } finally { 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 01c191d4e..eef6aa462 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseClientBuilder.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseClientBuilder.java @@ -6,7 +6,6 @@ import java.util.Objects; import java.util.ServiceLoader; import java.util.concurrent.ExecutorService; -import java.util.concurrent.ForkJoinPool; import com.clickhouse.client.config.ClickHouseOption; import com.clickhouse.client.config.ClickHouseDefaults; @@ -23,23 +22,17 @@ public class ClickHouseClientBuilder { static { int maxThreads = (int) ClickHouseDefaults.MAX_THREADS.getEffectiveDefaultValue(); int maxRequests = (int) ClickHouseDefaults.MAX_REQUESTS.getEffectiveDefaultValue(); + long keepAliveTimeoutMs = (long) ClickHouseDefaults.THREAD_KEEPALIVE_TIMEOUT.getEffectiveDefaultValue(); - if (maxThreads <= 0 && maxRequests <= 0) { - // java -XX:+UnlockDiagnosticVMOptions -XX:NativeMemoryTracking=summary - // -XX:+PrintNMTStatistics -version - defaultExecutor = ForkJoinPool.commonPool(); - } else { - if (maxThreads <= 0) { - maxThreads = Runtime.getRuntime().availableProcessors(); - } - - if (maxRequests <= 0) { - maxRequests = 0; - } - - defaultExecutor = ClickHouseUtils.newThreadPool(ClickHouseClient.class.getSimpleName(), maxThreads, - maxRequests); + if (maxThreads <= 0) { + maxThreads = Runtime.getRuntime().availableProcessors(); + } + if (maxRequests <= 0) { + maxRequests = 0; } + + defaultExecutor = ClickHouseUtils.newThreadPool(ClickHouseClient.class.getSimpleName(), maxThreads, + maxThreads * 2, maxRequests, keepAliveTimeoutMs); } protected ClickHouseConfig config; 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 6904f1895..ff5560bec 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseException.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseException.java @@ -37,7 +37,7 @@ private static String buildErrorMessage(int code, String message, ClickHouseNode } if (server != null) { - builder.append(" on server ").append(server); + builder.append(", server ").append(server); } return builder.toString(); 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 76c83d976..fb85bcd5b 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseInputStream.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseInputStream.java @@ -11,13 +11,17 @@ import java.util.concurrent.TimeUnit; /** - * Extended input stream. + * Extended input stream for read optimization. */ public abstract class ClickHouseInputStream extends InputStream { + /** + * Empty byte array. + */ + public static final byte[] EMPTY_BYTES = new byte[0]; /** * Empty and read-only byte buffer. */ - public static final ByteBuffer EMPTY = ByteBuffer.wrap(new byte[0]).asReadOnlyBuffer(); + public static final ByteBuffer EMPTY_BUFFER = ByteBuffer.wrap(EMPTY_BYTES).asReadOnlyBuffer(); static final class BlockingInputStream extends ClickHouseInputStream { private final BlockingQueue queue; @@ -28,7 +32,7 @@ static final class BlockingInputStream extends ClickHouseInputStream { private boolean closed; BlockingInputStream(BlockingQueue queue, int timeout) { - this.queue = queue; + this.queue = ClickHouseChecker.nonNull(queue, "Queue"); this.timeout = timeout; this.buffer = null; @@ -37,10 +41,12 @@ static final class BlockingInputStream extends ClickHouseInputStream { private void ensureOpen() throws IOException { if (closed) { - throw new IOException("Stream has been closed"); + throw new IOException( + ClickHouseUtils.format("Blocking stream(queue: %d, buffer: %d) has been closed", + queue.size(), buffer != null ? buffer.remaining() : 0)); } - if (buffer == null || (buffer != EMPTY && !buffer.hasRemaining())) { + if (buffer == null || (buffer != EMPTY_BUFFER && !buffer.hasRemaining())) { updateBuffer(); } } @@ -65,9 +71,11 @@ private int updateBuffer() throws IOException { @Override public int available() throws IOException { - ensureOpen(); + if (closed || buffer == EMPTY_BUFFER) { + return 0; + } - return buffer.remaining(); + return (buffer == null || !buffer.hasRemaining()) ? updateBuffer() : buffer.remaining(); } @Override @@ -87,7 +95,7 @@ public void close() throws IOException { public byte readByte() throws IOException { ensureOpen(); - if (buffer == EMPTY) { + if (buffer == EMPTY_BUFFER) { close(); throw new EOFException(); } @@ -99,7 +107,7 @@ public byte readByte() throws IOException { public int read() throws IOException { ensureOpen(); - if (buffer == EMPTY) { + if (buffer == EMPTY_BUFFER) { return -1; } @@ -112,7 +120,7 @@ public int read(byte[] b, int off, int len) throws IOException { int counter = 0; while (len > 0) { - if (buffer == EMPTY) { + if (buffer == EMPTY_BUFFER) { return counter > 0 ? counter : -1; } @@ -162,7 +170,7 @@ public long skip(long n) throws IOException { // peforms better but this is a bit tricky if (n == Long.MAX_VALUE) { long counter = buffer.remaining(); - while (buffer != EMPTY && buffer.limit() > 0) { + while (buffer != EMPTY_BUFFER && buffer.limit() > 0) { counter += buffer.limit(); updateBuffer(); } @@ -184,6 +192,12 @@ static final class WrappedInputStream extends ClickHouseInputStream { closed = false; } + private void ensureOpen() throws IOException { + if (closed) { + throw new IOException(ClickHouseUtils.format("Wrapped stream(%s) has been closed", in)); + } + } + @Override public int available() throws IOException { return !closed ? in.available() : 0; @@ -191,13 +205,19 @@ public int available() throws IOException { @Override public byte readByte() throws IOException { + ensureOpen(); + int v = in.read(); - if (v == -1) { - close(); - throw new EOFException(); + if (v != -1) { + return (byte) v; } - return (byte) v; + try { + close(); + } catch (IOException e) { + // ignore + } + throw new EOFException(); } @Override @@ -207,25 +227,30 @@ public boolean isClosed() { @Override public void close() throws IOException { - try { - in.close(); - } finally { - closed = true; + if (!closed) { + try { + in.close(); + } finally { + closed = true; + } } } @Override public int read() throws IOException { + ensureOpen(); return in.read(); } @Override public int read(byte[] b, int off, int len) throws IOException { + ensureOpen(); return in.read(b, off, len); } @Override public long skip(long n) throws IOException { + ensureOpen(); return in.skip(n); } } @@ -238,21 +263,23 @@ public long skip(long n) throws IOException { * @return wrapped input */ public static ClickHouseInputStream of(BlockingQueue queue, int timeout) { - return new BlockingInputStream(ClickHouseChecker.nonNull(queue, "queue"), timeout); + return new BlockingInputStream(queue, timeout); } /** * Wraps the given input stream. * * @param input non-null input stream - * @return wrapped input + * @return wrapped input, or the same input if it's instance of + * {@link ClickHouseInputStream} */ public static ClickHouseInputStream of(InputStream input) { return input instanceof ClickHouseInputStream ? (ClickHouseInputStream) input : new WrappedInputStream(input); } /** - * Reads an unsigned byte from the input stream. + * Reads an unsigned byte from the input stream. Unlike {@link #read()}, it will + * throw {@link IOException} if the input stream has been closed. * * @return unsigned byte * @throws IOException when failed to read value from input stream or reached @@ -263,8 +290,10 @@ public int readUnsignedByte() throws IOException { } /** - * Reads one single byte from the input stream. It's supposed to be faster than - * {@link #read()}. + * Reads one single byte from the input stream. Unlike {@link #read()}, it will + * throw {@link IOException} if the input stream has been closed. In general, + * this method should be faster than {@link #read()}, especially when it's an + * input stream backed by byte[] or {@link java.nio.ByteBuffer}. * * @return byte value if present * @throws IOException when failed to read value from input stream or reached @@ -274,7 +303,8 @@ public int readUnsignedByte() throws IOException { /** * Reads {@code length} bytes from the input stream. It behaves in the same - * way as {@link java.io.DataInput#readFully(byte[])}. + * way as {@link java.io.DataInput#readFully(byte[])}, and it will throw + * {@link IOException} when the input stream has been closed. * * @param length number of bytes to read * @return byte array and its length should be {@code length} @@ -282,6 +312,10 @@ public int readUnsignedByte() throws IOException { * retrieve all bytes, or reached end of the stream */ public byte[] readBytes(int length) throws IOException { + if (length <= 0) { + return EMPTY_BYTES; + } + byte[] bytes = new byte[length]; for (int l = length, c = 0, n = 0; l > 0; l -= n) { diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseThreadFactory.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseThreadFactory.java new file mode 100644 index 000000000..bdb7608ec --- /dev/null +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseThreadFactory.java @@ -0,0 +1,49 @@ +package com.clickhouse.client; + +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.atomic.AtomicInteger; + +public class ClickHouseThreadFactory implements ThreadFactory { + private final boolean daemon; + private final int priority; + + private final ThreadGroup group; + private final String namePrefix; + private final AtomicInteger threadNumber; + + public ClickHouseThreadFactory(Object owner) { + this(owner, false, Thread.NORM_PRIORITY); + } + + public ClickHouseThreadFactory(Object owner, boolean daemon, int priority) { + String prefix = null; + if (owner instanceof String) { + prefix = ((String) owner).trim(); + } else if (owner != null) { + prefix = new StringBuilder().append(owner.getClass().getSimpleName()).append('@').append(owner.hashCode()) + .toString(); + } + this.daemon = daemon; + this.priority = ClickHouseChecker.between(priority, "Priority", Thread.MIN_PRIORITY, Thread.MAX_PRIORITY); + + SecurityManager s = System.getSecurityManager(); + group = s != null ? s.getThreadGroup() : Thread.currentThread().getThreadGroup(); + namePrefix = !ClickHouseChecker.isNullOrBlank(prefix) ? prefix + : new StringBuilder().append(getClass().getSimpleName()).append('@').append(hashCode()) + .append('-').toString(); + threadNumber = new AtomicInteger(1); + } + + @Override + public Thread newThread(Runnable r) { + Thread t = new Thread(group, r, namePrefix + threadNumber.getAndIncrement(), 0); + if (daemon != t.isDaemon()) { + t.setDaemon(daemon); + } + if (priority != t.getPriority()) { + t.setPriority(priority); + } + // t.setUncaughtExceptionHandler(null); + return t; + } +} diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseUtils.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseUtils.java index 7e21e9dbc..44a54f9f1 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseUtils.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseUtils.java @@ -14,11 +14,9 @@ import java.nio.file.Paths; import java.util.ArrayDeque; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.Deque; -import java.util.HashMap; import java.util.LinkedHashMap; import java.util.LinkedList; import java.util.List; @@ -30,7 +28,6 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; import java.util.concurrent.ThreadPoolExecutor; import java.util.function.Supplier; @@ -109,19 +106,22 @@ private static T findFirstService(Class serviceInterface) { return service; } - public static ExecutorService newThreadPool(String owner, int maxThreads, int maxRequests) { + public static ExecutorService newThreadPool(Object owner, int maxThreads, int maxRequests) { + return newThreadPool(owner, maxThreads, 0, maxRequests, 0L); + } + + public static ExecutorService newThreadPool(Object owner, int coreThreads, int maxThreads, int maxRequests, + long keepAliveTimeoutMs) { BlockingQueue queue = maxRequests > 0 ? new ArrayBlockingQueue<>(maxRequests) : new LinkedBlockingQueue<>(); - - return new ThreadPoolExecutor(1, maxThreads < 1 ? 1 : maxThreads, 0L, TimeUnit.MILLISECONDS, queue, - new ThreadFactory() { - @Override - public Thread newThread(Runnable r) { - Thread thread = new Thread(r, owner); - thread.setUncaughtExceptionHandler(null); - return thread; - } - }, new ThreadPoolExecutor.AbortPolicy()); + if (coreThreads < 2) { + coreThreads = 2; + } + if (maxThreads < coreThreads) { + maxThreads = coreThreads; + } + return new ThreadPoolExecutor(coreThreads, maxThreads, keepAliveTimeoutMs < 0L ? 0L : keepAliveTimeoutMs, + TimeUnit.MILLISECONDS, queue, new ClickHouseThreadFactory(owner), new ThreadPoolExecutor.AbortPolicy()); } public static boolean isCloseBracket(char ch) { 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 6cc43f8cb..fa9d8c361 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 @@ -24,7 +24,6 @@ public enum ClickHouseClientOption implements ClickHouseOption { */ CLIENT_NAME("client_name", "ClickHouse Java Client", "Client name, which is either 'client_name' or 'http_user_agent' shows up in system.query_log table."), - /** * Whether server will compress response to client or not. */ @@ -42,7 +41,8 @@ public enum ClickHouseClientOption implements ClickHouseOption { * Compression algorithm server will use to decompress request, when * {@link #DECOMPRESS} is {@code true}. */ - DECOMPRESS_ALGORITHM("decompress_alogrithm", ClickHouseCompression.GZIP, "Algorithm for decompressing request."), + DECOMPRESS_ALGORITHM("decompress_alogrithm", ClickHouseCompression.GZIP, + "Algorithm for decompressing request."), /** * Compression level for compressing server response. */ @@ -143,7 +143,8 @@ public enum ClickHouseClientOption implements ClickHouseOption { /** * SSL mode. */ - SSL_MODE("sslmode", ClickHouseSslMode.STRICT, "verify or not certificate: none (don't verify), strict (verify)"), + SSL_MODE("sslmode", ClickHouseSslMode.STRICT, + "verify or not certificate: none (don't verify), strict (verify)"), /** * SSL root certificiate. */ diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/config/ClickHouseDefaults.java b/clickhouse-client/src/main/java/com/clickhouse/client/config/ClickHouseDefaults.java index feb36664e..f8172a3d4 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/config/ClickHouseDefaults.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/config/ClickHouseDefaults.java @@ -69,6 +69,11 @@ public enum ClickHouseDefaults implements ClickHouseOption { * Max requests. */ MAX_REQUESTS("max_requests", 0, "Maximum size of shared thread pool, 0 means no limit."), + /** + * Thread keep alive timeout in milliseconds. + */ + THREAD_KEEPALIVE_TIMEOUT("thread_keepalive_timeout", 0L, + "Thread keep alive timeout in milliseconds. 0 or negative number means additional thread will be closed immediately after execution completed."), /** * Server time zone, defaults to {@code UTC}. */ diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/data/BinaryStreamUtils.java b/clickhouse-client/src/main/java/com/clickhouse/client/data/BinaryStreamUtils.java index e1725e3f7..ac41e4f22 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/data/BinaryStreamUtils.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/data/BinaryStreamUtils.java @@ -76,6 +76,9 @@ public final class BinaryStreamUtils { public static final BigDecimal NANOS = new BigDecimal(BigInteger.TEN.pow(9)); + private static final int[] BASES = new int[] { 1, 10, 100, 1000, 10000, 100000, 1000000, 10000000, 100000000, + 1000000000 }; + private static > T toEnum(int value, Class enumType) { for (T t : ClickHouseChecker.nonNull(enumType, "enumType").getEnumConstants()) { if (t.ordinal() == value) { @@ -1403,11 +1406,7 @@ public static LocalDateTime readDateTime64(ClickHouseInputStream input, int scal long value = readInt64(input); int nanoSeconds = 0; if (ClickHouseChecker.between(scale, ClickHouseValues.PARAM_SCALE, 0, 9) > 0) { - int factor = 1; - for (int i = 0; i < scale; i++) { - factor *= 10; - } - + int factor = BASES[scale]; nanoSeconds = (int) (value % factor); value /= factor; if (nanoSeconds < 0) { @@ -1415,9 +1414,7 @@ public static LocalDateTime readDateTime64(ClickHouseInputStream input, int scal value--; } if (nanoSeconds > 0L) { - for (int i = 9 - scale; i > 0; i--) { - nanoSeconds *= 10; - } + nanoSeconds *= BASES[9 - scale]; } } @@ -1456,15 +1453,10 @@ public static void writeDateTime64(OutputStream output, LocalDateTime value, int : value.atZone(tz.toZoneId()).toEpochSecond(), ClickHouseValues.TYPE_DATE_TIME, DATETIME64_MIN, DATETIME64_MAX); if (ClickHouseChecker.between(scale, ClickHouseValues.PARAM_SCALE, 0, 9) > 0) { - for (int i = 0; i < scale; i++) { - v *= 10; - } + v *= BASES[scale]; int nanoSeconds = value.getNano(); if (nanoSeconds > 0L) { - for (int i = 9 - scale; i > 0; i--) { - nanoSeconds /= 10; - } - v += nanoSeconds; + v += nanoSeconds / BASES[9 - scale]; } } diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHousePipedStream.java b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHousePipedStream.java index de4ca2f02..86f468812 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHousePipedStream.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHousePipedStream.java @@ -81,7 +81,7 @@ public void close() throws IOException { flush(); - buffer = ClickHouseInputStream.EMPTY; + buffer = ClickHouseInputStream.EMPTY_BUFFER; try { if (timeout > 0) { if (!queue.offer(buffer, timeout, TimeUnit.MILLISECONDS)) { diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/AbstractClientTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/AbstractClientTest.java index 8a4f34f28..da8616ada 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/AbstractClientTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/AbstractClientTest.java @@ -19,7 +19,7 @@ protected Object[] newConnection(Object[] connection, ClickHouseNode server, Cli if (connection != null) { closeConnection(connection, false); } - + return new Object[] { request.getConfig(), server }; } @@ -124,7 +124,11 @@ public void testInit() { ClickHouseConfig config = new ClickHouseConfig(); sc.init(config); - Assert.assertEquals(sc.getExecutor(), ClickHouseClient.getExecutorService()); + if (config.getMaxThreadsPerClient() > 0) { + Assert.assertNotEquals(sc.getExecutor(), ClickHouseClient.getExecutorService()); + } else { + Assert.assertEquals(sc.getExecutor(), ClickHouseClient.getExecutorService()); + } Assert.assertTrue(sc.isInitialized()); Assert.assertTrue(sc.getConfig() == config); Assert.assertNull(sc.getServer()); @@ -133,7 +137,11 @@ public void testInit() { ClickHouseConfig newConfig = new ClickHouseConfig(); sc.init(newConfig); - Assert.assertEquals(sc.getExecutor(), ClickHouseClient.getExecutorService()); + if (config.getMaxThreadsPerClient() > 0) { + Assert.assertNotEquals(sc.getExecutor(), ClickHouseClient.getExecutorService()); + } else { + Assert.assertEquals(sc.getExecutor(), ClickHouseClient.getExecutorService()); + } Assert.assertTrue(sc.isInitialized()); Assert.assertTrue(sc.getConfig() != config); Assert.assertEquals(sc.getConnection(req), new Object[] { req.getConfig(), req.getServer() }); 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 ad41f1bec..ca31c37b9 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseExceptionTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseExceptionTest.java @@ -17,19 +17,19 @@ public void testConstructorWithCause() { e = new ClickHouseException(233, (Throwable) null, server); Assert.assertEquals(e.getErrorCode(), 233); Assert.assertNull(e.getCause()); - Assert.assertEquals(e.getMessage(), "Unknown error 233 on server " + server); + Assert.assertEquals(e.getMessage(), "Unknown error 233, server " + server); Throwable cause = new IllegalArgumentException(); e = new ClickHouseException(123, cause, server); Assert.assertEquals(e.getErrorCode(), 123); Assert.assertEquals(e.getCause(), cause); - Assert.assertEquals(e.getMessage(), "Unknown error 123 on server " + server); + Assert.assertEquals(e.getMessage(), "Unknown error 123, server " + server); cause = new IllegalArgumentException("Some error"); e = new ClickHouseException(111, cause, server); Assert.assertEquals(e.getErrorCode(), 111); Assert.assertEquals(e.getCause(), cause); - Assert.assertEquals(e.getMessage(), "Some error on server " + server); + Assert.assertEquals(e.getMessage(), "Some error, server " + server); } @Test(groups = { "unit" }) @@ -43,17 +43,17 @@ public void testConstructorWithoutCause() { e = new ClickHouseException(233, (String) null, server); Assert.assertEquals(e.getErrorCode(), 233); Assert.assertNull(e.getCause()); - Assert.assertEquals(e.getMessage(), "Unknown error 233 on server " + server); + Assert.assertEquals(e.getMessage(), "Unknown error 233, server " + server); e = new ClickHouseException(123, "", server); Assert.assertEquals(e.getErrorCode(), 123); Assert.assertNull(e.getCause()); - Assert.assertEquals(e.getMessage(), "Unknown error 123 on server " + server); + Assert.assertEquals(e.getMessage(), "Unknown error 123, server " + server); e = new ClickHouseException(111, "Some error", server); Assert.assertEquals(e.getErrorCode(), 111); Assert.assertNull(e.getCause()); - Assert.assertEquals(e.getMessage(), "Some error on server " + server); + Assert.assertEquals(e.getMessage(), "Some error, server " + server); } @Test(groups = { "unit" }) @@ -64,12 +64,12 @@ public void testHandleException() { Assert.assertEquals(e.getErrorCode(), ClickHouseException.ERROR_UNKNOWN); Assert.assertEquals(e.getCause(), cause); Assert.assertEquals(e.getMessage(), - "Unknown error " + ClickHouseException.ERROR_UNKNOWN + " on server " + server); + "Unknown error " + ClickHouseException.ERROR_UNKNOWN + ", server " + server); e = ClickHouseException.of("Some error", server); Assert.assertEquals(e.getErrorCode(), ClickHouseException.ERROR_UNKNOWN); Assert.assertNull(e.getCause()); - Assert.assertEquals(e.getMessage(), "Some error on server " + server); + Assert.assertEquals(e.getMessage(), "Some error, server " + server); Assert.assertEquals(e, ClickHouseException.of(e, server)); @@ -78,32 +78,32 @@ public void testHandleException() { Assert.assertEquals(e.getErrorCode(), ClickHouseException.ERROR_UNKNOWN); Assert.assertEquals(e.getCause(), cause); Assert.assertEquals(e.getMessage(), - "Unknown error " + ClickHouseException.ERROR_UNKNOWN + " on server " + server); + "Unknown error " + ClickHouseException.ERROR_UNKNOWN + ", server " + server); e = ClickHouseException.of((ExecutionException) null, server); Assert.assertEquals(e.getErrorCode(), ClickHouseException.ERROR_UNKNOWN); Assert.assertNull(e.getCause()); Assert.assertEquals(e.getMessage(), - "Unknown error " + ClickHouseException.ERROR_UNKNOWN + " on server " + server); + "Unknown error " + ClickHouseException.ERROR_UNKNOWN + ", server " + server); cause = new ExecutionException(new ClickHouseException(-100, (Throwable) null, server)); e = ClickHouseException.of(cause, server); Assert.assertEquals(e, cause.getCause()); Assert.assertEquals(e.getErrorCode(), -100); Assert.assertNull(e.getCause()); - Assert.assertEquals(e.getMessage(), "Unknown error -100 on server " + server); + Assert.assertEquals(e.getMessage(), "Unknown error -100, server " + server); cause = new ExecutionException(new IllegalArgumentException()); e = ClickHouseException.of(cause, server); Assert.assertEquals(e.getErrorCode(), ClickHouseException.ERROR_UNKNOWN); Assert.assertEquals(e.getCause(), cause.getCause()); Assert.assertEquals(e.getMessage(), - "Unknown error " + ClickHouseException.ERROR_UNKNOWN + " on server " + server); + "Unknown error " + ClickHouseException.ERROR_UNKNOWN + ", server " + server); cause = new ExecutionException(new IllegalArgumentException("Code: 12345. Something goes wrong...")); e = ClickHouseException.of(cause, server); Assert.assertEquals(e.getErrorCode(), 12345); Assert.assertEquals(e.getCause(), cause.getCause()); - Assert.assertEquals(e.getMessage(), cause.getCause().getMessage() + " on server " + server); + Assert.assertEquals(e.getMessage(), cause.getCause().getMessage() + ", server " + server); } } diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseInputStreamTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseInputStreamTest.java new file mode 100644 index 000000000..5fc50ae32 --- /dev/null +++ b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseInputStreamTest.java @@ -0,0 +1,183 @@ +package com.clickhouse.client; + +import java.io.BufferedInputStream; +import java.io.ByteArrayInputStream; +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Random; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; + +import org.testng.Assert; +import org.testng.annotations.Test; + +public class ClickHouseInputStreamTest { + private InputStream generateInputStream(byte[] bytes) { + if (bytes.length > 0) { + new Random().nextBytes(bytes); + } + return new BufferedInputStream(new ByteArrayInputStream(bytes)); + } + + @Test(groups = { "unit" }) + public void testNullEmptyOrClosedInput() throws IOException { + Assert.assertThrows(IllegalArgumentException.class, () -> ClickHouseInputStream.of(null)); + ClickHouseInputStream empty = ClickHouseInputStream + .of(generateInputStream(new byte[0])); + Assert.assertEquals(empty.isClosed(), false); + Assert.assertEquals(empty.available(), 0); + Assert.assertEquals(empty.read(), -1); + Assert.assertEquals(empty.read(), -1); + Assert.assertEquals(empty.read(new byte[1]), -1); + Assert.assertEquals(empty.read(new byte[1]), -1); + Assert.assertEquals(empty.readBytes(0), new byte[0]); + Assert.assertThrows(EOFException.class, () -> empty.readByte()); + Assert.assertEquals(empty.isClosed(), true); + Assert.assertThrows(IOException.class, () -> empty.read()); + + ClickHouseInputStream empty1 = ClickHouseInputStream + .of(generateInputStream(new byte[0])); + Assert.assertEquals(empty1.isClosed(), false); + Assert.assertThrows(EOFException.class, () -> empty1.readBytes(1)); + Assert.assertEquals(empty1.isClosed(), true); + Assert.assertThrows(IOException.class, () -> empty1.read()); + + InputStream in = generateInputStream(new byte[] { (byte) 123 }); + in.close(); + ClickHouseInputStream chIn = ClickHouseInputStream.of(in); + Assert.assertEquals(chIn.isClosed(), false); + Assert.assertThrows(IOException.class, () -> chIn.available()); + Assert.assertEquals(chIn.isClosed(), false); + Assert.assertEquals(ClickHouseInputStream.of(chIn), chIn); + Assert.assertEquals(chIn.readBytes(0), new byte[0]); + Assert.assertThrows(IOException.class, () -> chIn.readBytes(1)); + Assert.assertThrows(IOException.class, () -> chIn.read()); + Assert.assertThrows(IOException.class, () -> chIn.readByte()); + Assert.assertThrows(IOException.class, () -> chIn.read(new byte[0])); + chIn.close(); + Assert.assertEquals(chIn.isClosed(), true); + } + + @Test(groups = { "unit" }) + public void testWrappedInput() throws IOException { + int sample = 10000; + byte[] bytes = new byte[sample]; + try (InputStream in = generateInputStream(bytes); ClickHouseInputStream chIn = ClickHouseInputStream.of(in)) { + for (int i = 0; i < sample; i++) { + Assert.assertTrue(chIn.available() > 0); + Assert.assertEquals(chIn.readByte(), bytes[i]); + } + + Assert.assertEquals(chIn.available(), 0); + Assert.assertFalse(chIn.isClosed(), "Should not be closed"); + + Assert.assertThrows(EOFException.class, () -> chIn.readByte()); + Assert.assertTrue(chIn.isClosed(), "Should have been closed automatically"); + } + + try (InputStream in = generateInputStream(bytes); ClickHouseInputStream chIn = ClickHouseInputStream.of(in)) { + Assert.assertEquals(chIn.readBytes(sample), bytes); + Assert.assertFalse(chIn.isClosed(), "Should not be closed"); + Assert.assertThrows(EOFException.class, () -> chIn.readBytes(1)); + Assert.assertTrue(chIn.isClosed(), "Should have been closed automatically"); + } + } + + @Test(groups = { "unit" }) + public void testNullOrEmptyBlockingInput() throws IOException { + Assert.assertThrows(IllegalArgumentException.class, () -> ClickHouseInputStream.of(null, 0)); + Assert.assertThrows(IllegalArgumentException.class, + () -> ClickHouseInputStream.of(new ArrayBlockingQueue<>(0), -1)); + + BlockingQueue queue = new ArrayBlockingQueue<>(1); + ClickHouseInputStream empty = ClickHouseInputStream.of(queue, 10); + Assert.assertEquals(empty.isClosed(), false); + Assert.assertThrows(IOException.class, () -> empty.available()); + Assert.assertThrows(IOException.class, () -> empty.read()); + Assert.assertThrows(IOException.class, () -> empty.read(new byte[1])); + Assert.assertEquals(empty.readBytes(0), new byte[0]); + Assert.assertThrows(IOException.class, () -> empty.readByte()); + Assert.assertThrows(IOException.class, () -> empty.readBytes(1)); + Assert.assertEquals(empty.isClosed(), false); + + queue.offer(ClickHouseInputStream.EMPTY_BUFFER); + Assert.assertEquals(empty.available(), 0); + Assert.assertEquals(empty.read(), -1); + Assert.assertEquals(empty.read(), -1); + Assert.assertEquals(empty.read(new byte[1]), -1); + Assert.assertEquals(empty.read(new byte[2]), -1); + Assert.assertThrows(EOFException.class, () -> empty.readByte()); + Assert.assertEquals(empty.isClosed(), true); + Assert.assertThrows(IOException.class, () -> empty.read()); + } + + @Test(groups = { "unit" }) + public void testBlockingInput() throws IOException { + BlockingQueue queue = new LinkedBlockingQueue<>(); + Random r = new Random(); + byte[] values = new byte[1234567]; + r.nextBytes(values); + for (int i = 0; i < values.length; i++) { + int len = values.length - i - 1; + if (len > 1024) { + len = r.nextInt(1024); + } + byte[] bytes = new byte[len + 1]; + System.arraycopy(values, i, bytes, 0, bytes.length); + queue.offer(ByteBuffer.wrap(bytes)); + i += bytes.length - 1; + } + queue.offer(ClickHouseInputStream.EMPTY_BUFFER); + + ClickHouseInputStream in = ClickHouseInputStream.of(queue, 100); + for (int i = 0; i < values.length; i++) { + int length = Math.min(2048, values.length - i - 1) + 1; + Assert.assertTrue(in.available() > 0, "Should have at least " + length + " byte(s) to read"); + Assert.assertEquals(in.readBytes(length), Arrays.copyOfRange(values, i, i + length)); + i += length - 1; + } + Assert.assertFalse(in.isClosed(), "Should not be closed"); + Assert.assertTrue(in.available() == 0, "Should have all bytes read"); + in.close(); + Assert.assertTrue(in.available() == 0, "Should have all bytes read"); + Assert.assertTrue(in.isClosed(), "Should have been closed"); + } + + @Test(groups = { "unit" }) + public void testBlockingInputAsync() throws IOException { + BlockingQueue queue = new LinkedBlockingQueue<>(); + Random r = new Random(); + byte[] values = new byte[1234567]; + r.nextBytes(values); + + new Thread(() -> { + for (int i = 0; i < values.length; i++) { + int len = values.length - i - 1; + if (len > 1024) { + len = r.nextInt(1024); + } + byte[] bytes = new byte[len + 1]; + System.arraycopy(values, i, bytes, 0, bytes.length); + queue.offer(ByteBuffer.wrap(bytes)); + i += bytes.length - 1; + } + queue.offer(ClickHouseInputStream.EMPTY_BUFFER); + }).start(); + ClickHouseInputStream in = ClickHouseInputStream.of(queue, 0); + for (int i = 0; i < values.length; i++) { + int length = Math.min(2048, values.length - i - 1) + 1; + Assert.assertTrue(in.available() > 0, "Should have at least " + length + " byte(s) to read"); + Assert.assertEquals(in.readBytes(length), Arrays.copyOfRange(values, i, i + length)); + i += length - 1; + } + Assert.assertFalse(in.isClosed(), "Should not be closed"); + Assert.assertTrue(in.available() == 0, "Should have all bytes read"); + in.close(); + Assert.assertTrue(in.available() == 0, "Should have all bytes read"); + Assert.assertTrue(in.isClosed(), "Should have been closed"); + } +} diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/data/ClickHousePipedStreamTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/data/ClickHousePipedStreamTest.java index 9b2328c0b..66cc6bf73 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/data/ClickHousePipedStreamTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/data/ClickHousePipedStreamTest.java @@ -50,7 +50,7 @@ public void testRead() throws Exception { } stream.queue.clear(); - stream.queue.put(ClickHouseInputStream.EMPTY); + stream.queue.put(ClickHouseInputStream.EMPTY_BUFFER); Assert.assertEquals(stream.queue.size(), 1); try (InputStream in = stream.getInput()) { Assert.assertEquals(in.read(), -1); @@ -58,7 +58,7 @@ public void testRead() throws Exception { stream.queue.put((ByteBuffer) ((Buffer) buf).rewind()); // stream.queue.put(buf); - stream.queue.put(ClickHouseInputStream.EMPTY); + stream.queue.put(ClickHouseInputStream.EMPTY_BUFFER); Assert.assertEquals(stream.queue.size(), 2); try (InputStream in = stream.getInput()) { Assert.assertEquals(in.read(), 3); @@ -122,7 +122,7 @@ public void testReadBytes() throws Exception { buf = ByteBuffer.allocate(2).put(new byte[] { (byte) 3, (byte) 4 }); stream.queue.put((ByteBuffer) ((Buffer) buf).rewind()); - stream.queue.put(ClickHouseInputStream.EMPTY); + stream.queue.put(ClickHouseInputStream.EMPTY_BUFFER); Assert.assertEquals(stream.queue.size(), 2); try (InputStream in = stream.getInput()) { Assert.assertEquals(in.read(bytes, 0, 3), 2); diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/data/ClickhouseLZ4InputStreamTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/data/ClickhouseLZ4InputStreamTest.java index 7b934e101..b136880da 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/data/ClickhouseLZ4InputStreamTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/data/ClickhouseLZ4InputStreamTest.java @@ -30,17 +30,17 @@ private InputStream generateInputStream(String prefix, int samples, StringBuilde return new ByteArrayInputStream(result); } - @DataProvider(name = "prefixes") - private Object[][] getPrefixes() { - return new Object[][] { { "test" }, { "萌萌哒" }, - { "1😂2萌🥘" } }; + @DataProvider(name = "samples") + private Object[][] getSamples() { + return new Object[][] { { "", 0 }, { "test", 100000 }, { "萌萌哒", 1024 * 1024 }, + { "1😂2萌🥘", 2500000 } }; }; - @Test(dataProvider = "prefixes", groups = { "unit" }) - public void testReadByte(String prefix) throws IOException { + @Test(dataProvider = "samples", groups = { "unit" }) + public void testReadByte(String prefix, int samples) throws IOException { StringBuilder builder = new StringBuilder(); boolean readAll = false; - try (InputStream in = generateInputStream(prefix, 10000, builder); + try (InputStream in = generateInputStream(prefix, samples, builder); ClickHouseLZ4InputStream lz4In = new ClickHouseLZ4InputStream(in); ByteArrayOutputStream out = new ByteArrayOutputStream();) { try { @@ -58,11 +58,34 @@ public void testReadByte(String prefix) throws IOException { Assert.assertTrue(readAll, "All bytes should have read without any issue"); } - @Test(dataProvider = "prefixes", groups = { "unit" }) - public void testRead(String prefix) throws IOException { + @Test(dataProvider = "samples", groups = { "unit" }) + public void testReadByteWithAvailable(String prefix, int samples) throws IOException { StringBuilder builder = new StringBuilder(); boolean readAll = false; - try (InputStream in = generateInputStream(prefix, 10000, builder); + try (InputStream in = generateInputStream(prefix, samples, builder); + ClickHouseLZ4InputStream lz4In = new ClickHouseLZ4InputStream(in); + ByteArrayOutputStream out = new ByteArrayOutputStream();) { + while (true) { + if (lz4In.available() == 0) { + readAll = true; + break; + } + + out.write(0xFF & lz4In.readByte()); + } + + out.flush(); + + Assert.assertEquals(new String(out.toByteArray(), StandardCharsets.UTF_8), builder.toString()); + } + Assert.assertTrue(readAll, "All bytes should have read without any issue"); + } + + @Test(dataProvider = "samples", groups = { "unit" }) + public void testRead(String prefix, int samples) throws IOException { + StringBuilder builder = new StringBuilder(); + boolean readAll = false; + try (InputStream in = generateInputStream(prefix, samples, builder); ClickHouseLZ4InputStream lz4In = new ClickHouseLZ4InputStream(in); ByteArrayOutputStream out = new ByteArrayOutputStream();) { int result = 0; @@ -77,13 +100,31 @@ public void testRead(String prefix) throws IOException { Assert.assertTrue(readAll, "All bytes should have read without any issue"); } - @Test(dataProvider = "prefixes", groups = { "unit" }) - public void testReadBytes(String prefix) throws IOException { + @Test(dataProvider = "samples", groups = { "unit" }) + public void testReadWithAvailable(String prefix, int samples) throws IOException { + StringBuilder builder = new StringBuilder(); + boolean readAll = false; + try (InputStream in = generateInputStream(prefix, samples, builder); + ClickHouseLZ4InputStream lz4In = new ClickHouseLZ4InputStream(in); + ByteArrayOutputStream out = new ByteArrayOutputStream();) { + while (lz4In.available() > 0) { + out.write(lz4In.read()); + } + out.flush(); + readAll = true; + + Assert.assertEquals(new String(out.toByteArray(), StandardCharsets.UTF_8), builder.toString()); + } + Assert.assertTrue(readAll, "All bytes should have read without any issue"); + } + + @Test(dataProvider = "samples", groups = { "unit" }) + public void testReadBytes(String prefix, int samples) throws IOException { StringBuilder builder = new StringBuilder(); boolean readAll = false; - for (int i = 1; i < 1025; i++) { + for (int i : new int[] { 1, 2, 3, 11, 1025 }) { byte[] bytes = new byte[i]; - try (InputStream in = generateInputStream(prefix, 10000, builder); + try (InputStream in = generateInputStream(prefix, samples, builder); ClickHouseLZ4InputStream lz4In = new ClickHouseLZ4InputStream(in); ByteArrayOutputStream out = new ByteArrayOutputStream();) { int result = 0; @@ -99,6 +140,28 @@ public void testReadBytes(String prefix) throws IOException { } } + @Test(dataProvider = "samples", groups = { "unit" }) + public void testReadBytesWithAvailable(String prefix, int samples) throws IOException { + StringBuilder builder = new StringBuilder(); + boolean readAll = false; + for (int i : new int[] { 1, 2, 3, 11, 1025 }) { + byte[] bytes = new byte[i]; + try (InputStream in = generateInputStream(prefix, samples, builder); + ClickHouseLZ4InputStream lz4In = new ClickHouseLZ4InputStream(in); + ByteArrayOutputStream out = new ByteArrayOutputStream();) { + while (lz4In.available() > 0) { + int result = lz4In.read(bytes); + out.write(bytes, 0, result); + } + out.flush(); + readAll = true; + + Assert.assertEquals(new String(out.toByteArray(), StandardCharsets.UTF_8), builder.toString()); + } + Assert.assertTrue(readAll, "All bytes should have read without any issue"); + } + } + @Test(groups = { "unit" }) public void testLZ4Stream() throws IOException { StringBuilder sb = new StringBuilder(); 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 2067af894..64e4495fa 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,7 +1,6 @@ package com.clickhouse.client.http; import java.io.IOException; -import java.net.URLEncoder; import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; @@ -39,7 +38,7 @@ protected ClickHouseHttpConnection newConnection(ClickHouseHttpConnection connec } try { - return new DefaultHttpConnection(server, request); + return new DefaultHttpConnection(server, request, getExecutor()); } catch (IOException e) { throw new CompletionException(e); } diff --git a/clickhouse-http-client/src/main/java/com/clickhouse/client/http/DefaultHttpConnection.java b/clickhouse-http-client/src/main/java/com/clickhouse/client/http/DefaultHttpConnection.java index afa4369d4..3564f3817 100644 --- a/clickhouse-http-client/src/main/java/com/clickhouse/client/http/DefaultHttpConnection.java +++ b/clickhouse-http-client/src/main/java/com/clickhouse/client/http/DefaultHttpConnection.java @@ -28,6 +28,7 @@ import java.util.TimeZone; import java.util.UUID; import java.util.Map.Entry; +import java.util.concurrent.ExecutorService; import javax.net.ssl.HostnameVerifier; import javax.net.ssl.HttpsURLConnection; @@ -130,7 +131,8 @@ private void checkResponse(HttpURLConnection conn) throws IOException { } } - protected DefaultHttpConnection(ClickHouseNode server, ClickHouseRequest request) throws IOException { + protected DefaultHttpConnection(ClickHouseNode server, ClickHouseRequest request, ExecutorService executor) + throws IOException { super(server, request); conn = newConnection(url, true); diff --git a/clickhouse-http-client/src/main/java11/com/clickhouse/client/http/ClickHouseResponseHandler.java b/clickhouse-http-client/src/main/java11/com/clickhouse/client/http/ClickHouseResponseHandler.java index a04b3539a..6c46b4b23 100644 --- a/clickhouse-http-client/src/main/java11/com/clickhouse/client/http/ClickHouseResponseHandler.java +++ b/clickhouse-http-client/src/main/java11/com/clickhouse/client/http/ClickHouseResponseHandler.java @@ -8,7 +8,8 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; -import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.Flow.Subscription; import java.util.concurrent.atomic.AtomicBoolean; @@ -16,14 +17,14 @@ public class ClickHouseResponseHandler implements BodySubscriber { // An immutable ByteBuffer sentinel to mark that the last byte was received. - private static final List LAST_LIST = List.of(ClickHouseInputStream.EMPTY); + private static final List LAST_LIST = List.of(ClickHouseInputStream.EMPTY_BUFFER); private final BlockingQueue buffers; private final ClickHouseInputStream in; private final AtomicBoolean subscribed; - ClickHouseResponseHandler(int bufferSize, int timeout) { - buffers = new LinkedBlockingDeque<>(); + ClickHouseResponseHandler(int queueLength, int timeout) { + buffers = queueLength > 1 ? new ArrayBlockingQueue<>(queueLength) : new LinkedBlockingQueue<>(); in = ClickHouseInputStream.of(buffers, timeout); subscribed = new AtomicBoolean(); } @@ -71,7 +72,7 @@ public void onNext(List item) { @Override public void onError(Throwable throwable) { - buffers.offer(ClickHouseInputStream.EMPTY); + buffers.offer(ClickHouseInputStream.EMPTY_BUFFER); } @Override diff --git a/clickhouse-http-client/src/main/java11/com/clickhouse/client/http/DefaultHttpConnection.java b/clickhouse-http-client/src/main/java11/com/clickhouse/client/http/DefaultHttpConnection.java index 099593c5f..a8f612a54 100644 --- a/clickhouse-http-client/src/main/java11/com/clickhouse/client/http/DefaultHttpConnection.java +++ b/clickhouse-http-client/src/main/java11/com/clickhouse/client/http/DefaultHttpConnection.java @@ -36,13 +36,18 @@ import java.util.Map.Entry; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.function.Function; import javax.net.ssl.SSLContext; public class DefaultHttpConnection extends ClickHouseHttpConnection { private static final Logger log = LoggerFactory.getLogger(DefaultHttpConnection.class); + private static final int MAX_RETRIES = 1; + private final HttpClient httpClient; + private final HttpRequest pingRequest; private ClickHouseHttpResponse buildResponse(HttpResponse r) throws IOException { HttpHeaders headers = r.headers(); @@ -93,15 +98,16 @@ private HttpRequest newRequest(String url) { .timeout(Duration.ofMillis(config.getSocketTimeout())).build(); } - protected DefaultHttpConnection(ClickHouseNode server, ClickHouseRequest request) throws IOException { + protected DefaultHttpConnection(ClickHouseNode server, ClickHouseRequest request, ExecutorService executor) + throws IOException { super(server, request); HttpClient.Builder builder = HttpClient.newBuilder() + .version(Version.HTTP_1_1) .connectTimeout(Duration.ofMillis(config.getConnectionTimeout())) - .followRedirects(Redirect.ALWAYS) - .version(Version.HTTP_1_1); - if (config.isAsync()) { - builder.executor(ClickHouseClient.getExecutorService()); + .followRedirects(Redirect.NORMAL); + if (executor != null) { + builder.executor(executor); } if (config.isSsl()) { builder.sslContext(ClickHouseSslContextProvider.getProvider().getSslContext(SSLContext.class, config) @@ -109,6 +115,7 @@ protected DefaultHttpConnection(ClickHouseNode server, ClickHouseRequest requ } httpClient = builder.build(); + pingRequest = newRequest(getBaseUrl() + "ping"); } @Override @@ -116,15 +123,51 @@ protected boolean isReusable() { return true; } + private CompletableFuture> retry(Throwable firstError, int retry) { + if (retry >= MAX_RETRIES) { + final CompletableFuture> failure = new CompletableFuture<>(); + failure.completeExceptionally(firstError); + return failure; + } + + return httpClient.sendAsync(pingRequest, HttpResponse.BodyHandlers.discarding()) + .thenApply(CompletableFuture::completedFuture) + .exceptionally(t -> { + firstError.addSuppressed(t); + return retry(firstError, retry + 1); + }) + .thenCompose(Function.identity()); + } + + private CompletableFuture> postRequest(HttpRequest request) { + CompletableFuture> f; + // either change system property jdk.httpclient.keepalive.timeout or increase + // keep_alive_timeout on server + boolean retry = false; // config.isRetry() + if (retry) { + f = httpClient + .sendAsync(pingRequest, HttpResponse.BodyHandlers.discarding()) + .thenApply(CompletableFuture::completedFuture) + .exceptionally(t -> retry(t, 0)) + .thenCompose(t -> httpClient.sendAsync(request, + responseInfo -> new ClickHouseResponseHandler(config.getMaxQueuedBuffers(), + config.getSocketTimeout()))); + } else { + f = httpClient.sendAsync(request, + responseInfo -> new ClickHouseResponseHandler(config.getMaxQueuedBuffers(), + config.getSocketTimeout())); + } + return f; + } + private ClickHouseHttpResponse postStream(HttpRequest.Builder reqBuilder, String boundary, String sql, InputStream data, List tables) throws IOException { ClickHousePipedStream stream = new ClickHousePipedStream(config.getMaxBufferSize(), config.getMaxQueuedBuffers(), config.getSocketTimeout()); reqBuilder.POST(HttpRequest.BodyPublishers.ofInputStream(stream::getInput)); + // running in async is necessary to avoid deadlock of the piped stream - CompletableFuture> f = httpClient.sendAsync(reqBuilder.build(), - responseInfo -> new ClickHouseResponseHandler(config.getMaxBufferSize(), - config.getSocketTimeout())); + CompletableFuture> f = postRequest(reqBuilder.build()); try (BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(stream, StandardCharsets.UTF_8))) { if (boundary != null) { String line = "\r\n--" + boundary + "\r\n"; @@ -183,10 +226,7 @@ private ClickHouseHttpResponse postString(HttpRequest.Builder reqBuilder, String reqBuilder.POST(HttpRequest.BodyPublishers.ofString(sql)); HttpResponse r; try { - CompletableFuture> f = httpClient.sendAsync(reqBuilder.build(), - responseInfo -> new ClickHouseResponseHandler(config.getMaxBufferSize(), - config.getSocketTimeout())); - r = f.get(); + r = postRequest(reqBuilder.build()).get(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new IOException("Thread was interrupted when posting request or receiving response", e); @@ -225,13 +265,8 @@ protected ClickHouseHttpResponse post(String sql, InputStream data, List r = httpClient.send(newRequest(getBaseUrl() + "ping"), - HttpResponse.BodyHandlers.ofString()); - if (r.statusCode() != HttpURLConnection.HTTP_OK) { - throw new IOException(r.body()); - } - - return response.equals(r.body()); + HttpResponse r = httpClient.send(pingRequest, HttpResponse.BodyHandlers.ofString()); + return r.statusCode() == HttpURLConnection.HTTP_OK && response.equals(r.body()); } catch (InterruptedException e) { Thread.currentThread().interrupt(); } catch (IOException e) { diff --git a/clickhouse-http-client/src/test/java/com/clickhouse/client/http/DefaultHttpConnectionTest.java b/clickhouse-http-client/src/test/java/com/clickhouse/client/http/DefaultHttpConnectionTest.java index 861bf745a..a9153eccf 100644 --- a/clickhouse-http-client/src/test/java/com/clickhouse/client/http/DefaultHttpConnectionTest.java +++ b/clickhouse-http-client/src/test/java/com/clickhouse/client/http/DefaultHttpConnectionTest.java @@ -22,7 +22,7 @@ public void testConnectionReuse() throws Exception { try (ClickHouseClient client = ClickHouseClient.newInstance()) { ClickHouseRequest req = client.connect(server); - DefaultHttpConnection conn = new DefaultHttpConnection(server, req); + DefaultHttpConnection conn = new DefaultHttpConnection(server, req, null); } } } diff --git a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseStatementTest.java b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseStatementTest.java index 911b28e1f..8f2234be9 100644 --- a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseStatementTest.java +++ b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseStatementTest.java @@ -94,6 +94,26 @@ public void testQuery() throws SQLException { } } + @Test(groups = "integration") + public void testMultiStatementQuery() throws SQLException { + try (ClickHouseConnection conn = newConnection(new Properties())) { + ClickHouseStatement stmt = conn.createStatement(); + ResultSet rs = stmt.executeQuery("set join_use_nulls=1;\n" + + "select a.k, b.m from ( " + + " select 1 k, null v union all select 2 k, 'a' v " + + ") a left outer join ( select 1 f, 2 m ) b on a.k = b.f " + + "order by a.k"); + Assert.assertTrue(rs.next()); + Assert.assertEquals(rs.getInt(1), 1); + Assert.assertEquals(rs.getInt(2), 2); + Assert.assertTrue(rs.next()); + Assert.assertEquals(rs.getInt(1), 2); + Assert.assertEquals(rs.getInt(2), 0); + Assert.assertEquals(rs.getObject(2), null); + Assert.assertFalse(rs.next()); + } + } + @Test(groups = "integration") public void testTimestamp() throws SQLException { try (ClickHouseConnection conn = newConnection(new Properties()); From d9e9df8c92bd9404d4fb9a87f3aea627c6543b24 Mon Sep 17 00:00:00 2001 From: Zhichun Wu Date: Tue, 14 Dec 2021 17:28:30 +0800 Subject: [PATCH 2/6] Bump slf4j and remove log4j dependency --- .github/workflows/analysis.yml | 2 +- .github/workflows/benchmark.yml | 4 ++-- .github/workflows/release.yml | 2 +- .github/workflows/third_party_libs.yml | 2 +- clickhouse-client/pom.xml | 2 +- clickhouse-client/src/test/resources/log4j.properties | 5 ----- .../src/test/resources/simplelogger.properties | 7 +++++++ clickhouse-grpc-client/pom.xml | 2 +- clickhouse-grpc-client/src/test/resources/log4j.properties | 5 ----- .../src/test/resources/simplelogger.properties | 7 +++++++ clickhouse-http-client/pom.xml | 2 +- clickhouse-http-client/src/test/resources/log4j.properties | 5 ----- .../src/test/resources/simplelogger.properties | 7 +++++++ clickhouse-jdbc/legacy.xml | 2 +- clickhouse-jdbc/pom.xml | 2 +- clickhouse-jdbc/src/test/resources/log4j.properties | 6 ------ clickhouse-jdbc/src/test/resources/simplelogger.properties | 7 +++++++ clickhouse-tcp-client/pom.xml | 2 +- pom.xml | 4 ++-- 19 files changed, 41 insertions(+), 34 deletions(-) delete mode 100644 clickhouse-client/src/test/resources/log4j.properties create mode 100644 clickhouse-client/src/test/resources/simplelogger.properties delete mode 100644 clickhouse-grpc-client/src/test/resources/log4j.properties create mode 100644 clickhouse-grpc-client/src/test/resources/simplelogger.properties delete mode 100644 clickhouse-http-client/src/test/resources/log4j.properties create mode 100644 clickhouse-http-client/src/test/resources/simplelogger.properties delete mode 100644 clickhouse-jdbc/src/test/resources/log4j.properties create mode 100644 clickhouse-jdbc/src/test/resources/simplelogger.properties diff --git a/.github/workflows/analysis.yml b/.github/workflows/analysis.yml index bdbdeec0f..3964ed84a 100644 --- a/.github/workflows/analysis.yml +++ b/.github/workflows/analysis.yml @@ -63,7 +63,7 @@ jobs: GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} SONAR_TOKEN: ${{ secrets.SONAR_TOKEN }} run: | - find . -type f -name "log4j.*" -exec rm -fv '{}' \; + find . -type f -name "simplelogger.*" -exec rm -fv '{}' \; mvn -q --batch-mode --global-toolchains .github/toolchains.xml -Drelease \ -Panalysis verify org.sonarsource.scanner.maven:sonar-maven-plugin:sonar continue-on-error: true diff --git a/.github/workflows/benchmark.yml b/.github/workflows/benchmark.yml index 7e5fcdf6a..5f7addcf9 100644 --- a/.github/workflows/benchmark.yml +++ b/.github/workflows/benchmark.yml @@ -56,7 +56,7 @@ jobs: find . -type f -name "pom.xml" -exec sed -i -e 's|${revision}|${{ steps.version.outputs.value }}|g' \ -e 's|^\( \).*\(\)$|\1${{ steps.version.outputs.value }}\2|' \ -e 's|${parent.groupId}|com.clickhouse|g' '{}' \; - find . -type f -name "log4j.*" -exec rm -fv '{}' \; + find . -type f -name "simplelogger.*" -exec rm -fv '{}' \; - name: Build project run: | mvn --batch-mode --update-snapshots -q -DskipTests install @@ -100,7 +100,7 @@ jobs: find . -type f -name "pom.xml" -exec sed -i -e 's|${revision}|${{ github.event.inputs.driver }}|g' \ -e 's|^\( \).*\(\)$|\1${{ github.event.inputs.driver }}\2|' \ -e 's|${parent.groupId}|com.clickhouse|g' -e 's|${project.parent.groupId}|com.clickhouse|g' '{}' \; - find . -type f -name "log4j.*" -exec rm -fv '{}' \; + find . -type f -name "simplelogger.*" -exec rm -fv '{}' \; continue-on-error: true - name: Install driver as needed run: mvn --batch-mode --update-snapshots -q -DskipTests install diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index 9cd0c2c5f..8b9b7876f 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -49,7 +49,7 @@ jobs: find . -type f -name "pom.xml" -exec sed -i -e 's|${revision}|${{ github.event.inputs.version }}|g' \ -e 's|^\( \).*\(\)$|\1${{ github.event.inputs.version }}\2|' \ -e 's|${parent.groupId}|com.clickhouse|g' -e 's|${project.parent.groupId}|com.clickhouse|g' '{}' \; - find . -type f -name "log4j.*" -exec rm -fv '{}' \; + find . -type f -name "simplelogger.*" -exec rm -fv '{}' \; - name: Release Maven package uses: samuelmeuli/action-maven-publish@v1 with: diff --git a/.github/workflows/third_party_libs.yml b/.github/workflows/third_party_libs.yml index 6e0f53f04..80cfebf49 100644 --- a/.github/workflows/third_party_libs.yml +++ b/.github/workflows/third_party_libs.yml @@ -25,7 +25,7 @@ jobs: find . -type f -name "pom.xml" -exec sed -i -e 's|${revision}|${{ github.event.inputs.version }}|g' \ -e 's|^\( \).*\(\)$|\1${{ github.event.inputs.version }}\2|' \ -e 's|${parent.groupId}|com.clickhouse|g' -e 's|${project.parent.groupId}|com.clickhouse|g' '{}' \; - find . -type f -name "log4j.*" -exec rm -fv '{}' \; + find . -type f -name "simplelogger.*" -exec rm -fv '{}' \; - name: Release Maven package uses: samuelmeuli/action-maven-publish@v1 with: diff --git a/clickhouse-client/pom.xml b/clickhouse-client/pom.xml index df7cf24d5..05e72f6c4 100644 --- a/clickhouse-client/pom.xml +++ b/clickhouse-client/pom.xml @@ -56,7 +56,7 @@ org.slf4j - slf4j-log4j12 + slf4j-simple test diff --git a/clickhouse-client/src/test/resources/log4j.properties b/clickhouse-client/src/test/resources/log4j.properties deleted file mode 100644 index 2596a2ac7..000000000 --- a/clickhouse-client/src/test/resources/log4j.properties +++ /dev/null @@ -1,5 +0,0 @@ -plog4j.rootLogger=WARN, STDOUT -log4j.category.com.clickhouse.client=DEBUG -log4j.appender.STDOUT=org.apache.log4j.ConsoleAppender -log4j.appender.STDOUT.layout=org.apache.log4j.PatternLayout -log4j.appender.STDOUT.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss.sss} [%t] [%-5p] {%c{1}:%L} - %m%n diff --git a/clickhouse-client/src/test/resources/simplelogger.properties b/clickhouse-client/src/test/resources/simplelogger.properties new file mode 100644 index 000000000..b6db3b98b --- /dev/null +++ b/clickhouse-client/src/test/resources/simplelogger.properties @@ -0,0 +1,7 @@ +org.slf4j.simpleLogger.defaultLogLevel=info +org.slf4j.simpleLogger.log.com.clickhouse.client=info +org.slf4j.simpleLogger.showDateTime=true +org.slf4j.simpleLogger.dateTimeFormat=yyyy-MM-dd HH:mm:ss:SSS Z +org.slf4j.simpleLogger.showThreadName=true +org.slf4j.simpleLogger.showLogName=true +org.slf4j.simpleLogger.showShortLogName=true diff --git a/clickhouse-grpc-client/pom.xml b/clickhouse-grpc-client/pom.xml index bcc28f1e3..9f480de05 100644 --- a/clickhouse-grpc-client/pom.xml +++ b/clickhouse-grpc-client/pom.xml @@ -61,7 +61,7 @@ org.slf4j - slf4j-log4j12 + slf4j-simple test diff --git a/clickhouse-grpc-client/src/test/resources/log4j.properties b/clickhouse-grpc-client/src/test/resources/log4j.properties deleted file mode 100644 index 204c71e96..000000000 --- a/clickhouse-grpc-client/src/test/resources/log4j.properties +++ /dev/null @@ -1,5 +0,0 @@ -log4j.rootLogger=INFO, STDOUT -log4j.category.com.clickhouse.client=DEBUG -log4j.appender.STDOUT=org.apache.log4j.ConsoleAppender -log4j.appender.STDOUT.layout=org.apache.log4j.PatternLayout -log4j.appender.STDOUT.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss.sss} [%t] [%-5p] {%c{1}:%L} - %m%n diff --git a/clickhouse-grpc-client/src/test/resources/simplelogger.properties b/clickhouse-grpc-client/src/test/resources/simplelogger.properties new file mode 100644 index 000000000..7589a0e57 --- /dev/null +++ b/clickhouse-grpc-client/src/test/resources/simplelogger.properties @@ -0,0 +1,7 @@ +org.slf4j.simpleLogger.defaultLogLevel=info +org.slf4j.simpleLogger.log.com.clickhouse.client=debug +org.slf4j.simpleLogger.showDateTime=true +org.slf4j.simpleLogger.dateTimeFormat=yyyy-MM-dd HH:mm:ss:SSS Z +org.slf4j.simpleLogger.showThreadName=true +org.slf4j.simpleLogger.showLogName=true +org.slf4j.simpleLogger.showShortLogName=true diff --git a/clickhouse-http-client/pom.xml b/clickhouse-http-client/pom.xml index 36e0dbea7..2236a777e 100644 --- a/clickhouse-http-client/pom.xml +++ b/clickhouse-http-client/pom.xml @@ -61,7 +61,7 @@ org.slf4j - slf4j-log4j12 + slf4j-simple test diff --git a/clickhouse-http-client/src/test/resources/log4j.properties b/clickhouse-http-client/src/test/resources/log4j.properties deleted file mode 100644 index 204c71e96..000000000 --- a/clickhouse-http-client/src/test/resources/log4j.properties +++ /dev/null @@ -1,5 +0,0 @@ -log4j.rootLogger=INFO, STDOUT -log4j.category.com.clickhouse.client=DEBUG -log4j.appender.STDOUT=org.apache.log4j.ConsoleAppender -log4j.appender.STDOUT.layout=org.apache.log4j.PatternLayout -log4j.appender.STDOUT.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss.sss} [%t] [%-5p] {%c{1}:%L} - %m%n diff --git a/clickhouse-http-client/src/test/resources/simplelogger.properties b/clickhouse-http-client/src/test/resources/simplelogger.properties new file mode 100644 index 000000000..7589a0e57 --- /dev/null +++ b/clickhouse-http-client/src/test/resources/simplelogger.properties @@ -0,0 +1,7 @@ +org.slf4j.simpleLogger.defaultLogLevel=info +org.slf4j.simpleLogger.log.com.clickhouse.client=debug +org.slf4j.simpleLogger.showDateTime=true +org.slf4j.simpleLogger.dateTimeFormat=yyyy-MM-dd HH:mm:ss:SSS Z +org.slf4j.simpleLogger.showThreadName=true +org.slf4j.simpleLogger.showLogName=true +org.slf4j.simpleLogger.showShortLogName=true diff --git a/clickhouse-jdbc/legacy.xml b/clickhouse-jdbc/legacy.xml index 5442b8402..b8a8a5415 100644 --- a/clickhouse-jdbc/legacy.xml +++ b/clickhouse-jdbc/legacy.xml @@ -98,7 +98,7 @@ org.slf4j - slf4j-log4j12 + slf4j-simple test diff --git a/clickhouse-jdbc/pom.xml b/clickhouse-jdbc/pom.xml index 43c272497..a27f8556c 100644 --- a/clickhouse-jdbc/pom.xml +++ b/clickhouse-jdbc/pom.xml @@ -94,7 +94,7 @@ org.slf4j - slf4j-log4j12 + slf4j-simple test diff --git a/clickhouse-jdbc/src/test/resources/log4j.properties b/clickhouse-jdbc/src/test/resources/log4j.properties deleted file mode 100644 index c9e52c9ee..000000000 --- a/clickhouse-jdbc/src/test/resources/log4j.properties +++ /dev/null @@ -1,6 +0,0 @@ -log4j.rootLogger=WARN, STDOUT -log4j.category.com.clickhouse=DEBUG -#log4j.logger.org.apache.http=DEBUG -log4j.appender.STDOUT=org.apache.log4j.ConsoleAppender -log4j.appender.STDOUT.layout=org.apache.log4j.PatternLayout -log4j.appender.STDOUT.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss.sss} [%t] [%-5p] {%c{1}:%L} - %m%n diff --git a/clickhouse-jdbc/src/test/resources/simplelogger.properties b/clickhouse-jdbc/src/test/resources/simplelogger.properties new file mode 100644 index 000000000..7589a0e57 --- /dev/null +++ b/clickhouse-jdbc/src/test/resources/simplelogger.properties @@ -0,0 +1,7 @@ +org.slf4j.simpleLogger.defaultLogLevel=info +org.slf4j.simpleLogger.log.com.clickhouse.client=debug +org.slf4j.simpleLogger.showDateTime=true +org.slf4j.simpleLogger.dateTimeFormat=yyyy-MM-dd HH:mm:ss:SSS Z +org.slf4j.simpleLogger.showThreadName=true +org.slf4j.simpleLogger.showLogName=true +org.slf4j.simpleLogger.showShortLogName=true diff --git a/clickhouse-tcp-client/pom.xml b/clickhouse-tcp-client/pom.xml index 8d3cb1093..217ba1417 100644 --- a/clickhouse-tcp-client/pom.xml +++ b/clickhouse-tcp-client/pom.xml @@ -36,7 +36,7 @@ org.slf4j - slf4j-log4j12 + slf4j-simple test diff --git a/pom.xml b/pom.xml index d60039fe9..ef115f68b 100644 --- a/pom.xml +++ b/pom.xml @@ -89,7 +89,7 @@ 3.17.3 1.8.0 0.9.21 - 1.8.0-beta4 + 2.0.0-alpha5 3.12.4 2.31.0 1.16.0 @@ -236,7 +236,7 @@ org.slf4j - slf4j-log4j12 + slf4j-simple ${slf4j.version} From 1f6666a30f89407dafc200d7501ae1b7acf66fa4 Mon Sep 17 00:00:00 2001 From: Zhichun Wu Date: Wed, 15 Dec 2021 18:48:52 +0800 Subject: [PATCH 3/6] Make HttpUrlConnection default implementation --- clickhouse-benchmark/pom.xml | 2 +- .../clickhouse/benchmark/jdbc/JdbcDriver.java | 7 +- .../client/ClickHouseClientBuilder.java | 2 +- .../client/ClickHouseParameterizedQuery.java | 158 ++++++++++-------- .../client/ClickHouseThreadFactory.java | 2 +- .../clickhouse/client/ClickHouseUtils.java | 14 +- .../data/ClickHouseRowBinaryProcessor.java | 6 +- .../ClickHouseParameterizedQueryTest.java | 40 ++++- .../client/http/ClickHouseHttpClient.java | 2 +- .../client/http/ClickHouseHttpConnection.java | 2 +- .../http/ClickHouseHttpConnectionFactory.java | 14 ++ ...ection.java => HttpUrlConnectionImpl.java} | 10 +- .../http/config/ClickHouseHttpOption.java | 5 + .../http/config/HttpConnectionProvider.java | 6 + .../http/ClickHouseHttpConnectionFactory.java | 27 +++ ...ion.java => HttpClientConnectionImpl.java} | 6 +- .../http/DefaultHttpConnectionTest.java | 2 +- .../clickhouse/jdbc/ClickHouseResultSet.java | 7 +- .../jdbc/ClickHouseResultSetMetaData.java | 3 +- .../jdbc/JdbcParameterizedQuery.java | 101 ++++++++++- .../com/clickhouse/jdbc/JdbcTypeMapping.java | 18 ++ .../jdbc/ClickHousePreparedStatementTest.java | 20 ++- 22 files changed, 355 insertions(+), 99 deletions(-) create mode 100644 clickhouse-http-client/src/main/java/com/clickhouse/client/http/ClickHouseHttpConnectionFactory.java rename clickhouse-http-client/src/main/java/com/clickhouse/client/http/{DefaultHttpConnection.java => HttpUrlConnectionImpl.java} (95%) create mode 100644 clickhouse-http-client/src/main/java/com/clickhouse/client/http/config/HttpConnectionProvider.java create mode 100644 clickhouse-http-client/src/main/java11/com/clickhouse/client/http/ClickHouseHttpConnectionFactory.java rename clickhouse-http-client/src/main/java11/com/clickhouse/client/http/{DefaultHttpConnection.java => HttpClientConnectionImpl.java} (97%) diff --git a/clickhouse-benchmark/pom.xml b/clickhouse-benchmark/pom.xml index 18d27c64c..d6300ac0b 100644 --- a/clickhouse-benchmark/pom.xml +++ b/clickhouse-benchmark/pom.xml @@ -17,7 +17,7 @@ 1.4.4 - 2.6.1 + 2.6.3 UTF-8 1.33 benchmarks diff --git a/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/jdbc/JdbcDriver.java b/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/jdbc/JdbcDriver.java index 81a559e25..3376b187e 100644 --- a/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/jdbc/JdbcDriver.java +++ b/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/jdbc/JdbcDriver.java @@ -8,8 +8,11 @@ public enum JdbcDriver { "jdbc:clickhouse://%s:%s/%s?ssl=false&user=%s&password=%s&use_server_time_zone=false&use_time_zone=UTC&compress=%s", Constants.HTTP_PORT), // ClickHouse JDBC Driver - ClickhouseHttpJdbc("com.clickhouse.jdbc.ClickHouseDriver", - "jdbc:ch://%s:%s/%s?ssl=false&user=%s&password=%s&use_server_time_zone=false&use_time_zone=UTC&compress=%s", + ClickhouseHttpJdbc1("com.clickhouse.jdbc.ClickHouseDriver", + "jdbc:ch://%s:%s/%s?http_connection_provider=HTTP_URL_CONNECTION&ssl=false&user=%s&password=%s&use_server_time_zone=false&use_time_zone=UTC&compress=%s", + Constants.HTTP_PORT), + ClickhouseHttpJdbc2("com.clickhouse.jdbc.ClickHouseDriver", + "jdbc:ch://%s:%s/%s?http_connection_provider=HTTP_CLIENT&ssl=false&user=%s&password=%s&use_server_time_zone=false&use_time_zone=UTC&compress=%s", Constants.HTTP_PORT), ClickhouseGrpcJdbc("com.clickhouse.jdbc.ClickHouseDriver", "jdbc:ch:grpc://%s:%s/%s?ssl=false&user=%s&password=%s&use_server_time_zone=false&use_time_zone=UTC&max_inbound_message_size=2147483647&compress=%s", 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 eef6aa462..ad2a6f61d 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseClientBuilder.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseClientBuilder.java @@ -32,7 +32,7 @@ public class ClickHouseClientBuilder { } defaultExecutor = ClickHouseUtils.newThreadPool(ClickHouseClient.class.getSimpleName(), maxThreads, - maxThreads * 2, maxRequests, keepAliveTimeoutMs); + maxThreads * 2, maxRequests, keepAliveTimeoutMs, false); } protected ClickHouseConfig config; diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseParameterizedQuery.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseParameterizedQuery.java index fd29077a0..c690ba322 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseParameterizedQuery.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseParameterizedQuery.java @@ -4,22 +4,24 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.Iterator; import java.util.LinkedHashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Map.Entry; /** * A parameterized query is a parsed query with parameters being extracted for * substitution. *

* Here parameter is define in the format of {@code :[()]}. It - * starts with colon, followed by name, and then optionally type within - * brackets. For example: in query "select :no as no, :name(String) as name", - * both {@code no} and {@code name} are parameters. Moreover, type of the last - * parameter is {@code String}. + * starts with colon, immediately followed by name, and then optionally type + * within brackets. For example: in query "select :no as no, :name(String) as + * name", we have two parameters: {@code no} and {@code name}. Moreover, type of + * the last parameter is {@code String}. */ public class ClickHouseParameterizedQuery implements Serializable { private static final long serialVersionUID = 8108887349618342152L; @@ -271,6 +273,19 @@ protected String parse() { return partIndex < len ? originalQuery.substring(partIndex, len) : null; } + /** + * Appends last part of the query if it exists. + * + * @param builder non-null string builder + * @return the builder + */ + protected StringBuilder appendLastPartIfExists(StringBuilder builder) { + if (lastPart != null) { + builder.append(lastPart); + } + return builder; + } + /** * Converts given raw value to SQL expression. * @@ -305,10 +320,7 @@ public String apply(Map params) { builder.append(params.getOrDefault(p.paramName, ClickHouseValues.NULL_EXPR)); } - if (lastPart != null) { - builder.append(lastPart); - } - return builder.toString(); + return appendLastPartIfExists(builder).toString(); } /** @@ -318,23 +330,25 @@ public String apply(Map params) { * @return substituted query */ public String apply(Collection params) { - if (!hasParameter()) { - return originalQuery; - } - - StringBuilder builder = new StringBuilder(); - Iterator it = params == null ? null : params.iterator(); - boolean hasMore = it != null && it.hasNext(); - for (QueryPart p : parts) { - builder.append(p.part); - builder.append(hasMore ? it.next() : ClickHouseValues.NULL_EXPR); - hasMore = hasMore && it.hasNext(); - } - - if (lastPart != null) { - builder.append(lastPart); + if (params == null || params.isEmpty()) { + return apply(Collections.emptyMap()); + } + + Map map = null; + Iterator it = params.iterator(); + if (it.hasNext()) { + map = new HashMap<>(); + for (String n : names.keySet()) { + String v = it.next(); + if (v != null) { + map.put(n, v); + } + if (!it.hasNext()) { + break; + } + } } - return builder.toString(); + return apply(map); } /** @@ -351,22 +365,24 @@ public String apply(Object param, Object... more) { return originalQuery; } - int len = more == null ? 0 : more.length + 1; - StringBuilder builder = new StringBuilder(); - int index = 0; - for (QueryPart p : parts) { - builder.append(p.part); - if (index > 0) { - param = index < len ? more[index - 1] : null; + int len = more == null ? 0 : more.length; + Map map = new HashMap<>(); + int index = -1; + for (Entry e : names.entrySet()) { + ClickHouseValue v = e.getValue(); + if (index < 0) { + map.put(e.getKey(), + v != null ? v.update(param).toSqlExpression() : ClickHouseValues.convertToSqlExpression(param)); + } else if (index < len) { + map.put(e.getKey(), v != null ? v.update(more[index]).toSqlExpression() + : ClickHouseValues.convertToSqlExpression(more[index])); + } else { + break; } - builder.append(toSqlExpression(p.paramName, param)); index++; } - if (lastPart != null) { - builder.append(lastPart); - } - return builder.toString(); + return apply(map); } /** @@ -378,24 +394,25 @@ public String apply(Object param, Object... more) { * @return substituted query */ public String apply(Object[] values) { - if (!hasParameter()) { - return originalQuery; + int len = values == null ? 0 : values.length; + if (len == 0) { + return apply(Collections.emptyMap()); } - int len = values == null ? 0 : values.length; - StringBuilder builder = new StringBuilder(); + Map map = new HashMap<>(); int index = 0; - for (QueryPart p : parts) { - builder.append(p.part); - builder.append( - index < len ? toSqlExpression(p.paramName, values[index]) : ClickHouseValues.NULL_EXPR); + for (Entry e : names.entrySet()) { + ClickHouseValue v = e.getValue(); + if (index < len) { + map.put(e.getKey(), v != null ? v.update(values[index]).toSqlExpression() + : ClickHouseValues.convertToSqlExpression(values[index])); + } else { + break; + } index++; } - if (lastPart != null) { - builder.append(lastPart); - } - return builder.toString(); + return apply(map); } /** @@ -410,22 +427,21 @@ public String apply(String param, String... more) { return originalQuery; } - int len = more == null ? 0 : more.length + 1; - StringBuilder builder = new StringBuilder(); - int index = 0; - for (QueryPart p : parts) { - builder.append(p.part); - if (index > 0) { - param = index < len ? more[index - 1] : ClickHouseValues.NULL_EXPR; + int len = more == null ? 0 : more.length; + Map map = new HashMap<>(); + int index = -1; + for (String n : names.keySet()) { + if (index < 0) { + map.put(n, param); + } else if (index < len) { + map.put(n, more[index]); + } else { + break; } - builder.append(param); index++; } - if (lastPart != null) { - builder.append(lastPart); - } - return builder.toString(); + return apply(map); } /** @@ -435,23 +451,23 @@ public String apply(String param, String... more) { * @return substituted query */ public String apply(String[] values) { - if (!hasParameter()) { - return originalQuery; + int len = values == null ? 0 : values.length; + if (len == 0) { + return apply(Collections.emptyMap()); } - int len = values == null ? 0 : values.length; - StringBuilder builder = new StringBuilder(); + Map map = new HashMap<>(); int index = 0; - for (QueryPart p : parts) { - builder.append(p.part); - builder.append(index < len ? values[index] : ClickHouseValues.NULL_EXPR); + for (String n : names.keySet()) { + if (index < len) { + map.put(n, values[index]); + } else { + break; + } index++; } - if (lastPart != null) { - builder.append(lastPart); - } - return builder.toString(); + return apply(map); } /** diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseThreadFactory.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseThreadFactory.java index bdb7608ec..a4cd2ffe7 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseThreadFactory.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseThreadFactory.java @@ -12,7 +12,7 @@ public class ClickHouseThreadFactory implements ThreadFactory { private final AtomicInteger threadNumber; public ClickHouseThreadFactory(Object owner) { - this(owner, false, Thread.NORM_PRIORITY); + this(owner, true, Thread.NORM_PRIORITY); } public ClickHouseThreadFactory(Object owner, boolean daemon, int priority) { diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseUtils.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseUtils.java index 44a54f9f1..2748f2ff7 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseUtils.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseUtils.java @@ -107,11 +107,11 @@ private static T findFirstService(Class serviceInterface) { } public static ExecutorService newThreadPool(Object owner, int maxThreads, int maxRequests) { - return newThreadPool(owner, maxThreads, 0, maxRequests, 0L); + return newThreadPool(owner, maxThreads, 0, maxRequests, 0L, true); } public static ExecutorService newThreadPool(Object owner, int coreThreads, int maxThreads, int maxRequests, - long keepAliveTimeoutMs) { + long keepAliveTimeoutMs, boolean allowCoreThreadTimeout) { BlockingQueue queue = maxRequests > 0 ? new ArrayBlockingQueue<>(maxRequests) : new LinkedBlockingQueue<>(); if (coreThreads < 2) { @@ -120,8 +120,16 @@ public static ExecutorService newThreadPool(Object owner, int coreThreads, int m if (maxThreads < coreThreads) { maxThreads = coreThreads; } - return new ThreadPoolExecutor(coreThreads, maxThreads, keepAliveTimeoutMs < 0L ? 0L : keepAliveTimeoutMs, + if (keepAliveTimeoutMs <= 0L) { + keepAliveTimeoutMs = allowCoreThreadTimeout ? 1000L : 0L; + } + + ThreadPoolExecutor pool = new ThreadPoolExecutor(coreThreads, maxThreads, keepAliveTimeoutMs, TimeUnit.MILLISECONDS, queue, new ClickHouseThreadFactory(owner), new ThreadPoolExecutor.AbortPolicy()); + if (allowCoreThreadTimeout) { + pool.allowCoreThreadTimeOut(true); + } + return pool; } public static boolean isCloseBracket(char ch) { diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseRowBinaryProcessor.java b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseRowBinaryProcessor.java index 79c2c8294..31a0aadb4 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseRowBinaryProcessor.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseRowBinaryProcessor.java @@ -505,11 +505,11 @@ public boolean hasNext() { @Override public ClickHouseRecord next() { - if (!hasNext()) { + ClickHouseRecord r = readNextRow(); + if (r == null) { throw new NoSuchElementException("No more record"); } - - return readNextRow(); + return r; } } diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseParameterizedQueryTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseParameterizedQueryTest.java index 3335d5516..2ef7ad7cd 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseParameterizedQueryTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseParameterizedQueryTest.java @@ -44,6 +44,19 @@ public void testApplyCollection() { "select first::String,last + 1 as result"); Assert.assertEquals(q.apply(Arrays.asList(new String[] { "first", "last", "more" })), "select first::String,last + 1 as result"); + + query = "select :p1 p1, :p2 p2, :p1 p3"; + q = ClickHouseParameterizedQuery.of(query); + Assert.assertTrue(q.getOriginalQuery() == query); + Assert.assertEquals(q.apply((Collection) null), "select NULL p1, NULL p2, NULL p3"); + Assert.assertEquals(q.apply(Collections.emptyList()), "select NULL p1, NULL p2, NULL p3"); + Assert.assertEquals(q.apply(Collections.emptySet()), "select NULL p1, NULL p2, NULL p3"); + Assert.assertEquals(q.apply(Arrays.asList(new String[] { "first" })), + "select first p1, NULL p2, first p3"); + Assert.assertEquals(q.apply(Arrays.asList(new String[] { "first", "last" })), + "select first p1, last p2, first p3"); + Assert.assertEquals(q.apply(Arrays.asList(new String[] { "first", "last", "more" })), + "select first p1, last p2, first p3"); } @Test(groups = { "unit" }) @@ -71,6 +84,22 @@ public void testApplyObjects() { Assert.assertEquals(q.apply(Arrays.asList(1, null)), "select (1,NULL)::String,NULL + 1 as result"); Assert.assertEquals(q.apply(Arrays.asList(ClickHouseDateTimeValue.ofNull(3).update(1), null)), "select ('1970-01-01 00:00:00.001',NULL)::String,NULL + 1 as result"); + + query = "select :p1 p1, :p2 p2, :p1 p3"; + q = ClickHouseParameterizedQuery.of(query); + Assert.assertTrue(q.getOriginalQuery() == query); + Assert.assertEquals(q.apply((Object) null), "select NULL p1, NULL p2, NULL p3"); + Assert.assertEquals(q.apply((Object) null, (Object) null), "select NULL p1, NULL p2, NULL p3"); + Assert.assertEquals(q.apply('a'), "select 97 p1, NULL p2, 97 p3"); + Assert.assertEquals(q.apply(1, (Object) null), "select 1 p1, NULL p2, 1 p3"); + Assert.assertEquals(q.apply(ClickHouseDateTimeValue.ofNull(3).update(1), (Object) null), + "select '1970-01-01 00:00:00.001' p1, NULL p2, '1970-01-01 00:00:00.001' p3"); + Assert.assertEquals(q.apply(Collections.singletonList('a')), "select (97) p1, NULL p2, (97) p3"); + Assert.assertEquals(q.apply(Arrays.asList(1, null)), "select (1,NULL) p1, NULL p2, (1,NULL) p3"); + Assert.assertEquals(q.apply(Arrays.asList(ClickHouseDateTimeValue.ofNull(3).update(1), null)), + "select ('1970-01-01 00:00:00.001',NULL) p1, NULL p2, ('1970-01-01 00:00:00.001',NULL) p3"); + Assert.assertEquals(q.apply(new StringBuilder("321"), new StringBuilder("123"), new StringBuilder("456")), + "select 321 p1, 123 p2, 321 p3"); } @Test(groups = { "unit" }) @@ -116,6 +145,15 @@ public void testApplyStrings() { Assert.assertEquals(q.apply((String) null, (String) null), "select null::String,null + 1 as result"); Assert.assertEquals(q.apply("'a'"), "select 'a'::String,NULL + 1 as result"); Assert.assertEquals(q.apply("1", (String) null), "select 1::String,null + 1 as result"); + + query = "select :p1 p1, :p2 p2, :p1 p3"; + q = ClickHouseParameterizedQuery.of(query); + Assert.assertTrue(q.getOriginalQuery() == query); + Assert.assertEquals(q.apply((String) null), "select null p1, NULL p2, null p3"); + Assert.assertEquals(q.apply((String) null, (String) null), "select null p1, null p2, null p3"); + Assert.assertEquals(q.apply("'a'"), "select 'a' p1, NULL p2, 'a' p3"); + Assert.assertEquals(q.apply("1", (String) null), "select 1 p1, null p2, 1 p3"); + Assert.assertEquals(q.apply("1", "2", "3"), "select 1 p1, 2 p2, 1 p3"); } @Test(groups = { "unit" }) @@ -195,7 +233,7 @@ public void testApplyTypedParameters() { Assert.assertNull(templates[0]); Assert.assertTrue(templates[1] instanceof ClickHouseDateTimeValue); Assert.assertEquals(((ClickHouseDateTimeValue) templates[1]).getScale(), 0); - Assert.assertEquals(pq.apply(ts, ts, ts), // shoud support only two parameters + Assert.assertEquals(pq.apply(ts, ts), "select '1970-01-01 02:46:40.123456789' ts1, '1970-01-01 02:46:40' ts2, '1970-01-01 02:46:40' ts3"); } } 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 64e4495fa..f3a750761 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 @@ -38,7 +38,7 @@ protected ClickHouseHttpConnection newConnection(ClickHouseHttpConnection connec } try { - return new DefaultHttpConnection(server, request, getExecutor()); + return ClickHouseHttpConnectionFactory.createConnection(server, request, getExecutor()); } catch (IOException e) { throw new CompletionException(e); } 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 f362066e1..1c4f8ef92 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 @@ -248,7 +248,7 @@ protected ClickHouseInputStream getResponseInputStream(InputStream in) throws IO } } - return in instanceof ClickHouseInputStream ? (ClickHouseInputStream) in : ClickHouseInputStream.of(in); + return ClickHouseInputStream.of(in); } /** diff --git a/clickhouse-http-client/src/main/java/com/clickhouse/client/http/ClickHouseHttpConnectionFactory.java b/clickhouse-http-client/src/main/java/com/clickhouse/client/http/ClickHouseHttpConnectionFactory.java new file mode 100644 index 000000000..ac06e19ae --- /dev/null +++ b/clickhouse-http-client/src/main/java/com/clickhouse/client/http/ClickHouseHttpConnectionFactory.java @@ -0,0 +1,14 @@ +package com.clickhouse.client.http; + +import java.io.IOException; +import java.util.concurrent.ExecutorService; + +import com.clickhouse.client.ClickHouseNode; +import com.clickhouse.client.ClickHouseRequest; + +public abstract class ClickHouseHttpConnectionFactory { + public static ClickHouseHttpConnection createConnection(ClickHouseNode server, ClickHouseRequest request, + ExecutorService executor) throws IOException { + return new HttpUrlConnectionImpl(server, request, executor); + } +} diff --git a/clickhouse-http-client/src/main/java/com/clickhouse/client/http/DefaultHttpConnection.java b/clickhouse-http-client/src/main/java/com/clickhouse/client/http/HttpUrlConnectionImpl.java similarity index 95% rename from clickhouse-http-client/src/main/java/com/clickhouse/client/http/DefaultHttpConnection.java rename to clickhouse-http-client/src/main/java/com/clickhouse/client/http/HttpUrlConnectionImpl.java index 3564f3817..a0607e6d3 100644 --- a/clickhouse-http-client/src/main/java/com/clickhouse/client/http/DefaultHttpConnection.java +++ b/clickhouse-http-client/src/main/java/com/clickhouse/client/http/HttpUrlConnectionImpl.java @@ -11,6 +11,7 @@ import com.clickhouse.client.logging.Logger; import com.clickhouse.client.logging.LoggerFactory; +import java.io.BufferedInputStream; import java.io.BufferedReader; import java.io.BufferedWriter; import java.io.ByteArrayOutputStream; @@ -34,8 +35,8 @@ import javax.net.ssl.HttpsURLConnection; import javax.net.ssl.SSLContext; -public class DefaultHttpConnection extends ClickHouseHttpConnection { - private static final Logger log = LoggerFactory.getLogger(DefaultHttpConnection.class); +public class HttpUrlConnectionImpl extends ClickHouseHttpConnection { + private static final Logger log = LoggerFactory.getLogger(HttpUrlConnectionImpl.class); private final HttpURLConnection conn; @@ -62,7 +63,8 @@ private ClickHouseHttpResponse buildResponse() throws IOException { : timeZone; } - return new ClickHouseHttpResponse(this, getResponseInputStream(conn.getInputStream()), + return new ClickHouseHttpResponse(this, + getResponseInputStream(new BufferedInputStream(conn.getInputStream(), config.getMaxBufferSize())), displayName, queryId, summary, format, timeZone); } @@ -131,7 +133,7 @@ private void checkResponse(HttpURLConnection conn) throws IOException { } } - protected DefaultHttpConnection(ClickHouseNode server, ClickHouseRequest request, ExecutorService executor) + protected HttpUrlConnectionImpl(ClickHouseNode server, ClickHouseRequest request, ExecutorService executor) throws IOException { super(server, request); diff --git a/clickhouse-http-client/src/main/java/com/clickhouse/client/http/config/ClickHouseHttpOption.java b/clickhouse-http-client/src/main/java/com/clickhouse/client/http/config/ClickHouseHttpOption.java index cdf90592f..74e9272bc 100644 --- a/clickhouse-http-client/src/main/java/com/clickhouse/client/http/config/ClickHouseHttpOption.java +++ b/clickhouse-http-client/src/main/java/com/clickhouse/client/http/config/ClickHouseHttpOption.java @@ -9,6 +9,11 @@ * gRPC client options. */ public enum ClickHouseHttpOption implements ClickHouseOption { + /** + * HTTP connection provider. + */ + CONNECTION_PROVIDER("http_connection_provider", HttpConnectionProvider.HTTP_URL_CONNECTION, + "HTTP connection provider. HTTP_CLIENT is only supported in JDK 11 or above."), /** * Custom HTTP headers. */ diff --git a/clickhouse-http-client/src/main/java/com/clickhouse/client/http/config/HttpConnectionProvider.java b/clickhouse-http-client/src/main/java/com/clickhouse/client/http/config/HttpConnectionProvider.java new file mode 100644 index 000000000..2eba73bdf --- /dev/null +++ b/clickhouse-http-client/src/main/java/com/clickhouse/client/http/config/HttpConnectionProvider.java @@ -0,0 +1,6 @@ +package com.clickhouse.client.http.config; + +public enum HttpConnectionProvider { + HTTP_CLIENT, + HTTP_URL_CONNECTION +} diff --git a/clickhouse-http-client/src/main/java11/com/clickhouse/client/http/ClickHouseHttpConnectionFactory.java b/clickhouse-http-client/src/main/java11/com/clickhouse/client/http/ClickHouseHttpConnectionFactory.java new file mode 100644 index 000000000..f5cf08329 --- /dev/null +++ b/clickhouse-http-client/src/main/java11/com/clickhouse/client/http/ClickHouseHttpConnectionFactory.java @@ -0,0 +1,27 @@ +package com.clickhouse.client.http; + +import java.io.IOException; +import java.util.concurrent.ExecutorService; + +import com.clickhouse.client.ClickHouseNode; +import com.clickhouse.client.ClickHouseRequest; +import com.clickhouse.client.http.config.ClickHouseHttpOption; +import com.clickhouse.client.http.config.HttpConnectionProvider; + +public abstract class ClickHouseHttpConnectionFactory { + public static ClickHouseHttpConnection createConnection(ClickHouseNode server, ClickHouseRequest request, + ExecutorService executor) throws IOException { + HttpConnectionProvider provider = (HttpConnectionProvider) request.getConfig() + .getOption(ClickHouseHttpOption.CONNECTION_PROVIDER); + + try { + return provider == null || provider == HttpConnectionProvider.HTTP_URL_CONNECTION + ? new HttpUrlConnectionImpl(server, request, executor) + : new HttpClientConnectionImpl(server, request, executor); + } catch (IOException e) { + throw e; + } catch (Throwable t) { + return new HttpUrlConnectionImpl(server, request, executor); + } + } +} diff --git a/clickhouse-http-client/src/main/java11/com/clickhouse/client/http/DefaultHttpConnection.java b/clickhouse-http-client/src/main/java11/com/clickhouse/client/http/HttpClientConnectionImpl.java similarity index 97% rename from clickhouse-http-client/src/main/java11/com/clickhouse/client/http/DefaultHttpConnection.java rename to clickhouse-http-client/src/main/java11/com/clickhouse/client/http/HttpClientConnectionImpl.java index a8f612a54..8658e985c 100644 --- a/clickhouse-http-client/src/main/java11/com/clickhouse/client/http/DefaultHttpConnection.java +++ b/clickhouse-http-client/src/main/java11/com/clickhouse/client/http/HttpClientConnectionImpl.java @@ -41,8 +41,8 @@ import javax.net.ssl.SSLContext; -public class DefaultHttpConnection extends ClickHouseHttpConnection { - private static final Logger log = LoggerFactory.getLogger(DefaultHttpConnection.class); +public class HttpClientConnectionImpl extends ClickHouseHttpConnection { + private static final Logger log = LoggerFactory.getLogger(HttpClientConnectionImpl.class); private static final int MAX_RETRIES = 1; @@ -98,7 +98,7 @@ private HttpRequest newRequest(String url) { .timeout(Duration.ofMillis(config.getSocketTimeout())).build(); } - protected DefaultHttpConnection(ClickHouseNode server, ClickHouseRequest request, ExecutorService executor) + protected HttpClientConnectionImpl(ClickHouseNode server, ClickHouseRequest request, ExecutorService executor) throws IOException { super(server, request); diff --git a/clickhouse-http-client/src/test/java/com/clickhouse/client/http/DefaultHttpConnectionTest.java b/clickhouse-http-client/src/test/java/com/clickhouse/client/http/DefaultHttpConnectionTest.java index a9153eccf..c8b62b8b1 100644 --- a/clickhouse-http-client/src/test/java/com/clickhouse/client/http/DefaultHttpConnectionTest.java +++ b/clickhouse-http-client/src/test/java/com/clickhouse/client/http/DefaultHttpConnectionTest.java @@ -22,7 +22,7 @@ public void testConnectionReuse() throws Exception { try (ClickHouseClient client = ClickHouseClient.newInstance()) { ClickHouseRequest req = client.connect(server); - DefaultHttpConnection conn = new DefaultHttpConnection(server, req, null); + ClickHouseHttpConnection conn = ClickHouseHttpConnectionFactory.createConnection(server, req, null); } } } diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseResultSet.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseResultSet.java index f94c6d68a..087e4cb15 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseResultSet.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseResultSet.java @@ -2,6 +2,7 @@ import java.io.InputStream; import java.io.Reader; +import java.io.UncheckedIOException; import java.math.BigDecimal; import java.math.BigInteger; import java.net.MalformedURLException; @@ -695,7 +696,11 @@ public boolean next() throws SQLException { lastReadColumn = 0; boolean hasNext = true; if (hasNext()) { - currentRow = rowCursor.next(); + try { + currentRow = rowCursor.next(); + } catch (UncheckedIOException e) { + throw SqlExceptionUtils.handle(e); + } rowNumber++; } else { currentRow = null; diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseResultSetMetaData.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseResultSetMetaData.java index d7ea52b1d..a78cc1555 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseResultSetMetaData.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseResultSetMetaData.java @@ -2,7 +2,6 @@ import java.sql.ResultSetMetaData; import java.sql.SQLException; -import java.sql.Types; import java.util.List; import com.clickhouse.client.ClickHouseColumn; @@ -142,6 +141,6 @@ public boolean isDefinitelyWritable(int column) throws SQLException { @Override public String getColumnClassName(int column) throws SQLException { - return getColumn(column).getDataType().getObjectClass().getCanonicalName(); + return JdbcTypeMapping.toJavaClass(getColumn(column)).getCanonicalName(); } } diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcParameterizedQuery.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcParameterizedQuery.java index 4fe51f147..2467a0f93 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcParameterizedQuery.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcParameterizedQuery.java @@ -1,11 +1,11 @@ package com.clickhouse.jdbc; import java.util.Collection; -import java.util.Collections; -import java.util.Map; +import java.util.Iterator; import com.clickhouse.client.ClickHouseParameterizedQuery; import com.clickhouse.client.ClickHouseUtils; +import com.clickhouse.client.ClickHouseValues; /** * A parameterized query is a parsed query with parameters being extracted for @@ -61,4 +61,101 @@ protected String parse() { return partIndex < len ? originalQuery.substring(partIndex, len) : null; } + + @Override + public String apply(Collection params) { + if (!hasParameter()) { + return originalQuery; + } + + StringBuilder builder = new StringBuilder(); + Iterator it = params == null ? null : params.iterator(); + boolean hasMore = it != null && it.hasNext(); + for (QueryPart p : getParts()) { + builder.append(p.part); + builder.append(hasMore ? it.next() : ClickHouseValues.NULL_EXPR); + hasMore = hasMore && it.hasNext(); + } + + return appendLastPartIfExists(builder).toString(); + } + + @Override + public String apply(Object param, Object... more) { + if (!hasParameter()) { + return originalQuery; + } + + int len = more == null ? 0 : more.length + 1; + StringBuilder builder = new StringBuilder(); + int index = 0; + for (QueryPart p : getParts()) { + builder.append(p.part); + if (index > 0) { + param = index < len ? more[index - 1] : null; + } + builder.append(toSqlExpression(p.paramName, param)); + index++; + } + + return appendLastPartIfExists(builder).toString(); + } + + @Override + public String apply(Object[] values) { + if (!hasParameter()) { + return originalQuery; + } + + int len = values == null ? 0 : values.length; + StringBuilder builder = new StringBuilder(); + int index = 0; + for (QueryPart p : getParts()) { + builder.append(p.part); + builder.append( + index < len ? toSqlExpression(p.paramName, values[index]) : ClickHouseValues.NULL_EXPR); + index++; + } + + return appendLastPartIfExists(builder).toString(); + } + + @Override + public String apply(String param, String... more) { + if (!hasParameter()) { + return originalQuery; + } + + int len = more == null ? 0 : more.length + 1; + StringBuilder builder = new StringBuilder(); + int index = 0; + for (QueryPart p : getParts()) { + builder.append(p.part); + if (index > 0) { + param = index < len ? more[index - 1] : ClickHouseValues.NULL_EXPR; + } + builder.append(param); + index++; + } + + return appendLastPartIfExists(builder).toString(); + } + + @Override + public String apply(String[] values) { + if (!hasParameter()) { + return originalQuery; + } + + int len = values == null ? 0 : values.length; + StringBuilder builder = new StringBuilder(); + int index = 0; + for (QueryPart p : getParts()) { + builder.append(p.part); + builder.append(index < len ? values[index] : ClickHouseValues.NULL_EXPR); + index++; + } + + return appendLastPartIfExists(builder).toString(); + } } diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcTypeMapping.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcTypeMapping.java index 20ebf4155..eaa4adb7a 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcTypeMapping.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcTypeMapping.java @@ -1,6 +1,8 @@ package com.clickhouse.jdbc; import java.sql.Types; +import java.time.LocalDateTime; +import java.time.OffsetDateTime; import com.clickhouse.client.ClickHouseColumn; import com.clickhouse.client.ClickHouseDataType; @@ -100,6 +102,22 @@ public static int toJdbcType(ClickHouseColumn column) { return sqlType; } + public static Class toJavaClass(ClickHouseColumn column) { + Class clazz; + ClickHouseDataType type = column.getDataType(); + switch (type) { + case DateTime: + case DateTime32: + case DateTime64: + clazz = column.getTimeZone() != null ? OffsetDateTime.class : LocalDateTime.class; + break; + default: + clazz = type.getObjectClass(); + break; + } + return clazz; + } + public static ClickHouseColumn fromJdbcType(int jdbcType, int scaleOrLength) { ClickHouseDataType dataType = fromJdbcType(jdbcType); ClickHouseColumn column = null; diff --git a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHousePreparedStatementTest.java b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHousePreparedStatementTest.java index e4d0eed06..4abd581ab 100644 --- a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHousePreparedStatementTest.java +++ b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHousePreparedStatementTest.java @@ -151,13 +151,31 @@ public void testQueryWithNamedParameter() throws SQLException { LocalDateTime ts = LocalDateTime.ofEpochSecond(10000, 123456789, ZoneOffset.UTC); try (ClickHouseConnection conn = newConnection(props); PreparedStatement stmt = conn - .prepareStatement("select :ts1 ts1, :ts2(DateTime32) ts2")) { + .prepareStatement("select :ts1 ts1, :ts2(DateTime32) ts2, :ts2 ts3")) { + // just two parameters here - ts2 is referenced twice stmt.setObject(1, ts); stmt.setObject(2, ts); ResultSet rs = stmt.executeQuery(); Assert.assertTrue(rs.next()); Assert.assertEquals(rs.getString(1), "1970-01-01 02:46:40.123456789"); Assert.assertEquals(rs.getString(2), "1970-01-01 02:46:40"); + Assert.assertEquals(rs.getString(3), "1970-01-01 02:46:40"); + Assert.assertFalse(rs.next()); + } + + // try again using JDBC standard question mark placeholder + try (ClickHouseConnection conn = newConnection(); + PreparedStatement stmt = conn + .prepareStatement("select ? ts1, ? ts2, ? ts3")) { + // unlike above, this time we have 3 parameters + stmt.setObject(1, "1970-01-01 02:46:40.123456789"); + stmt.setObject(2, "1970-01-01 02:46:40"); + stmt.setObject(3, "1970-01-01 02:46:40"); + ResultSet rs = stmt.executeQuery(); + Assert.assertTrue(rs.next()); + Assert.assertEquals(rs.getString(1), "1970-01-01 02:46:40.123456789"); + Assert.assertEquals(rs.getString(2), "1970-01-01 02:46:40"); + Assert.assertEquals(rs.getString(3), "1970-01-01 02:46:40"); Assert.assertFalse(rs.next()); } } From 262b3ebbfcc818b20451025b7a040890a3a4cf96 Mon Sep 17 00:00:00 2001 From: Zhichun Wu Date: Tue, 21 Dec 2021 22:42:02 +0800 Subject: [PATCH 4/6] Optimize LZ4 input and output stream --- .../client/ClickHouseInputStream.java | 163 ++++++++++++++++-- .../client/data/BinaryStreamUtils.java | 46 +++-- .../client/data/ClickHouseBlockChecksum.java | 66 ------- .../client/data/ClickHouseLZ4InputStream.java | 120 ++++++------- .../data/ClickHouseLZ4OutputStream.java | 49 +++--- .../client/ClickHouseInputStreamTest.java | 30 +++- .../client/http/ClickHouseHttpConnection.java | 2 +- .../client/http/HttpUrlConnectionImpl.java | 5 +- .../clickhouse/jdbc/ClickHouseResultSet.java | 50 ++++-- 9 files changed, 316 insertions(+), 215 deletions(-) delete mode 100644 clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseBlockChecksum.java 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 fb85bcd5b..e5e6d1c64 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseInputStream.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseInputStream.java @@ -10,6 +10,8 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.TimeUnit; +import com.clickhouse.client.config.ClickHouseClientOption; + /** * Extended input stream for read optimization. */ @@ -184,11 +186,17 @@ public long skip(long n) throws IOException { static final class WrappedInputStream extends ClickHouseInputStream { private final InputStream in; + private final byte[] buffer; + private int position; + private int limit; private boolean closed; - WrappedInputStream(InputStream input) { + WrappedInputStream(InputStream input, int bufferSize) { in = ClickHouseChecker.nonNull(input, "InputStream"); + buffer = new byte[bufferSize]; + position = 0; + limit = 0; closed = false; } @@ -198,26 +206,34 @@ private void ensureOpen() throws IOException { } } + private int updateBuffer() throws IOException { + if (closed) { + return -1; + } + + position = 0; + int count = in.read(buffer); + limit = count > 0 ? count : 0; + return count; + } + @Override public int available() throws IOException { - return !closed ? in.available() : 0; + return !closed && (position < limit || updateBuffer() > 0) ? limit - position : 0; } @Override public byte readByte() throws IOException { - ensureOpen(); - - int v = in.read(); - if (v != -1) { - return (byte) v; + if (position >= limit && updateBuffer() < 0) { + try { + close(); + } catch (IOException e) { + // ignore + } + throw new EOFException(); } - try { - close(); - } catch (IOException e) { - // ignore - } - throw new EOFException(); + return buffer[position++]; } @Override @@ -232,6 +248,8 @@ public void close() throws IOException { in.close(); } finally { closed = true; + position = 0; + limit = 0; } } } @@ -239,19 +257,116 @@ public void close() throws IOException { @Override public int read() throws IOException { ensureOpen(); - return in.read(); + + int value = -1; + if (position < limit || updateBuffer() > 0) { + value = 0xFF & buffer[position++]; + } + return value; } @Override public int read(byte[] b, int off, int len) throws IOException { + if (position >= limit && updateBuffer() < 0) { + return -1; + } + ensureOpen(); - return in.read(b, off, len); + + int counter = 0; + while (counter < len) { + int size = Math.min(limit - position, len - counter); + System.arraycopy(buffer, position, b, off, size); + position += size; + off += size; + counter += size; + + if (position >= limit && updateBuffer() < 0) { + break; + } + } + + return counter; + } + + @Override + public byte[] readBytes(int length) throws IOException { + if (length <= 0) { + return EMPTY_BYTES; + } + + ensureOpen(); + + byte[] bytes = new byte[length]; + int offset = 0; + int counter = 0; + while (counter < length) { + if (position >= limit && updateBuffer() < 0) { + try { + close(); + } catch (IOException e) { + // ignore + } + throw counter == 0 ? new EOFException() + : new IOException(ClickHouseUtils + .format("Reached end of input stream after reading %d of %d bytes", counter, + bytes.length)); + } + + int size = Math.min(limit - position, length - counter); + System.arraycopy(buffer, position, bytes, offset, size); + position += size; + offset += size; + counter += size; + } + + return bytes; + } + + @Override + public String readString(int byteLength, Charset charset) throws IOException { + ensureOpen(); + + if (byteLength < 1) { + return ""; + } + + if (charset == null) { + charset = StandardCharsets.UTF_8; + } + + if (limit - position > byteLength) { + int offset = position; + position += byteLength; + return new String(buffer, offset, byteLength, charset); + } + + return new String(readBytes(byteLength), charset); } @Override public long skip(long n) throws IOException { ensureOpen(); - return in.skip(n); + + long counter = 0L; + while (n > 0L) { + if (position >= limit && updateBuffer() < 0) { + break; + } else { + int remain = limit - position; + if (n > remain) { + n -= remain; + counter += remain; + position = limit; + } else { + counter += n; + position += n; + n = 0L; + } + } + } + + return counter; } } @@ -274,7 +389,21 @@ public static ClickHouseInputStream of(BlockingQueue queue, int time * {@link ClickHouseInputStream} */ public static ClickHouseInputStream of(InputStream input) { - return input instanceof ClickHouseInputStream ? (ClickHouseInputStream) input : new WrappedInputStream(input); + return of(input, (int) ClickHouseClientOption.MAX_BUFFER_SIZE.getDefaultValue()); + } + + /** + * Wraps the given input stream. + * + * @param input non-null input stream + * @param bufferSize buffer size which is always greater than zero(usually 4096 + * or larger) + * @return wrapped input, or the same input if it's instance of + * {@link ClickHouseInputStream} + */ + public static ClickHouseInputStream of(InputStream input, int bufferSize) { + return input instanceof ClickHouseInputStream ? (ClickHouseInputStream) input + : new WrappedInputStream(input, bufferSize); } /** diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/data/BinaryStreamUtils.java b/clickhouse-client/src/main/java/com/clickhouse/client/data/BinaryStreamUtils.java index ac41e4f22..f26dece4d 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/data/BinaryStreamUtils.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/data/BinaryStreamUtils.java @@ -91,15 +91,33 @@ private static > T toEnum(int value, Class enumType) { } public static int toInt32(byte[] bytes, int offset) { - return (0xFF & bytes[offset]) | ((0xFF & bytes[offset + 1]) << 8) | ((0xFF & bytes[offset + 2]) << 16) - | ((0xFF & bytes[offset + 3]) << 24); + return (0xFF & bytes[offset++]) | ((0xFF & bytes[offset++]) << 8) | ((0xFF & bytes[offset++]) << 16) + | ((0xFF & bytes[offset]) << 24); } public static long toInt64(byte[] bytes, int offset) { - return (0xFFL & bytes[offset]) | ((0xFFL & bytes[offset + 1]) << 8) | ((0xFFL & bytes[offset + 2]) << 16) - | ((0xFFL & bytes[offset + 3]) << 24) | ((0xFFL & bytes[offset + 4]) << 32) - | ((0xFFL & bytes[offset + 5]) << 40) | ((0xFFL & bytes[offset + 6]) << 48) - | ((0xFFL & bytes[offset + 7]) << 56); + return (0xFFL & bytes[offset++]) | ((0xFFL & bytes[offset++]) << 8) | ((0xFFL & bytes[offset++]) << 16) + | ((0xFFL & bytes[offset++]) << 24) | ((0xFFL & bytes[offset++]) << 32) + | ((0xFFL & bytes[offset++]) << 40) | ((0xFFL & bytes[offset++]) << 48) + | ((0xFFL & bytes[offset]) << 56); + } + + public static void setInt32(byte[] bytes, int offset, int value) { + bytes[offset++] = (byte) (0xFF & value); + bytes[offset++] = (byte) (0xFF & (value >> 8)); + bytes[offset++] = (byte) (0xFF & (value >> 16)); + bytes[offset] = (byte) (0xFF & (value >> 24)); + } + + public static void setInt64(byte[] bytes, int offset, long value) { + bytes[offset++] = (byte) (0xFF & value); + bytes[offset++] = (byte) (0xFF & (value >> 8)); + bytes[offset++] = (byte) (0xFF & (value >> 16)); + bytes[offset++] = (byte) (0xFF & (value >> 24)); + bytes[offset++] = (byte) (0xFF & (value >> 32)); + bytes[offset++] = (byte) (0xFF & (value >> 40)); + bytes[offset++] = (byte) (0xFF & (value >> 48)); + bytes[offset] = (byte) (0xFF & (value >> 56)); } /** @@ -662,7 +680,7 @@ public static short readUnsignedInt8(ClickHouseInputStream input) throws IOExcep * end of the stream */ public static void writeUnsignedInt8(OutputStream output, int value) throws IOException { - output.write((byte) (ClickHouseChecker.between(value, ClickHouseValues.TYPE_INT, 0, U_INT8_MAX) & 0xFFL)); + output.write((byte) (0xFF & ClickHouseChecker.between(value, ClickHouseValues.TYPE_INT, 0, U_INT8_MAX))); } /** @@ -686,7 +704,7 @@ public static short readInt16(ClickHouseInputStream input) throws IOException { * end of the stream */ public static void writeInt16(OutputStream output, short value) throws IOException { - output.write(new byte[] { (byte) (0xFFL & value), (byte) (0xFFL & (value >> 8)) }); + output.write(new byte[] { (byte) (0xFF & value), (byte) (0xFF & (value >> 8)) }); } /** @@ -749,8 +767,8 @@ public static int readInt32(ClickHouseInputStream input) throws IOException { * end of the stream */ public static void writeInt32(OutputStream output, int value) throws IOException { - output.write(new byte[] { (byte) (0xFFL & value), (byte) (0xFFL & (value >> 8)), (byte) (0xFFL & (value >> 16)), - (byte) (0xFFL & (value >> 24)) }); + output.write(new byte[] { (byte) (0xFF & value), (byte) (0xFF & (value >> 8)), (byte) (0xFF & (value >> 16)), + (byte) (0xFF & (value >> 24)) }); } /** @@ -799,14 +817,8 @@ public static long readInt64(ClickHouseInputStream input) throws IOException { * end of the stream */ public static void writeInt64(OutputStream output, long value) throws IOException { - value = Long.reverseBytes(value); - byte[] bytes = new byte[8]; - for (int i = 7; i >= 0; i--) { - bytes[i] = (byte) (value & 0xFFL); - value >>= 8; - } - + setInt64(bytes, 0, value); output.write(bytes); } diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseBlockChecksum.java b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseBlockChecksum.java deleted file mode 100644 index 00f31bdec..000000000 --- a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseBlockChecksum.java +++ /dev/null @@ -1,66 +0,0 @@ -package com.clickhouse.client.data; - -import java.nio.Buffer; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; - -public class ClickHouseBlockChecksum { - private final long first; - private final long second; - - public ClickHouseBlockChecksum(long first, long second) { - this.first = first; - this.second = second; - } - - public static ClickHouseBlockChecksum fromBytes(byte[] checksum) { - ByteBuffer buffer = ByteBuffer.allocate(16).order(ByteOrder.LITTLE_ENDIAN).put(checksum); - ((Buffer) buffer).flip(); - return new ClickHouseBlockChecksum(buffer.getLong(), buffer.getLong()); - } - - public static ClickHouseBlockChecksum calculateForBlock(byte magic, int compressedSizeWithHeader, - int uncompressedSize, byte[] data, int length) { - ByteBuffer buffer = ByteBuffer.allocate(compressedSizeWithHeader).order(ByteOrder.LITTLE_ENDIAN) - .put((byte) magic).putInt(compressedSizeWithHeader).putInt(uncompressedSize).put(data, 0, length); - ((Buffer) buffer).flip(); - return calculate(buffer.array()); - } - - public byte[] asBytes() { - ByteBuffer buffer = ByteBuffer.allocate(16).order(ByteOrder.LITTLE_ENDIAN).putLong(first).putLong(second); - ((Buffer) buffer).flip(); - return buffer.array(); - } - - private static ClickHouseBlockChecksum calculate(byte[] data) { - long[] sum = ClickHouseCityHash.cityHash128(data, 0, data.length); - return new ClickHouseBlockChecksum(sum[0], sum[1]); - } - - @Override - public boolean equals(Object o) { - if (this == o) - return true; - if (o == null || getClass() != o.getClass()) - return false; - - ClickHouseBlockChecksum that = (ClickHouseBlockChecksum) o; - - if (first != that.first) - return false; - return second == that.second; - } - - @Override - public int hashCode() { - int result = (int) (first ^ (first >>> 32)); - result = 31 * result + (int) (second ^ (second >>> 32)); - return result; - } - - @Override - public String toString() { - return "{" + first + ", " + second + '}'; - } -} diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseLZ4InputStream.java b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseLZ4InputStream.java index d563f8096..0cee8bfed 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseLZ4InputStream.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseLZ4InputStream.java @@ -3,13 +3,12 @@ import java.io.EOFException; import java.io.IOException; import java.io.InputStream; -import java.nio.Buffer; -import java.nio.ByteBuffer; import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; import com.clickhouse.client.ClickHouseChecker; import com.clickhouse.client.ClickHouseInputStream; +import com.clickhouse.client.ClickHouseUtils; import net.jpountz.lz4.LZ4Factory; import net.jpountz.lz4.LZ4FastDecompressor; @@ -20,95 +19,84 @@ public class ClickHouseLZ4InputStream extends ClickHouseInputStream { private static final LZ4Factory factory = LZ4Factory.fastestInstance(); - static final int MAGIC = 0x82; + static final byte MAGIC = (byte) 0x82; + static final int HEADER_LENGTH = 25; + private final LZ4FastDecompressor decompressor; private final InputStream stream; + private final byte[] header; - private ByteBuffer currentBlock; + private byte[] currentBlock; + private int position; private boolean closed; private boolean checkNext() throws IOException { - if (currentBlock == null || !currentBlock.hasRemaining()) { + if (!closed && position >= currentBlock.length) { currentBlock = readNextBlock(); } - return currentBlock != null; + return currentBlock.length > 0; } // every block is: - private ByteBuffer readNextBlock() throws IOException { - int read = stream.read(); - if (read < 0) { - return null; - } + private byte[] readNextBlock() throws IOException { + position = 0; - byte[] bytes = new byte[16]; - bytes[0] = (byte) read; - // checksum - 16 bytes. - readFully(bytes, 1, 15); - ClickHouseBlockChecksum expected = ClickHouseBlockChecksum.fromBytes(bytes); - // header: - // 1 byte - 0x82 (shows this is LZ4) - int magic = readUnsignedByteFromInput(); - if (magic != MAGIC) { - throw new IOException("Magic is not correct: " + magic); + // checksum(16 bytes) + 1 magic byte + header(8 bytes) + if (!readFully(header, 0, HEADER_LENGTH)) { + return EMPTY_BYTES; + } else if (header[16] != MAGIC) { + // 1 byte - 0x82 (shows this is LZ4) + throw new IOException( + ClickHouseUtils.format("Magic is not correct - expect [%d] but got [%d]", MAGIC, header[16])); } - readFully(bytes, 0, 8); // 4 bytes - size of the compressed data including 9 bytes of the header - int compressedSizeWithHeader = BinaryStreamUtils.toInt32(bytes, 0); + int compressedSizeWithHeader = BinaryStreamUtils.toInt32(header, 17); // 4 bytes - size of uncompressed data - int uncompressedSize = BinaryStreamUtils.toInt32(bytes, 4); - int compressedSize = compressedSizeWithHeader - 9; // header - byte[] block = new byte[compressedSize]; - // compressed data: compressed_size - 9 байт. - readFully(block, 0, block.length); - - ClickHouseBlockChecksum real = ClickHouseBlockChecksum.calculateForBlock((byte) magic, compressedSizeWithHeader, - uncompressedSize, block, compressedSize); - if (!real.equals(expected)) { + int uncompressedSize = BinaryStreamUtils.toInt32(header, 21); + int offset = 9; + byte[] block = new byte[compressedSizeWithHeader]; + block[0] = header[16]; + BinaryStreamUtils.setInt32(block, 1, compressedSizeWithHeader); + BinaryStreamUtils.setInt32(block, 5, uncompressedSize); + // compressed data: compressed_size - 9 bytes + if (!readFully(block, offset, compressedSizeWithHeader - offset)) { + throw new EOFException(); + } + + long[] real = ClickHouseCityHash.cityHash128(block, 0, block.length); + if (real[0] != BinaryStreamUtils.toInt64(header, 0) || real[1] != BinaryStreamUtils.toInt64(header, 8)) { throw new IllegalArgumentException("Checksum doesn't match: corrupted data."); } byte[] decompressed = new byte[uncompressedSize]; - LZ4FastDecompressor decompressor = factory.fastDecompressor(); - decompressor.decompress(block, 0, decompressed, 0, uncompressedSize); - return ByteBuffer.wrap(decompressed); + decompressor.decompress(block, offset, decompressed, 0, uncompressedSize); + return decompressed; } - private void readFully(byte b[], int off, int len) throws IOException { - if (len < 0) { - throw new IndexOutOfBoundsException(); - } + private boolean readFully(byte[] b, int off, int len) throws IOException { int n = 0; while (n < len) { int count = stream.read(b, off + n, len - n); if (count < 0) { - try { - close(); - } catch (IOException e) { - // ignore + if (n == 0) { + return false; } throw new EOFException(); } n += count; } - } - private int readUnsignedByteFromInput() throws IOException { - int ch = stream.read(); - if (ch < 0) { - try { - close(); - } catch (IOException e) { - // ignore - } - throw new EOFException(); - } - return ch; + return true; } public ClickHouseLZ4InputStream(InputStream stream) { + this.decompressor = factory.fastDecompressor(); this.stream = ClickHouseChecker.nonNull(stream, "InputStream"); + this.header = new byte[HEADER_LENGTH]; + + this.currentBlock = EMPTY_BYTES; + this.position = 0; this.closed = false; } @@ -123,7 +111,7 @@ public byte readByte() throws IOException { throw new EOFException(); } - return currentBlock.get(); + return currentBlock[position++]; } @Override @@ -132,16 +120,16 @@ public int available() throws IOException { return 0; } - int estimated = stream.available(); + int estimated = currentBlock.length - position; if (estimated == 0 && checkNext()) { - estimated = currentBlock.remaining(); + estimated = currentBlock.length - position; } return estimated; } @Override public int read() throws IOException { - return checkNext() ? 0xFF & currentBlock.get() : -1; + return checkNext() ? 0xFF & currentBlock[position++] : -1; } @Override @@ -160,15 +148,15 @@ public int read(byte[] b, int off, int len) throws IOException { int copied = 0; while (copied != len) { - int toCopy = Math.min(currentBlock.remaining(), len - copied); - currentBlock.get(b, off, toCopy); + int toCopy = Math.min(currentBlock.length - position, len - copied); + System.arraycopy(currentBlock, position, b, off, toCopy); + position += toCopy; off += toCopy; copied += toCopy; if (!checkNext()) { break; } - } return copied; @@ -205,10 +193,10 @@ public String readString(int byteLength, Charset charset) throws IOException { charset = StandardCharsets.UTF_8; } - if (byteLength > 8 && currentBlock.remaining() > byteLength) { - int pos = currentBlock.position(); - ((Buffer) currentBlock).position(pos + byteLength); - return charset.decode(ByteBuffer.wrap(currentBlock.array(), pos, byteLength)).toString(); + if (currentBlock.length - position > byteLength) { + int offset = position; + position += byteLength; + return new String(currentBlock, offset, byteLength, charset); } return new String(readBytes(byteLength), charset); diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseLZ4OutputStream.java b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseLZ4OutputStream.java index c7da08d7a..4779edaea 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseLZ4OutputStream.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseLZ4OutputStream.java @@ -1,14 +1,17 @@ package com.clickhouse.client.data; -import java.io.DataOutputStream; import java.io.IOException; import java.io.OutputStream; + +import com.clickhouse.client.ClickHouseChecker; + import net.jpountz.lz4.LZ4Compressor; import net.jpountz.lz4.LZ4Factory; public class ClickHouseLZ4OutputStream extends OutputStream { private static final LZ4Factory factory = LZ4Factory.fastestInstance(); - private final DataOutputStream dataWrapper; + + private final OutputStream output; private final LZ4Compressor compressor; private final byte[] currentBlock; @@ -17,23 +20,20 @@ public class ClickHouseLZ4OutputStream extends OutputStream { private int pointer; public ClickHouseLZ4OutputStream(OutputStream stream, int maxCompressBlockSize) { - dataWrapper = new DataOutputStream(stream); + output = ClickHouseChecker.nonNull(stream, "output"); + compressor = factory.fastCompressor(); currentBlock = new byte[maxCompressBlockSize]; - compressedBlock = new byte[compressor.maxCompressedLength(maxCompressBlockSize)]; - } + // reserve the first 9 bytes for calculating checksum + compressedBlock = new byte[compressor.maxCompressedLength(maxCompressBlockSize) + 15]; + compressedBlock[16] = ClickHouseLZ4InputStream.MAGIC; - /** - * @return Location of pointer in the byte buffer (bytes not yet flushed) - */ - public int position() { - return pointer; + pointer = 0; } @Override public void write(int b) throws IOException { - currentBlock[pointer] = (byte) b; - pointer++; + currentBlock[pointer++] = (byte) b; if (pointer == currentBlock.length) { writeBlock(); @@ -72,25 +72,18 @@ public void flush() throws IOException { if (pointer != 0) { writeBlock(); } - dataWrapper.flush(); - } - - private void writeInt(int value) throws IOException { - dataWrapper.write(0xFF & value); - dataWrapper.write(0xFF & (value >> 8)); - dataWrapper.write(0xFF & (value >> 16)); - dataWrapper.write(0xFF & (value >> 24)); + output.flush(); } private void writeBlock() throws IOException { - int compressed = compressor.compress(currentBlock, 0, pointer, compressedBlock, 0); - ClickHouseBlockChecksum checksum = ClickHouseBlockChecksum.calculateForBlock( - (byte) ClickHouseLZ4InputStream.MAGIC, compressed + 9, pointer, compressedBlock, compressed); - dataWrapper.write(checksum.asBytes()); - dataWrapper.writeByte(ClickHouseLZ4InputStream.MAGIC); - writeInt(compressed + 9); // compressed size with header - writeInt(pointer); // uncompressed size - dataWrapper.write(compressedBlock, 0, compressed); + int compressed = compressor.compress(currentBlock, 0, pointer, compressedBlock, 25); + int compressedSizeWithHeader = compressed + 9; + BinaryStreamUtils.setInt32(compressedBlock, 17, compressedSizeWithHeader); // compressed size with header + BinaryStreamUtils.setInt32(compressedBlock, 21, pointer); // uncompressed size + long[] hash = ClickHouseCityHash.cityHash128(compressedBlock, 16, compressedSizeWithHeader); + BinaryStreamUtils.setInt64(compressedBlock, 0, hash[0]); + BinaryStreamUtils.setInt64(compressedBlock, 8, hash[1]); + output.write(compressedBlock, 0, compressed + 25); pointer = 0; } } diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseInputStreamTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseInputStreamTest.java index 5fc50ae32..bdc91f32f 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseInputStreamTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseInputStreamTest.java @@ -89,7 +89,8 @@ public void testWrappedInput() throws IOException { @Test(groups = { "unit" }) public void testNullOrEmptyBlockingInput() throws IOException { - Assert.assertThrows(IllegalArgumentException.class, () -> ClickHouseInputStream.of(null, 0)); + Assert.assertThrows(IllegalArgumentException.class, + () -> ClickHouseInputStream.of((BlockingQueue) null, 0)); Assert.assertThrows(IllegalArgumentException.class, () -> ClickHouseInputStream.of(new ArrayBlockingQueue<>(0), -1)); @@ -180,4 +181,31 @@ public void testBlockingInputAsync() throws IOException { Assert.assertTrue(in.available() == 0, "Should have all bytes read"); Assert.assertTrue(in.isClosed(), "Should have been closed"); } + + @Test(groups = { "unit" }) + public void testSkipInput() throws IOException { + Assert.assertEquals(ClickHouseInputStream.of(generateInputStream(new byte[0])).skip(0L), 0L); + Assert.assertEquals(ClickHouseInputStream.of(generateInputStream(new byte[0])).skip(1L), 0L); + Assert.assertEquals(ClickHouseInputStream.of(generateInputStream(new byte[0])).skip(Long.MAX_VALUE), 0L); + + Assert.assertEquals(ClickHouseInputStream.of(generateInputStream(new byte[1])).skip(0L), 0L); + Assert.assertEquals(ClickHouseInputStream.of(generateInputStream(new byte[1])).skip(1L), 1L); + Assert.assertEquals(ClickHouseInputStream.of(generateInputStream(new byte[1])).skip(Long.MAX_VALUE), 1L); + + Assert.assertEquals(ClickHouseInputStream.of(generateInputStream(new byte[2])).skip(0L), 0L); + Assert.assertEquals(ClickHouseInputStream.of(generateInputStream(new byte[2])).skip(1L), 1L); + Assert.assertEquals(ClickHouseInputStream.of(generateInputStream(new byte[2])).skip(Long.MAX_VALUE), 2L); + + Assert.assertEquals(ClickHouseInputStream.of(generateInputStream(new byte[3]), 2).skip(0L), 0L); + Assert.assertEquals(ClickHouseInputStream.of(generateInputStream(new byte[3]), 2).skip(1L), 1L); + Assert.assertEquals(ClickHouseInputStream.of(generateInputStream(new byte[3]), 2).skip(2L), 2L); + Assert.assertEquals(ClickHouseInputStream.of(generateInputStream(new byte[3]), 2).skip(Long.MAX_VALUE), 3L); + + ClickHouseInputStream in = ClickHouseInputStream.of(new ByteArrayInputStream(new byte[] { 1, 2, 3, 4, 5 }), 2); + Assert.assertEquals(in.read(), 1); + Assert.assertEquals(in.skip(1L), 1L); + Assert.assertEquals(in.read(), 3); + Assert.assertEquals(in.skip(2L), 2L); + Assert.assertEquals(in.read(), -1); + } } 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 1c4f8ef92..087362a05 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 @@ -248,7 +248,7 @@ protected ClickHouseInputStream getResponseInputStream(InputStream in) throws IO } } - return ClickHouseInputStream.of(in); + return ClickHouseInputStream.of(in, config.getMaxBufferSize()); } /** 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 a0607e6d3..01453c747 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 @@ -63,9 +63,8 @@ private ClickHouseHttpResponse buildResponse() throws IOException { : timeZone; } - return new ClickHouseHttpResponse(this, - getResponseInputStream(new BufferedInputStream(conn.getInputStream(), config.getMaxBufferSize())), - displayName, queryId, summary, format, timeZone); + return new ClickHouseHttpResponse(this, getResponseInputStream(conn.getInputStream()), displayName, queryId, + summary, format, timeZone); } private HttpURLConnection newConnection(String url, boolean post) throws IOException { diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseResultSet.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseResultSet.java index 087e4cb15..0d45700ff 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseResultSet.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseResultSet.java @@ -24,6 +24,7 @@ import java.time.LocalDateTime; import java.time.LocalTime; import java.util.Calendar; +import java.util.Collections; import java.util.GregorianCalendar; import java.util.Iterator; import java.util.List; @@ -58,6 +59,8 @@ public class ClickHouseResultSet extends AbstractResultSet { protected final int maxRows; protected final ClickHouseResultSetMetaData metaData; + protected final Map> defaultTypeMap; + // only for testing purpose ClickHouseResultSet(String database, String table, ClickHouseResponse response) { this.database = database; @@ -80,6 +83,7 @@ public class ClickHouseResultSet extends AbstractResultSet { } catch (Exception e) { throw new IllegalStateException(e); } + this.defaultTypeMap = Collections.emptyMap(); this.rowNumber = 0; // before the first row this.lastReadColumn = 0; @@ -115,6 +119,9 @@ public ClickHouseResultSet(String database, String table, ClickHouseStatement st } catch (Exception e) { throw SqlExceptionUtils.handle(e); } + Map> typeMap = conn.getTypeMap(); + this.defaultTypeMap = typeMap != null && !typeMap.isEmpty() ? Collections.unmodifiableMap(typeMap) + : Collections.emptyMap(); this.rowNumber = 0; // before the first row this.lastReadColumn = 0; @@ -449,34 +456,45 @@ public String getNString(String columnLabel) throws SQLException { @Override public Object getObject(int columnIndex) throws SQLException { - if (!wrapObject) { - return getValue(columnIndex).asObject(); - } - - ClickHouseValue v = getValue(columnIndex); - ClickHouseColumn c = columns.get(columnIndex - 1); - if (c.isArray()) { - return new ClickHouseArray(this, columnIndex); - } else if (c.isTuple() || c.isNested()) { - return new ClickHouseStruct(c.getDataType().name(), v.asArray()); - } else { - return v.asObject(); - } + return getObject(columnIndex, defaultTypeMap); } @Override public Object getObject(String columnLabel) throws SQLException { - return getObject(findColumn(columnLabel)); + return getObject(findColumn(columnLabel), defaultTypeMap); } @Override public Object getObject(int columnIndex, Map> map) throws SQLException { - return getObject(columnIndex); + if (map == null) { + map = defaultTypeMap; + } + + ClickHouseValue v = getValue(columnIndex); + ClickHouseColumn c = columns.get(columnIndex - 1); + + Class javaType = null; + if (!map.isEmpty() && (javaType = map.get(c.getOriginalTypeName())) == null) { + javaType = map.get(c.getDataType().name()); + } + + Object value; + if (!wrapObject) { + value = javaType != null ? v.asObject(javaType) : v.asObject(); + } else if (c.isArray()) { + value = new ClickHouseArray(this, columnIndex); + } else if (c.isTuple() || c.isNested()) { + value = new ClickHouseStruct(c.getDataType().name(), v.asArray()); + } else { + value = javaType != null ? v.asObject(javaType) : v.asObject(); + } + + return value; } @Override public Object getObject(String columnLabel, Map> map) throws SQLException { - return getObject(columnLabel); + return getObject(findColumn(columnLabel), map); } @Override From 6b7d1f69de0c8821634bf694ea6a464e75471b14 Mon Sep 17 00:00:00 2001 From: Zhichun Wu Date: Tue, 21 Dec 2021 22:46:04 +0800 Subject: [PATCH 5/6] Better support of Bool, Enum, and custom type mapping --- .../clickhouse/client/ClickHouseColumn.java | 122 ++++--- .../clickhouse/client/ClickHouseDataType.java | 3 +- .../com/clickhouse/client/ClickHouseEnum.java | 135 ++++++++ .../clickhouse/client/ClickHouseValue.java | 53 ++- .../client/data/ClickHouseBoolValue.java | 267 +++++++++++++++ .../client/data/ClickHouseEnumValue.java | 305 +++++++++++------- .../client/data/ClickHouseObjectValue.java | 5 - .../data/ClickHouseRowBinaryProcessor.java | 19 +- .../client/ClickHouseColumnTest.java | 21 ++ .../client/ClickHouseDataTypeTest.java | 2 +- .../client/ClickHouseValuesTest.java | 2 +- .../client/data/ClickHouseEnumValueTest.java | 26 +- .../client/grpc/ClickHouseGrpcClientTest.java | 6 +- .../com/clickhouse/jdbc/ClickHouseArray.java | 3 +- .../jdbc/ClickHouseDatabaseMetaData.java | 29 +- .../clickhouse/jdbc/ClickHouseResultSet.java | 12 +- .../jdbc/ClickHouseResultSetMetaData.java | 13 +- .../java/com/clickhouse/jdbc/JdbcConfig.java | 57 ++++ .../com/clickhouse/jdbc/JdbcTypeMapping.java | 73 ++++- .../internal/ClickHouseConnectionImpl.java | 5 +- .../internal/ClickHouseJdbcUrlParser.java | 5 +- .../internal/ClickHouseParameterMetaData.java | 2 +- .../internal/ClickHouseStatementImpl.java | 29 +- .../clickhouse/domain/ClickHouseDataType.java | 145 ++++----- .../jdbc/ClickHouseStatementTest.java | 33 +- .../clickhouse/jdbc/JdbcIntegrationTest.java | 3 +- .../jdbc/parser/ClickHouseSqlParserTest.java | 200 ++++++++---- 27 files changed, 1183 insertions(+), 392 deletions(-) create mode 100644 clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseEnum.java create mode 100644 clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseBoolValue.java diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseColumn.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseColumn.java index 72baac450..f50a3ea11 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseColumn.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseColumn.java @@ -36,69 +36,76 @@ public final class ClickHouseColumn implements Serializable { private int scale; private List nested; private List parameters; + private ClickHouseEnum enumConstants; private int arrayLevel; private ClickHouseColumn arrayBaseColumn; private static ClickHouseColumn update(ClickHouseColumn column) { + column.enumConstants = ClickHouseEnum.EMPTY; int size = column.parameters.size(); switch (column.dataType) { - case Array: - column.arrayLevel = 1; - column.arrayBaseColumn = column.nested.get(0); - while (column.arrayLevel < 255) { - if (column.arrayBaseColumn.dataType == ClickHouseDataType.Array) { - column.arrayLevel++; - column.arrayBaseColumn = column.arrayBaseColumn.nested.get(0); - } else { - break; + case Array: + column.arrayLevel = 1; + column.arrayBaseColumn = column.nested.get(0); + while (column.arrayLevel < 255) { + if (column.arrayBaseColumn.dataType == ClickHouseDataType.Array) { + column.arrayLevel++; + column.arrayBaseColumn = column.arrayBaseColumn.nested.get(0); + } else { + break; + } } - } - break; - case DateTime: - if (size >= 2) { // same as DateTime64 - column.scale = Integer.parseInt(column.parameters.get(0)); - column.timeZone = TimeZone.getTimeZone(column.parameters.get(1).replace("'", "")); - } else if (size == 1) { // same as DateTime32 - // unfortunately this will fall back to GMT if the time zone - // cannot be resolved - TimeZone tz = TimeZone.getTimeZone(column.parameters.get(0).replace("'", "")); - column.timeZone = tz; - } - break; - case DateTime32: - if (size > 0) { - // unfortunately this will fall back to GMT if the time zone - // cannot be resolved - TimeZone tz = TimeZone.getTimeZone(column.parameters.get(0).replace("'", "")); - column.timeZone = tz; - } - break; - case DateTime64: - if (size > 0) { + break; + case Enum: + case Enum8: + case Enum16: + column.enumConstants = new ClickHouseEnum(column.parameters); + break; + case DateTime: + if (size >= 2) { // same as DateTime64 + column.scale = Integer.parseInt(column.parameters.get(0)); + column.timeZone = TimeZone.getTimeZone(column.parameters.get(1).replace("'", "")); + } else if (size == 1) { // same as DateTime32 + // unfortunately this will fall back to GMT if the time zone + // cannot be resolved + TimeZone tz = TimeZone.getTimeZone(column.parameters.get(0).replace("'", "")); + column.timeZone = tz; + } + break; + case DateTime32: + if (size > 0) { + // unfortunately this will fall back to GMT if the time zone + // cannot be resolved + TimeZone tz = TimeZone.getTimeZone(column.parameters.get(0).replace("'", "")); + column.timeZone = tz; + } + break; + case DateTime64: + if (size > 0) { + column.scale = Integer.parseInt(column.parameters.get(0)); + } + if (size > 1) { + column.timeZone = TimeZone.getTimeZone(column.parameters.get(1).replace("'", "")); + } + break; + case Decimal: + if (size >= 2) { + column.precision = Integer.parseInt(column.parameters.get(0)); + column.scale = Integer.parseInt(column.parameters.get(1)); + } + break; + case Decimal32: + case Decimal64: + case Decimal128: + case Decimal256: column.scale = Integer.parseInt(column.parameters.get(0)); - } - if (size > 1) { - column.timeZone = TimeZone.getTimeZone(column.parameters.get(1).replace("'", "")); - } - break; - case Decimal: - if (size >= 2) { + break; + case FixedString: column.precision = Integer.parseInt(column.parameters.get(0)); - column.scale = Integer.parseInt(column.parameters.get(1)); - } - break; - case Decimal32: - case Decimal64: - case Decimal128: - case Decimal256: - column.scale = Integer.parseInt(column.parameters.get(0)); - break; - case FixedString: - column.precision = Integer.parseInt(column.parameters.get(0)); - break; - default: - break; + break; + default: + break; } return column; @@ -395,6 +402,11 @@ public boolean isArray() { return dataType == ClickHouseDataType.Array; } + public boolean isEnum() { + return dataType == ClickHouseDataType.Enum || dataType == ClickHouseDataType.Enum8 + || dataType == ClickHouseDataType.Enum16; + } + public boolean isMap() { return dataType == ClickHouseDataType.Map; } @@ -419,6 +431,10 @@ public ClickHouseDataType getDataType() { return dataType; } + public ClickHouseEnum getEnumConstants() { + return enumConstants; + } + public String getOriginalTypeName() { return originalTypeName; } diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDataType.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDataType.java index f99b771c9..e34c94981 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDataType.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseDataType.java @@ -45,13 +45,14 @@ public enum ClickHouseDataType { UInt64(Long.class, false, true, false, 8, 20, 0, 0, 0, "BIGINT UNSIGNED"), UInt128(BigInteger.class, false, true, false, 16, 39, 0, 0, 0), UInt256(BigInteger.class, false, true, false, 32, 78, 0, 0, 0), Int8(Byte.class, false, true, true, 1, 3, 0, 0, 0, - "BOOL", "BOOLEAN", "BYTE", "INT1", "INT1 SIGNED", "TINYINT", "TINYINT SIGNED"), + "BYTE", "INT1", "INT1 SIGNED", "TINYINT", "TINYINT SIGNED"), Int16(Short.class, false, true, true, 2, 5, 0, 0, 0, "SMALLINT", "SMALLINT SIGNED"), Int32(Integer.class, false, true, true, 4, 10, 0, 0, 0, "INT", "INTEGER", "MEDIUMINT", "INT SIGNED", "INTEGER SIGNED", "MEDIUMINT SIGNED"), Int64(Long.class, false, true, true, 8, 19, 0, 0, 0, "BIGINT", "BIGINT SIGNED"), Int128(BigInteger.class, false, true, true, 16, 39, 0, 0, 0), Int256(BigInteger.class, false, true, true, 32, 77, 0, 0, 0), + Bool(Boolean.class, false, false, true, 1, 3, 0, 0, 0, "BOOLEAN"), Date(LocalDate.class, false, false, false, 2, 10, 0, 0, 0), Date32(LocalDate.class, false, false, false, 4, 10, 0, 0, 0), DateTime(LocalDateTime.class, true, false, false, 0, 29, 0, 0, 9, "TIMESTAMP"), diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseEnum.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseEnum.java new file mode 100644 index 000000000..daa10839a --- /dev/null +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseEnum.java @@ -0,0 +1,135 @@ +package com.clickhouse.client; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; + +public class ClickHouseEnum { + public static final ClickHouseEnum EMPTY = new ClickHouseEnum(Collections.emptyList()); + + public static ClickHouseEnum of(Class clazz) { + if (clazz == null || !Enum.class.isAssignableFrom(clazz)) { + return EMPTY; + } + + Enum[] constants = clazz.getEnumConstants(); + int size = constants.length; + String[] names = new String[size]; + int[] values = new int[size]; + int i = 0; + for (Enum e : clazz.getEnumConstants()) { + names[i] = e.name(); + values[i] = e.ordinal(); + i++; + } + + return new ClickHouseEnum(names, values); + } + + private final int size; + private final String[] names; + private final int[] values; + + protected ClickHouseEnum(Collection params) { + size = params.size(); + names = new String[size]; + values = new int[size]; + + int i = 0; + for (String p : params) { + int index = p.lastIndexOf('='); + if (index > 0) { + names[i] = ClickHouseUtils.unescape(p.substring(0, index)); + values[i] = Integer.parseInt(p.substring(index + 1)); + } else { + throw new IllegalArgumentException("Invalid enum entry: " + p); + } + i++; + } + } + + protected ClickHouseEnum(String[] names, int[] values) { + if (names == null || values == null) { + throw new IllegalArgumentException("Non-null names and values are required"); + } else if (names.length != values.length) { + throw new IllegalArgumentException("Names and values should have same length"); + } + + this.size = names.length; + this.names = names; + this.values = values; + } + + public String validate(String name) { + for (int i = 0; i < size; i++) { + if (names[i].equals(name)) { + return name; + } + } + + throw new IllegalArgumentException("Unknown enum name: " + name); + } + + public int validate(int value) { + for (int i = 0; i < size; i++) { + if (values[i] == value) { + return value; + } + } + + throw new IllegalArgumentException("Unknown enum value: " + value); + } + + public String name(int value) { + for (int i = 0; i < size; i++) { + if (values[i] == value) { + return names[i]; + } + } + + throw new IllegalArgumentException("Unknown enum value: " + value); + } + + public int value(String name) { + for (int i = 0; i < size; i++) { + if (names[i].equals(name)) { + return values[i]; + } + } + + throw new IllegalArgumentException("Unknown enum name: " + name); + } + + @Override + public int hashCode() { + final int prime = 31; + int result = prime + size; + result = prime * result + Arrays.hashCode(names); + result = prime * result + Arrays.hashCode(values); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + ClickHouseEnum other = (ClickHouseEnum) obj; + return size == other.size && Arrays.equals(names, other.names) && Arrays.equals(values, other.values); + } + + public String toSqlException() { + StringBuilder builder = new StringBuilder(); + for (int i = 0; i < size; i++) { + builder.append('\'').append(ClickHouseUtils.escape(names[i], '\'')).append('\'').append('=') + .append(values[i]).append(','); + } + if (builder.length() > 0) { + builder.setLength(builder.length() - 1); + } + return builder.toString(); + } +} diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseValue.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseValue.java index c2a6a1fd3..7c39aea47 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseValue.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseValue.java @@ -454,11 +454,60 @@ default Map asMap(Class keyClass, Class valueClass) { * Gets value as a typed object. * * @param type of the object + * @param type of the enum * @param clazz class of the object * @return a typed object representing the value, could be null */ - default T asObject(Class clazz) { - return isNullOrEmpty() ? null : ClickHouseChecker.nonNull(clazz, ClickHouseValues.TYPE_CLASS).cast(asObject()); + default > T asObject(Class clazz) { + if (clazz == null) { + return null; + } else if (clazz == boolean.class || clazz == Boolean.class) { + return clazz.cast(asBoolean()); + } else if (clazz == byte.class || clazz == Byte.class) { + return clazz.cast(asByte()); + } else if (clazz == char.class || clazz == Character.class) { + return clazz.cast(asCharacter()); + } else if (clazz == short.class || clazz == Short.class) { + return clazz.cast(asShort()); + } else if (clazz == int.class || clazz == Integer.class) { + return clazz.cast(asInteger()); + } else if (clazz == long.class || clazz == Long.class) { + return clazz.cast(asLong()); + } else if (clazz == float.class || clazz == Float.class) { + return clazz.cast(asFloat()); + } else if (clazz == double.class || clazz == Double.class) { + return clazz.cast(asDouble()); + } else if (clazz == String.class) { + return clazz.cast(asString()); + } else if (clazz == LocalDate.class) { + return clazz.cast(asDate()); + } else if (clazz == LocalDateTime.class) { + return clazz.cast(asDateTime()); + } else if (clazz == OffsetDateTime.class) { + return clazz.cast(asOffsetDateTime()); + } else if (clazz == ZonedDateTime.class) { + return clazz.cast(asZonedDateTime()); + } else if (clazz == LocalTime.class) { + return clazz.cast(asTime()); + } else if (clazz == BigInteger.class) { + return clazz.cast(asBigInteger()); + } else if (clazz == BigDecimal.class) { + return clazz.cast(asBigDecimal()); + } else if (clazz == Inet4Address.class) { + return clazz.cast(asInet4Address()); + } else if (clazz == Inet6Address.class) { + return clazz.cast(asInet6Address()); + } else if (clazz == UUID.class) { + return clazz.cast(asUuid()); + } else if (Array.class.isAssignableFrom(clazz)) { + return clazz.cast(asArray()); + } else if (List.class.isAssignableFrom(clazz)) { + return clazz.cast(asTuple()); + } else if (Enum.class.isAssignableFrom(clazz)) { + return clazz.cast(asEnum((Class) clazz)); + } else { + return clazz.cast(asObject()); + } } /** diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseBoolValue.java b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseBoolValue.java new file mode 100644 index 000000000..5034670f2 --- /dev/null +++ b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseBoolValue.java @@ -0,0 +1,267 @@ +package com.clickhouse.client.data; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; +import com.clickhouse.client.ClickHouseChecker; +import com.clickhouse.client.ClickHouseValue; +import com.clickhouse.client.ClickHouseValues; + +/** + * Wraper class of bool. + */ +public class ClickHouseBoolValue implements ClickHouseValue { + /** + * Create a new instance representing null value. + * + * @return new instance representing null value + */ + public static ClickHouseBoolValue ofNull() { + return ofNull(null); + } + + /** + * Update given value to null or create a new instance if {@code ref} is null. + * + * @param ref object to update, could be null + * @return same object as {@code ref} or a new instance if it's null + */ + public static ClickHouseBoolValue ofNull(ClickHouseValue ref) { + return ref instanceof ClickHouseBoolValue ? ((ClickHouseBoolValue) ref).set(true, false) + : new ClickHouseBoolValue(true, false); + } + + /** + * Wrap the given value. + * + * @param value value + * @return object representing the value + */ + public static ClickHouseBoolValue of(boolean value) { + return of(null, value); + } + + /** + * Wrap the given value. + * + * @param value value + * @return object representing the value + */ + public static ClickHouseBoolValue of(int value) { + return of(null, value == 1); + } + + /** + * Update value of the given object or create a new instance if {@code ref} is + * null. + * + * @param ref object to update, could be null + * @param value value + * @return same object as {@code ref} or a new instance if it's null + */ + public static ClickHouseBoolValue of(ClickHouseValue ref, boolean value) { + return ref instanceof ClickHouseBoolValue ? ((ClickHouseBoolValue) ref).set(false, value) + : new ClickHouseBoolValue(false, value); + } + + private boolean isNull; + private boolean value; + + protected ClickHouseBoolValue(boolean isNull, boolean value) { + set(isNull, value); + } + + protected ClickHouseBoolValue set(boolean isNull, boolean value) { + this.isNull = isNull; + this.value = !isNull && value; + + return this; + } + + /** + * Gets value. + * + * @return value + */ + public boolean getValue() { + return value; + } + + @Override + public ClickHouseBoolValue copy(boolean deep) { + return new ClickHouseBoolValue(isNull, value); + } + + @Override + public boolean isNullOrEmpty() { + return isNull; + } + + @Override + public byte asByte() { + return value ? (byte) 1 : (byte) 0; + } + + @Override + public short asShort() { + return value ? (short) 1 : (short) 0; + } + + @Override + public int asInteger() { + return value ? 1 : 0; + } + + @Override + public long asLong() { + return value ? 1L : 0L; + } + + @Override + public BigInteger asBigInteger() { + return isNull ? null : (value ? BigInteger.ONE : BigInteger.ZERO); + } + + @Override + public float asFloat() { + return value ? 1F : 0F; + } + + @Override + public double asDouble() { + return value ? 1D : 0D; + } + + @Override + public BigDecimal asBigDecimal(int scale) { + return isNull ? null : (value ? BigDecimal.ONE : BigDecimal.ZERO); + } + + @Override + public Object asObject() { + return isNull ? null : Boolean.valueOf(value); + } + + @Override + public String asString(int length, Charset charset) { + if (isNull) { + return null; + } + + String str = String.valueOf(value); + if (length > 0) { + ClickHouseChecker.notWithDifferentLength(str.getBytes(charset == null ? StandardCharsets.UTF_8 : charset), + length); + } + + return str; + } + + @Override + public ClickHouseBoolValue resetToNullOrEmpty() { + return set(true, false); + } + + @Override + public String toSqlExpression() { + return isNull ? ClickHouseValues.NULL_EXPR : String.valueOf(value ? 1 : 0); + } + + @Override + public ClickHouseBoolValue update(char value) { + return set(false, value == 1); + } + + @Override + public ClickHouseBoolValue update(byte value) { + return set(false, value == (byte) 1); + } + + @Override + public ClickHouseBoolValue update(short value) { + return set(false, value == (short) 1); + } + + @Override + public ClickHouseBoolValue update(int value) { + return set(false, value == 1); + } + + @Override + public ClickHouseBoolValue update(long value) { + return set(false, value == 1L); + } + + @Override + public ClickHouseBoolValue update(float value) { + return set(false, value == 1F); + } + + @Override + public ClickHouseBoolValue update(double value) { + return set(false, value == 1D); + } + + @Override + public ClickHouseBoolValue update(BigInteger value) { + return value == null ? resetToNullOrEmpty() : set(false, BigInteger.ONE.equals(value)); + } + + @Override + public ClickHouseBoolValue update(BigDecimal value) { + return value == null ? resetToNullOrEmpty() : set(false, BigDecimal.ONE.equals(value)); + } + + @Override + public ClickHouseBoolValue update(Enum value) { + return value == null ? resetToNullOrEmpty() : set(false, value.ordinal() == 1); + } + + @Override + public ClickHouseBoolValue update(String value) { + return value == null ? resetToNullOrEmpty() : set(false, Boolean.parseBoolean(value) || "1".equals(value)); + } + + @Override + public ClickHouseBoolValue update(ClickHouseValue value) { + return value == null ? resetToNullOrEmpty() : set(false, value.asBoolean()); + } + + @Override + public ClickHouseBoolValue update(Object value) { + if (value instanceof Boolean) { + return set(false, (boolean) value); + } else if (value instanceof Number) { + return set(false, ((Number) value).byteValue() == (byte) 0); + } else if (value instanceof ClickHouseValue) { + return set(false, ((ClickHouseValue) value).asBoolean()); + } + + ClickHouseValue.super.update(value); + return this; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { // too bad this is a mutable class :< + return true; + } else if (obj == null || getClass() != obj.getClass()) { + return false; + } + + ClickHouseBoolValue v = (ClickHouseBoolValue) obj; + return isNull == v.isNull && value == v.value; + } + + @Override + public int hashCode() { + // not going to use Objects.hash(isNull, value) due to autoboxing + return (31 + (isNull ? 1231 : 1237)) * 31 + (value ? 1231 : 1237); + } + + @Override + public String toString() { + return ClickHouseValues.convertToString(this); + } +} diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseEnumValue.java b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseEnumValue.java index 0b32b93b9..93386febf 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseEnumValue.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseEnumValue.java @@ -5,132 +5,220 @@ import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; import com.clickhouse.client.ClickHouseChecker; +import com.clickhouse.client.ClickHouseEnum; +import com.clickhouse.client.ClickHouseUtils; import com.clickhouse.client.ClickHouseValue; import com.clickhouse.client.ClickHouseValues; /** * Wraper class of enum. */ -public class ClickHouseEnumValue> extends ClickHouseObjectValue { - static final String ERROR_NO_ENUM_TYPE = "Failed to convert value due to lack of enum type: "; +public class ClickHouseEnumValue implements ClickHouseValue { + /** + * Create a new instance representing null value. + * + * @param clazz enum class + * @return new instance representing null value + */ + public static ClickHouseEnumValue ofNull(Class clazz) { + return ofNull(null, ClickHouseEnum.of(clazz)); + } /** * Create a new instance representing null value. * - * @param enum type + * @param type enum type, null is same as {@link ClickHouseEnum#EMPTY} * @return new instance representing null value */ - public static > ClickHouseEnumValue ofNull() { - return ofNull(null); + public static ClickHouseEnumValue ofNull(ClickHouseEnum type) { + return ofNull(null, type); } /** * Update given value to null or create a new instance if {@code ref} is null. * - * @param enum type - * @param ref object to update, could be null + * @param ref object to update, could be null + * @param clazz enum class * @return same object as {@code ref} or a new instance if it's null */ - @SuppressWarnings("unchecked") - public static > ClickHouseEnumValue ofNull(ClickHouseValue ref) { - return ref instanceof ClickHouseEnumValue ? (ClickHouseEnumValue) ((ClickHouseEnumValue) ref).set(null) - : new ClickHouseEnumValue<>(null); + public static ClickHouseEnumValue ofNull(ClickHouseValue ref, Class clazz) { + return ref instanceof ClickHouseEnumValue ? ((ClickHouseEnumValue) ref).set(true, 0) + : new ClickHouseEnumValue(ClickHouseEnum.of(clazz), true, 0); + } + + /** + * Update given value to null or create a new instance if {@code ref} is null. + * + * @param ref object to update, could be null + * @param type enum type, null is same as {@link ClickHouseEnum#EMPTY} + * @return same object as {@code ref} or a new instance if it's null + */ + public static ClickHouseEnumValue ofNull(ClickHouseValue ref, ClickHouseEnum type) { + return ref instanceof ClickHouseEnumValue ? ((ClickHouseEnumValue) ref).set(true, 0) + : new ClickHouseEnumValue(type, true, 0); } /** * Wrap the given value. * - * @param enum type * @param value value * @return object representing the value */ - public static > ClickHouseEnumValue of(T value) { + public static ClickHouseEnumValue of(Enum value) { return of(null, value); } + /** + * Wrap the given value. + * + * @param value value + * @param type enum type + * @return object representing the value + */ + public static ClickHouseEnumValue of(ClickHouseEnum type, int value) { + return of(null, type, value); + } + + /** + * Wrap the given value. + * + * @param value value + * @param type enum type + * @return object representing the value + */ + public static ClickHouseEnumValue of(ClickHouseEnum type, Number value) { + return value == null ? ofNull(null, type) : of(null, type, value.intValue()); + } + /** * Update value of the given object or create a new instance if {@code ref} is * null. * - * @param enum type * @param ref object to update, could be null * @param value value * @return same object as {@code ref} or a new instance if it's null */ - @SuppressWarnings("unchecked") - public static > ClickHouseEnumValue of(ClickHouseValue ref, T value) { - return ref instanceof ClickHouseEnumValue - ? (ClickHouseEnumValue) ((ClickHouseEnumValue) ref).update(value) - : new ClickHouseEnumValue<>(value); + public static ClickHouseEnumValue of(ClickHouseValue ref, Enum value) { + ClickHouseEnumValue v; + if (ref instanceof ClickHouseEnumValue) { + v = (ClickHouseEnumValue) ref; + if (value != null) { + v.set(false, value.ordinal()); + } else { + v.resetToNullOrEmpty(); + } + } else { + if (value != null) { + v = new ClickHouseEnumValue(ClickHouseEnum.of(value.getClass()), false, value.ordinal()); + } else { + v = new ClickHouseEnumValue(ClickHouseEnum.EMPTY, true, 0); + } + } + return v; } - protected ClickHouseEnumValue(T value) { - super(value); + /** + * Update value of the given object or create a new instance if {@code ref} is + * null. + * + * @param ref object to update, could be null + * @param type enum type, null is same as {@link ClickHouseEnum#EMPTY} + * @param value value + * @return same object as {@code ref} or a new instance if it's null + */ + public static ClickHouseEnumValue of(ClickHouseValue ref, ClickHouseEnum type, int value) { + return ref instanceof ClickHouseEnumValue ? ((ClickHouseEnumValue) ref).set(false, value) + : new ClickHouseEnumValue(type, false, value); + } + + private final ClickHouseEnum type; + + private boolean isNull; + private int value; + + protected ClickHouseEnumValue(ClickHouseEnum type, boolean isNull, int value) { + this.type = type != null ? type : ClickHouseEnum.EMPTY; + + set(isNull, value); + } + + protected ClickHouseEnumValue set(boolean isNull, int value) { + this.isNull = isNull; + this.value = isNull ? 0 : type.validate(value); + return this; + } + + /** + * Gets value. + * + * @return value + */ + public int getValue() { + return value; + } + + @Override + public ClickHouseEnumValue copy(boolean deep) { + return new ClickHouseEnumValue(type, isNull, value); } @Override - public ClickHouseEnumValue copy(boolean deep) { - return new ClickHouseEnumValue<>(getValue()); + public boolean isNullOrEmpty() { + return isNull; } @Override public byte asByte() { - return isNullOrEmpty() ? (byte) 0 : (byte) getValue().ordinal(); + return (byte) value; } @Override public short asShort() { - return isNullOrEmpty() ? (short) 0 : (short) getValue().ordinal(); + return (short) value; } @Override public int asInteger() { - return isNullOrEmpty() ? 0 : getValue().ordinal(); + return value; } @Override public long asLong() { - return isNullOrEmpty() ? 0L : getValue().ordinal(); + return value; } @Override public BigInteger asBigInteger() { - return isNullOrEmpty() ? null : BigInteger.valueOf(getValue().ordinal()); + return isNull ? null : BigInteger.valueOf(value); } @Override public float asFloat() { - return isNullOrEmpty() ? 0F : getValue().ordinal(); + return value; } @Override public double asDouble() { - return isNullOrEmpty() ? 0D : getValue().ordinal(); + return value; } @Override public BigDecimal asBigDecimal(int scale) { - return isNullOrEmpty() ? null : BigDecimal.valueOf(getValue().ordinal(), scale); - } - - @Override - @SuppressWarnings("unchecked") - public > E asEnum(Class enumType) { - return (E) getValue(); + return isNull ? null : BigDecimal.valueOf(value, scale); } @Override public Object asObject() { - return getValue(); + return isNull ? null : type.name(value); } @Override public String asString(int length, Charset charset) { - if (isNullOrEmpty()) { + if (isNull) { return null; } - String str = String.valueOf(getValue().name()); + String str = type.name(value); if (length > 0) { ClickHouseChecker.notWithDifferentLength(str.getBytes(charset == null ? StandardCharsets.UTF_8 : charset), length); @@ -140,130 +228,115 @@ public String asString(int length, Charset charset) { } @Override - public String toSqlExpression() { - return ClickHouseValues.convertToQuotedString(asString(0, null)); + public ClickHouseEnumValue resetToNullOrEmpty() { + return set(true, (byte) 0); } @Override - public ClickHouseEnumValue update(boolean value) { - return update(value ? 1 : 0); + public String toSqlExpression() { + return isNull ? ClickHouseValues.NULL_EXPR + : new StringBuilder().append('\'').append(ClickHouseUtils.escape(type.name(value), '\'')).append('\'') + .toString(); } @Override - public ClickHouseEnumValue update(char value) { - return update((int) value); + public ClickHouseEnumValue update(char value) { + return set(false, value); } @Override - public ClickHouseEnumValue update(byte value) { - return update((int) value); + public ClickHouseEnumValue update(byte value) { + return set(false, value); } @Override - public ClickHouseEnumValue update(short value) { - return update((int) value); + public ClickHouseEnumValue update(short value) { + return set(false, value); } @Override - @SuppressWarnings("unchecked") - public ClickHouseEnumValue update(int value) { - if (isNullOrEmpty()) { - throw new IllegalArgumentException(ERROR_NO_ENUM_TYPE + value); - } - - Class clazz = (Class) getValue().getClass(); - for (T t : clazz.getEnumConstants()) { - if (t.ordinal() == value) { - return update(t); - } - } - - throw new IllegalArgumentException(); + public ClickHouseEnumValue update(int value) { + return set(false, value); } @Override - public ClickHouseEnumValue update(long value) { - return update((int) value); + public ClickHouseEnumValue update(long value) { + return set(false, (int) value); } @Override - public ClickHouseEnumValue update(float value) { - return update((int) value); + public ClickHouseEnumValue update(float value) { + return set(false, (int) value); } @Override - public ClickHouseEnumValue update(double value) { - return update((int) value); + public ClickHouseEnumValue update(double value) { + return set(false, (int) value); } @Override - public ClickHouseEnumValue update(BigInteger value) { - if (value == null) { - resetToNullOrEmpty(); - return this; - } + public ClickHouseEnumValue update(BigInteger value) { + return value == null ? resetToNullOrEmpty() : set(false, value.intValueExact()); + } - return update(value.intValueExact()); + @Override + public ClickHouseEnumValue update(BigDecimal value) { + return value == null ? resetToNullOrEmpty() : set(false, value.intValueExact()); } @Override - public ClickHouseEnumValue update(BigDecimal value) { - if (value == null) { - resetToNullOrEmpty(); - return this; - } + public ClickHouseEnumValue update(Enum value) { + return value == null ? resetToNullOrEmpty() : set(false, value.ordinal()); + } - return update(value.intValueExact()); + @Override + public ClickHouseEnumValue update(String value) { + return value == null ? resetToNullOrEmpty() : set(false, type.value(value)); } @Override - @SuppressWarnings("unchecked") - public ClickHouseEnumValue update(Enum value) { - set((T) value); - return this; + public ClickHouseEnumValue update(ClickHouseValue value) { + return value == null ? resetToNullOrEmpty() : set(false, value.asInteger()); } @Override - @SuppressWarnings("unchecked") - public ClickHouseEnumValue update(ClickHouseValue value) { - if (value == null || value.isNullOrEmpty()) { - resetToNullOrEmpty(); - } else if (value instanceof ClickHouseEnumValue) { - set(((ClickHouseEnumValue) value).getValue()); - } else if (isNullOrEmpty()) { - throw new IllegalArgumentException(ERROR_NO_ENUM_TYPE + value); - } else { - set(value.asEnum(isNullOrEmpty() ? null : (Class) getValue().getClass())); + public ClickHouseEnumValue update(Object value) { + if (value instanceof Number) { + return set(false, ((Number) value).intValue()); + } else if (value instanceof String) { + return set(false, type.value((String) value)); + } else if (value instanceof ClickHouseValue) { + return set(false, ((ClickHouseValue) value).asInteger()); } + + ClickHouseValue.super.update(value); return this; } @Override - @SuppressWarnings("unchecked") - public ClickHouseEnumValue update(String value) { - if (value == null) { - resetToNullOrEmpty(); - } else if (isNullOrEmpty()) { - throw new IllegalArgumentException(ERROR_NO_ENUM_TYPE + value); - } else { - set((T) Enum.valueOf(getValue().getClass(), value)); + public boolean equals(Object obj) { + if (this == obj) { // too bad this is a mutable class :< + return true; + } else if (obj == null || getClass() != obj.getClass()) { + return false; } - return this; + ClickHouseEnumValue v = (ClickHouseEnumValue) obj; + return isNull == v.isNull && value == v.value && type.equals(v.type); } @Override - @SuppressWarnings("unchecked") - public ClickHouseEnumValue update(Object value) { - if (value instanceof Enum) { - set((T) value); - return this; - } else if (value instanceof ClickHouseEnumValue) { - set(((ClickHouseEnumValue) value).getValue()); - return this; - } + public int hashCode() { + // not going to use Objects.hash(isNull, value) due to autoboxing + final int prime = 31; + int result = prime + (isNull ? 1231 : 1237); + result = prime * result + value; + result = prime * result + type.hashCode(); + return result; + } - super.update(value); - return this; + @Override + public String toString() { + return ClickHouseValues.convertToString(this); } } diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseObjectValue.java b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseObjectValue.java index 8fd94c8ff..45fe395c6 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseObjectValue.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseObjectValue.java @@ -114,11 +114,6 @@ public Object asObject() { return getValue(); } - @Override - public E asObject(Class clazz) { - return ClickHouseChecker.nonNull(clazz, ClickHouseValues.TYPE_CLASS).cast(getValue()); - } - @Override public String asString(int length, Charset charset) { if (isNullOrEmpty()) { diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseRowBinaryProcessor.java b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseRowBinaryProcessor.java index 31a0aadb4..336653c42 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseRowBinaryProcessor.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseRowBinaryProcessor.java @@ -163,18 +163,27 @@ private MappedFunctions() { deserializers = new EnumMap<>(ClickHouseDataType.class); serializers = new EnumMap<>(ClickHouseDataType.class); - // enum and numbers + // enums buildMappings(deserializers, serializers, - (r, f, c, i) -> ClickHouseByteValue.of(r, BinaryStreamUtils.readInt8(i)), + (r, f, c, i) -> ClickHouseEnumValue.of(r, c.getEnumConstants(), BinaryStreamUtils.readInt8(i)), (v, f, c, o) -> BinaryStreamUtils.writeInt8(o, v.asByte()), ClickHouseDataType.Enum, - ClickHouseDataType.Enum8, ClickHouseDataType.Int8); + ClickHouseDataType.Enum8); + buildMappings(deserializers, serializers, + (r, f, c, i) -> ClickHouseEnumValue.of(r, c.getEnumConstants(), BinaryStreamUtils.readInt16(i)), + (v, f, c, o) -> BinaryStreamUtils.writeInt16(o, v.asShort()), ClickHouseDataType.Enum16); + // bool and numbers + buildMappings(deserializers, serializers, + (r, f, c, i) -> ClickHouseBoolValue.of(r, BinaryStreamUtils.readBoolean(i)), + (v, f, c, o) -> BinaryStreamUtils.writeBoolean(o, v.asBoolean()), ClickHouseDataType.Bool); + buildMappings(deserializers, serializers, + (r, f, c, i) -> ClickHouseByteValue.of(r, BinaryStreamUtils.readInt8(i)), + (v, f, c, o) -> BinaryStreamUtils.writeInt8(o, v.asByte()), ClickHouseDataType.Int8); buildMappings(deserializers, serializers, (r, f, c, i) -> ClickHouseShortValue.of(r, BinaryStreamUtils.readUnsignedInt8(i)), (v, f, c, o) -> BinaryStreamUtils.writeUnsignedInt8(o, v.asInteger()), ClickHouseDataType.UInt8); buildMappings(deserializers, serializers, (r, f, c, i) -> ClickHouseShortValue.of(r, BinaryStreamUtils.readInt16(i)), - (v, f, c, o) -> BinaryStreamUtils.writeInt16(o, v.asShort()), ClickHouseDataType.Enum16, - ClickHouseDataType.Int16); + (v, f, c, o) -> BinaryStreamUtils.writeInt16(o, v.asShort()), ClickHouseDataType.Int16); buildMappings(deserializers, serializers, (r, f, c, i) -> ClickHouseIntegerValue.of(r, BinaryStreamUtils.readUnsignedInt16(i)), (v, f, c, o) -> BinaryStreamUtils.writeUnsignedInt16(o, v.asInteger()), ClickHouseDataType.UInt16); diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseColumnTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseColumnTest.java index 105b4806e..b5a78c599 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseColumnTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseColumnTest.java @@ -4,9 +4,14 @@ import java.util.LinkedList; import java.util.List; import org.testng.Assert; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; public class ClickHouseColumnTest { + @DataProvider(name = "enumTypesProvider") + private Object[][] getEnumTypes() { + return new Object[][] { { "Enum" }, { "Enum8" }, { "Enum16" } }; + } @Test(groups = { "unit" }) public void testReadColumn() { @@ -148,4 +153,20 @@ public void testArray() throws Exception { Assert.assertEquals(c.getArrayBaseColumn().getOriginalTypeName(), "LowCardinality(Nullable(String))"); Assert.assertFalse(c.getArrayBaseColumn().isArray()); } + + @Test(dataProvider = "enumTypesProvider", groups = { "unit" }) + public void testEnum(String typeName) throws Exception { + Assert.assertThrows(IllegalArgumentException.class, + () -> ClickHouseColumn.of("e", typeName + "('Query''Start' = a)")); + Assert.assertThrows(IllegalArgumentException.class, () -> ClickHouseColumn.of("e", typeName + "(aa,1)")); + ClickHouseColumn column = ClickHouseColumn.of("e", typeName + "('Query''Start' = 1, 'Query\\'Finish' = 10)"); + Assert.assertTrue(column.isEnum()); + Assert.assertEquals(column.getDataType(), ClickHouseDataType.of(typeName)); + Assert.assertThrows(IllegalArgumentException.class, () -> column.getEnumConstants().name(2)); + Assert.assertThrows(IllegalArgumentException.class, () -> column.getEnumConstants().value("")); + Assert.assertEquals(column.getEnumConstants().name(1), "Query'Start"); + Assert.assertEquals(column.getEnumConstants().name(10), "Query'Finish"); + Assert.assertEquals(column.getEnumConstants().value("Query'Start"), 1); + Assert.assertEquals(column.getEnumConstants().value("Query'Finish"), 10); + } } diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseDataTypeTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseDataTypeTest.java index 1cdf1d5b4..7ccfb60b0 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseDataTypeTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseDataTypeTest.java @@ -12,7 +12,7 @@ public void testAlias() { } for (ClickHouseDataType t : ClickHouseDataType.values()) { - Assert.assertFalse(ClickHouseDataType.isAlias(t.name())); + Assert.assertFalse(ClickHouseDataType.isAlias(t.name()), t.name() + " should not be an alias"); } } diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseValuesTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseValuesTest.java index 410a966e8..27db2f679 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseValuesTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseValuesTest.java @@ -193,6 +193,6 @@ public void testConvertToSqlExpression() throws UnknownHostException { LocalDate.of(2021, 11, 12), LocalTime.of(11, 12, 13, 123456789), LocalDateTime.of(LocalDate.of(2021, 11, 12), LocalTime.of(11, 12, 13, 123456789)), new boolean[] { false, true } }), - "[1,97,1,2,3,4,5.555,6.666666,'\\'x\\'','00000000-0000-0000-0000-000000000002','127.0.0.1','0:0:0:0:0:0:0:1',29,123456789,1.23456789,NULL,'2021-11-12','11:12:13.123456789','2021-11-12 11:12:13.123456789',[0,1]]"); + "[1,97,1,2,3,4,5.555,6.666666,'\\'x\\'','00000000-0000-0000-0000-000000000002','127.0.0.1','0:0:0:0:0:0:0:1',30,123456789,1.23456789,NULL,'2021-11-12','11:12:13.123456789','2021-11-12 11:12:13.123456789',[0,1]]"); } } diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/data/ClickHouseEnumValueTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/data/ClickHouseEnumValueTest.java index bdfc88d0d..fd4789f92 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/data/ClickHouseEnumValueTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/data/ClickHouseEnumValueTest.java @@ -9,31 +9,38 @@ public class ClickHouseEnumValueTest extends BaseClickHouseValueTest { @Test(groups = { "unit" }) public void testCopy() { - sameValue(ClickHouseEnumValue.ofNull(), ClickHouseEnumValue.ofNull(), 3, 9, Object.class, + sameValue(ClickHouseEnumValue.ofNull(ClickHouseDataType.class), + ClickHouseEnumValue.ofNull(ClickHouseDataType.class), 3, 9, Object.class, ClickHouseDataType.class, Object.class, Object.class); - sameValue(ClickHouseEnumValue.of(ClickHouseDataType.String), ClickHouseEnumValue.of(ClickHouseDataType.String), + sameValue(ClickHouseEnumValue.of(ClickHouseDataType.String), + ClickHouseEnumValue.of(ClickHouseDataType.String), 3, 9, Object.class, ClickHouseDataType.class, Object.class, Object.class); - ClickHouseEnumValue v = ClickHouseEnumValue.of(ClickHouseDataType.Array); + ClickHouseEnumValue v = ClickHouseEnumValue.of(ClickHouseDataType.Array); sameValue(v, v.copy(), 3, 9, Object.class, ClickHouseDataType.class, Object.class, Object.class); } @Test(groups = { "unit" }) public void testUpdate() { - sameValue(ClickHouseEnumValue.ofNull(), - ClickHouseEnumValue.ofNull().update(ClickHouseDataType.Date32).set(null), 3, 9, Object.class, + sameValue(ClickHouseEnumValue.ofNull(ClickHouseDataType.class), + ClickHouseEnumValue.ofNull(ClickHouseDataType.class).update(ClickHouseDataType.Date32).set(true, 0), 3, + 9, + Object.class, ClickHouseDataType.class, Object.class, Object.class); sameValue(ClickHouseEnumValue.of(ClickHouseDataType.Date32), - ClickHouseEnumValue.ofNull().update(ClickHouseDataType.Date32), 3, 9, Object.class, + ClickHouseEnumValue.ofNull(ClickHouseDataType.class).update(ClickHouseDataType.Date32), 3, 9, + Object.class, ClickHouseDataType.class, Object.class, Object.class); sameValue(ClickHouseEnumValue.of(ClickHouseDataType.Date32), - ClickHouseEnumValue.of(ClickHouseFormat.Arrow).update(ClickHouseDataType.Date32), 3, 9, Object.class, + ClickHouseEnumValue.of(ClickHouseDataType.Array).update(ClickHouseDataType.Date32), 3, 9, + Object.class, ClickHouseDataType.class, Object.class, Object.class); sameValue(ClickHouseEnumValue.of(ClickHouseDataType.IntervalYear), ClickHouseEnumValue.of(ClickHouseDataType.String).update(false), 3, 9, Object.class, ClickHouseDataType.class, Object.class, Object.class); sameValue(ClickHouseEnumValue.of(ClickHouseDataType.IntervalYear), - ClickHouseEnumValue.of(ClickHouseDataType.String).update(new boolean[] { false }), 3, 9, Object.class, + ClickHouseEnumValue.of(ClickHouseDataType.String).update(new boolean[] { false }), 3, 9, + Object.class, ClickHouseDataType.class, Object.class, Object.class); sameValue(ClickHouseEnumValue.of(ClickHouseDataType.IntervalYear), ClickHouseEnumValue.of(ClickHouseDataType.String).update('\0'), 3, 9, Object.class, @@ -51,7 +58,8 @@ public void testUpdate() { ClickHouseEnumValue.of(ClickHouseDataType.String).update(0L), 3, 9, Object.class, ClickHouseDataType.class, Object.class, Object.class); sameValue(ClickHouseEnumValue.of(ClickHouseDataType.IntervalYear), - ClickHouseEnumValue.of(ClickHouseDataType.String).update("IntervalYear"), 3, 9, Object.class, + ClickHouseEnumValue.of(ClickHouseDataType.String).update("IntervalYear"), 3, 9, + Object.class, ClickHouseDataType.class, Object.class, Object.class); } } 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 a45edbbb9..d57eb590b 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 @@ -51,9 +51,9 @@ public class ClickHouseGrpcClientTest extends BaseIntegrationTest { @DataProvider(name = "simpleTypeProvider") public Object[][] getSimpleTypes() { return new Object[][] { - { ClickHouseDataType.Enum.name() + "('v-1' = -1, 'v0' = 0, 'v+1' = 1)", "0", "-1", "1" }, - { ClickHouseDataType.Enum8.name() + "('v-1' = -1, 'v0' = 0, 'v+1' = 1)", "0", "-1", "1" }, - { ClickHouseDataType.Enum16.name() + "('v-1' = -1, 'v0' = 0, 'v+1' = 1)", "0", "-1", "1" }, + { ClickHouseDataType.Enum.name() + "('v-1' = -1, 'v0' = 0, 'v+1' = 1)", "v0", "v-1", "v+1" }, + { ClickHouseDataType.Enum8.name() + "('v-1' = -1, 'v0' = 0, 'v+1' = 1)", "v0", "v-1", "v+1" }, + { ClickHouseDataType.Enum16.name() + "('v-1' = -1, 'v0' = 0, 'v+1' = 1)", "v0", "v-1", "v+1" }, { ClickHouseDataType.Int8.name(), "0", "-1", "1" }, { ClickHouseDataType.UInt8.name(), "0", "255", "1" }, { ClickHouseDataType.Int16.name(), "0", "-1", "1" }, diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseArray.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseArray.java index 716165d33..8b2dcd2c0 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseArray.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseArray.java @@ -39,7 +39,8 @@ public String getBaseTypeName() throws SQLException { public int getBaseType() throws SQLException { ensureValid(); - return JdbcTypeMapping.toJdbcType(getBaseColumn()); + // don't really want to pass type mapping to Array object... + return JdbcTypeMapping.toJdbcType(null, getBaseColumn()); } @Override 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 4d354300f..afff94883 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseDatabaseMetaData.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseDatabaseMetaData.java @@ -11,6 +11,7 @@ import java.util.HashMap; import java.util.LinkedList; import java.util.List; +import java.util.Locale; import java.util.Map; import com.clickhouse.client.ClickHouseChecker; @@ -34,6 +35,7 @@ public class ClickHouseDatabaseMetaData extends JdbcWrapper implements DatabaseM "REMOTE TABLE", "TABLE", "VIEW", "SYSTEM TABLE", "TEMPORARY TABLE" }; private final ClickHouseConnection connection; + private final Map> typeMaps; protected ResultSet empty(String columns) throws SQLException { return fixed(columns, null); @@ -77,8 +79,9 @@ protected ResultSet query(String sql, ClickHouseRecordTransformer func, boolean } } - public ClickHouseDatabaseMetaData(ClickHouseConnection connection) { + public ClickHouseDatabaseMetaData(ClickHouseConnection connection) throws SQLException { this.connection = ClickHouseChecker.nonNull(connection, "Connection"); + this.typeMaps = connection.getTypeMap(); } @Override @@ -749,7 +752,7 @@ public ResultSet getTables(String catalog, String schemaPattern, String tableNam List results = new ArrayList<>(databases.size()); for (String database : databases) { Map params = new HashMap<>(); - params.put("comment", connection.getServerVersion().check("[18.16,)") ? "t.comment" : "''"); + params.put("comment", connection.getServerVersion().check("[20.8,)") ? "t.comment" : "''"); params.put("database", ClickHouseValues.convertToQuotedString(database)); params.put("table", ClickHouseChecker.isNullOrEmpty(tableNamePattern) ? "'%'" : ClickHouseValues.convertToQuotedString(tableNamePattern)); @@ -760,7 +763,7 @@ public ResultSet getTables(String catalog, String schemaPattern, String tableNam + "when t.engine in ('Buffer', 'Memory', 'Set') then 'MEMORY TABLE' " + "when t.is_temporary != 0 then 'TEMPORARY TABLE' " + "when t.engine like '%View' then 'VIEW' when t.engine = 'Dictionary' then 'DICTIONARY' " - + "when t.engine like 'System%' then 'SYSTEM TABLE' " + + "when t.engine like 'Async%' or t.engine like 'System%' then 'SYSTEM TABLE' " + "when empty(t.data_paths) then 'REMOTE TABLE' else 'TABLE' end as TABLE_TYPE, " + ":comment as REMARKS, null as TYPE_CAT, d.engine as TYPE_SCHEM, " + "t.engine as TYPE_NAME, null as SELF_REFERENCING_COL_NAME, null as REF_GENERATION\n" @@ -821,7 +824,7 @@ public ResultSet getColumns(String catalog, String schemaPattern, String tableNa String typeName = r.getValue("TYPE_NAME").asString(); try { ClickHouseColumn column = ClickHouseColumn.of("", typeName); - r.getValue("DATA_TYPE").update(JdbcTypeMapping.toJdbcType(column)); + r.getValue("DATA_TYPE").update(JdbcTypeMapping.toJdbcType(typeMaps, column)); r.getValue("COLUMN_SIZE").update(column.getDataType().getByteLength()); if (column.isNullable()) { r.getValue("NULLABLE").update(DatabaseMetaData.typeNullable); @@ -964,7 +967,7 @@ private Object[] toTypeRow(String typeName, String aliasTo) { break; } return new Object[] { typeName, - JdbcTypeMapping.toJdbcType(ClickHouseColumn.of("", type, false, false, new String[0])), + JdbcTypeMapping.toJdbcType(typeMaps, ClickHouseColumn.of("", type, false, false, new String[0])), type.getMaxPrecision(), prefix, suffix, params, nullable, type.isCaseSensitive() ? 1 : 0, searchable, type.getMaxPrecision() > 0 && !type.isSigned() ? 1 : 0, money, 0, aliasTo == null || aliasTo.isEmpty() ? type.name() : aliasTo, type.getMinScale(), type.getMaxScale(), 0, @@ -1240,17 +1243,21 @@ public ResultSet getClientInfoProperties() throws SQLException { @Override public ResultSet getFunctions(String catalog, String schemaPattern, String functionNamePattern) throws SQLException { + Map params = new HashMap<>(); + params.put("filter", ClickHouseChecker.isNullOrEmpty(schemaPattern) + || "system".contains(schemaPattern.toLowerCase(Locale.ROOT)) ? "1" : "0"); + params.put("pattern", ClickHouseChecker.isNullOrEmpty(functionNamePattern) ? "'%'" + : ClickHouseValues.convertToQuotedString(functionNamePattern)); + String sql = ClickHouseParameterizedQuery.apply( - "select null as FUNCTION_CAT, null as FUNCTION_SCHEM, name as FUNCTION_NAME,\n" + "select * from (select null as FUNCTION_CAT, 'system' as FUNCTION_SCHEM, name as FUNCTION_NAME,\n" + "concat('case-', case_insensitive ? 'in' : '', 'sensitive function', is_aggregate ? ' for aggregation' : '') as REMARKS," + "1 as FUNCTION_TYPE, name as SPECIFIC_NAME from system.functions\n" + "where alias_to = '' and name like :pattern order by name union all\n" - + "select null as FUNCTION_CAT, null as FUNCTION_SCHEM, name as FUNCTION_NAME,\n" + + "select null as FUNCTION_CAT, 'system' as FUNCTION_SCHEM, name as FUNCTION_NAME,\n" + "'case-sensistive table function' as REMARKS, 2 as FUNCTION_TYPE, name as SPECIFIC_NAME from system.table_functions\n" - + "order by name", - Collections.singletonMap("pattern", ClickHouseChecker.isNullOrEmpty(functionNamePattern) ? "'%'" - : ClickHouseValues.convertToQuotedString(functionNamePattern))); - + + "order by name) where :filter", + params); return query(sql); } diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseResultSet.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseResultSet.java index 0d45700ff..a8348103d 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseResultSet.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseResultSet.java @@ -74,16 +74,16 @@ public class ClickHouseResultSet extends AbstractResultSet { this.tsTimeZone = null; // TimeZone.getDefault(); this.dateTimeZone = this.tsTimeZone; + this.defaultTypeMap = Collections.emptyMap(); this.currentRow = null; try { this.columns = response.getColumns(); - this.metaData = new ClickHouseResultSetMetaData(database, table, columns); + this.metaData = new ClickHouseResultSetMetaData(database, table, columns, defaultTypeMap); this.rowCursor = response.records().iterator(); } catch (Exception e) { throw new IllegalStateException(e); } - this.defaultTypeMap = Collections.emptyMap(); this.rowNumber = 0; // before the first row this.lastReadColumn = 0; @@ -110,18 +110,18 @@ public ClickHouseResultSet(String database, String table, ClickHouseStatement st this.tsTimeZone = conn.getEffectiveTimeZone().orElse(null); this.dateTimeZone = this.tsTimeZone; + Map> typeMap = conn.getTypeMap(); + this.defaultTypeMap = typeMap != null && !typeMap.isEmpty() ? Collections.unmodifiableMap(typeMap) + : Collections.emptyMap(); this.currentRow = null; try { this.columns = response.getColumns(); - this.metaData = new ClickHouseResultSetMetaData(database, table, columns); + this.metaData = new ClickHouseResultSetMetaData(database, table, columns, defaultTypeMap); this.rowCursor = response.records().iterator(); } catch (Exception e) { throw SqlExceptionUtils.handle(e); } - Map> typeMap = conn.getTypeMap(); - this.defaultTypeMap = typeMap != null && !typeMap.isEmpty() ? Collections.unmodifiableMap(typeMap) - : Collections.emptyMap(); this.rowNumber = 0; // before the first row this.lastReadColumn = 0; diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseResultSetMetaData.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseResultSetMetaData.java index a78cc1555..4ed0eac9f 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseResultSetMetaData.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseResultSetMetaData.java @@ -3,28 +3,33 @@ import java.sql.ResultSetMetaData; import java.sql.SQLException; import java.util.List; +import java.util.Map; import com.clickhouse.client.ClickHouseColumn; import com.clickhouse.client.ClickHouseUtils; public class ClickHouseResultSetMetaData extends JdbcWrapper implements ResultSetMetaData { - public static ResultSetMetaData of(String database, String table, List columns) + public static ResultSetMetaData of(String database, String table, List columns, + Map> typeMap) throws SQLException { if (database == null || table == null || columns == null) { throw SqlExceptionUtils.clientError("Non-null database, table, and column list are required"); } - return new ClickHouseResultSetMetaData(database, table, columns); + return new ClickHouseResultSetMetaData(database, table, columns, typeMap); } private final String database; private final String table; private final List columns; + private final Map> typeMap; - protected ClickHouseResultSetMetaData(String database, String table, List columns) { + protected ClickHouseResultSetMetaData(String database, String table, List columns, + Map> typeMap) { this.database = database; this.table = table; this.columns = columns; + this.typeMap = typeMap; } protected List getColumns() { @@ -116,7 +121,7 @@ public String getCatalogName(int column) throws SQLException { @Override public int getColumnType(int column) throws SQLException { - return JdbcTypeMapping.toJdbcType(getColumn(column)); + return JdbcTypeMapping.toJdbcType(typeMap, getColumn(column)); } @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 9466e1af0..6ecb0c2a8 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcConfig.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcConfig.java @@ -2,15 +2,26 @@ import java.sql.DriverPropertyInfo; import java.util.Collections; +import java.util.LinkedHashMap; import java.util.LinkedList; import java.util.List; +import java.util.Map; import java.util.Properties; +import java.util.Map.Entry; + +import com.clickhouse.client.ClickHouseChecker; +import com.clickhouse.client.ClickHouseUtils; +import com.clickhouse.client.logging.Logger; +import com.clickhouse.client.logging.LoggerFactory; public class JdbcConfig { + private static final Logger log = LoggerFactory.getLogger(JdbcConfig.class); + public static final String PROP_AUTO_COMMIT = "autoCommit"; public static final String PROP_FETCH_SIZE = "fetchSize"; public static final String PROP_JDBC_COMPLIANT = "jdbcCompliant"; public static final String PROP_NAMED_PARAM = "namedParameter"; + public static final String PROP_TYPE_MAP = "typeMappings"; public static final String PROP_WRAPPER_OBJ = "wrapperObject"; private static final String BOOLEAN_FALSE = "false"; @@ -20,6 +31,7 @@ public class JdbcConfig { private static final String DEFAULT_FETCH_SIZE = "0"; private static final String DEFAULT_JDBC_COMPLIANT = BOOLEAN_TRUE; private static final String DEFAULT_NAMED_PARAM = BOOLEAN_FALSE; + private static final String DEFAULT_TYPE_MAP = ""; private static final String DEFAULT_WRAPPER_OBJ = BOOLEAN_FALSE; static boolean extractBooleanValue(Properties props, String key, String defaultValue) { @@ -40,6 +52,36 @@ static int extractIntValue(Properties props, String key, String defaultValue) { return Integer.parseInt(value != null ? value.toString() : defaultValue); } + static Map> extractTypeMapValue(Properties props, String key, String defaultValue) { + String value = null; + if (props == null || props.isEmpty() || key == null || key.isEmpty()) { + value = defaultValue; + } else { + Object v = props.remove(key); + value = v != null ? v.toString() : defaultValue; + } + + if (ClickHouseChecker.isNullOrBlank(value)) { + return Collections.emptyMap(); + } + + Map> map = new LinkedHashMap<>(); + ClassLoader loader = JdbcConfig.class.getClassLoader(); + for (Entry e : ClickHouseUtils.getKeyValuePairs(value).entrySet()) { + Class clazz = null; + try { + clazz = loader.loadClass(e.getValue()); + } catch (Throwable t) { + log.warn("Failed to add mapping [%s]=[%s], due to: %s", e.getKey(), e.getValue(), t.getMessage()); + } + if (clazz != null) { + map.put(e.getKey(), clazz); + } + } + + return Collections.unmodifiableMap(map); + } + public static List getDriverProperties() { List list = new LinkedList<>(); DriverPropertyInfo info = new DriverPropertyInfo(PROP_AUTO_COMMIT, DEFAULT_AUTO_COMMIT); @@ -61,6 +103,10 @@ public static List getDriverProperties() { info.description = "Whether to use named parameter(e.g. :ts(DateTime64(6)) or :value etc.) instead of standard JDBC question mark placeholder."; 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); + info = new DriverPropertyInfo(PROP_WRAPPER_OBJ, DEFAULT_WRAPPER_OBJ); info.choices = new String[] { BOOLEAN_TRUE, BOOLEAN_FALSE }; info.description = "Whether to return wrapper object like Array or Struct in ResultSet.getObject method."; @@ -73,6 +119,7 @@ public static List getDriverProperties() { private final int fetchSize; private final boolean jdbcCompliant; private final boolean namedParameter; + private final Map> typeMap; private final boolean wrapperObject; public JdbcConfig() { @@ -88,6 +135,7 @@ public JdbcConfig(Properties props) { this.fetchSize = extractIntValue(props, PROP_FETCH_SIZE, DEFAULT_FETCH_SIZE); this.jdbcCompliant = extractBooleanValue(props, PROP_JDBC_COMPLIANT, DEFAULT_JDBC_COMPLIANT); this.namedParameter = extractBooleanValue(props, PROP_NAMED_PARAM, DEFAULT_NAMED_PARAM); + this.typeMap = extractTypeMapValue(props, PROP_TYPE_MAP, DEFAULT_TYPE_MAP); this.wrapperObject = extractBooleanValue(props, PROP_WRAPPER_OBJ, DEFAULT_WRAPPER_OBJ); } @@ -110,6 +158,15 @@ public int getFetchSize() { return fetchSize; } + /** + * Gets custom type map. + * + * @return non-null custom type map + */ + public Map> getTypeMap() { + return typeMap; + } + /** * Checks whether JDBC-complaint mode is enabled or not. * diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcTypeMapping.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcTypeMapping.java index eaa4adb7a..0d323454d 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcTypeMapping.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcTypeMapping.java @@ -1,30 +1,92 @@ package com.clickhouse.jdbc; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; import java.sql.Types; +import java.time.LocalDate; import java.time.LocalDateTime; +import java.time.LocalTime; import java.time.OffsetDateTime; +import java.time.ZonedDateTime; +import java.util.Map; import com.clickhouse.client.ClickHouseColumn; import com.clickhouse.client.ClickHouseDataType; public final class JdbcTypeMapping { + /** + * Gets corresponding JDBC type for the given Java class. + * + * @param javaClass non-null Java class + * @return JDBC type + */ + public static int toJdbcType(Class javaClass) { + int sqlType = Types.OTHER; + if (javaClass == boolean.class || javaClass == Boolean.class) { + sqlType = Types.BOOLEAN; + } else if (javaClass == byte.class || javaClass == Byte.class) { + sqlType = Types.TINYINT; + } else if (javaClass == short.class || javaClass == Short.class) { + sqlType = Types.SMALLINT; + } else if (javaClass == int.class || javaClass == Integer.class) { + sqlType = Types.INTEGER; + } else if (javaClass == long.class || javaClass == Long.class) { + sqlType = Types.BIGINT; + } else if (javaClass == float.class || javaClass == Float.class) { + sqlType = Types.FLOAT; + } else if (javaClass == double.class || javaClass == Double.class) { + sqlType = Types.DOUBLE; + } else if (javaClass == BigInteger.class) { + sqlType = Types.NUMERIC; + } else if (javaClass == BigDecimal.class) { + sqlType = Types.DECIMAL; + } else if (javaClass == Date.class || javaClass == LocalDate.class) { + sqlType = Types.DATE; + } else if (javaClass == Time.class || javaClass == LocalTime.class) { + sqlType = Types.TIME; + } else if (javaClass == Timestamp.class || javaClass == LocalDateTime.class) { + sqlType = Types.TIMESTAMP; + } else if (javaClass == OffsetDateTime.class || javaClass == ZonedDateTime.class) { + sqlType = Types.TIMESTAMP_WITH_TIMEZONE; + } else if (javaClass == String.class || javaClass == byte[].class || Enum.class.isAssignableFrom(javaClass)) { + sqlType = Types.VARCHAR; + } else if (javaClass.isArray()) { + sqlType = Types.ARRAY; + } + return sqlType; + } + /** * Gets corresponding JDBC type for the given column. * * @param column non-null column definition * @return JDBC type */ - public static int toJdbcType(ClickHouseColumn column) { + public static int toJdbcType(Map> typeMap, ClickHouseColumn column) { + if (typeMap != null && !typeMap.isEmpty()) { + Class javaClass = typeMap.get(column.getOriginalTypeName()); + if (javaClass == null) { + javaClass = typeMap.get(column.getDataType().name()); + } + + if (javaClass != null) { + return toJdbcType(javaClass); + } + } + int sqlType = Types.OTHER; switch (column.getDataType()) { - case Enum: - case Enum8: + case Bool: + sqlType = Types.BOOLEAN; + break; case Int8: sqlType = Types.TINYINT; break; case UInt8: - case Enum16: case Int16: sqlType = Types.SMALLINT; break; @@ -73,6 +135,9 @@ public static int toJdbcType(ClickHouseColumn column) { case DateTime64: sqlType = column.getTimeZone() != null ? Types.TIMESTAMP_WITH_TIMEZONE : Types.TIMESTAMP; break; + case Enum: + case Enum8: + case Enum16: case IPv4: case IPv6: case FixedString: 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 957280ba4..c206221f7 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 @@ -167,7 +167,8 @@ public ClickHouseConnectionImpl(ConnectionInfo connInfo) throws SQLException { try (ClickHouseResponse response = clientRequest.copy().option(ClickHouseClientOption.ASYNC, false) .option(ClickHouseClientOption.COMPRESS, false).option(ClickHouseClientOption.DECOMPRESS, false) .option(ClickHouseClientOption.FORMAT, ClickHouseFormat.RowBinaryWithNamesAndTypes) - .query("select currentDatabase(), currentUser(), timezone(), version()").execute().get()) { + .query("select currentDatabase(), currentUser(), timezone(), version() FORMAT RowBinaryWithNamesAndTypes") + .execute().get()) { ClickHouseRecord r = response.firstRecord(); currentDb = r.getValue(0).asString(); currentUser = r.getValue(1).asString(); @@ -215,7 +216,7 @@ public ClickHouseConnectionImpl(ConnectionInfo connInfo) throws SQLException { defaultCalendar = new GregorianCalendar(clientTimeZone.get()); } this.serverVersion = version; - this.typeMap = new HashMap<>(); + this.typeMap = new HashMap<>(jdbcConf.getTypeMap()); this.fakeTransaction = new AtomicReference<>(); } diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseJdbcUrlParser.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseJdbcUrlParser.java index 435799260..574788969 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseJdbcUrlParser.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseJdbcUrlParser.java @@ -6,6 +6,7 @@ import java.net.URLDecoder; import java.nio.charset.StandardCharsets; import java.sql.SQLException; +import java.util.Locale; import java.util.Properties; import com.clickhouse.client.ClickHouseChecker; @@ -31,8 +32,8 @@ public static class ConnectionInfo { private final Properties props; protected ConnectionInfo(URI uri, ClickHouseNode server, Properties props) throws URISyntaxException { - this.uri = new URI("jdbc:clickhouse:" + server.getProtocol().name().toLowerCase(), null, server.getHost(), - server.getPort(), "/" + server.getDatabase().orElse(""), + this.uri = new URI("jdbc:clickhouse:" + server.getProtocol().name().toLowerCase(Locale.ROOT), null, + server.getHost(), server.getPort(), "/" + server.getDatabase().orElse(""), removeCredentialsFromQuery(uri.getRawQuery()), null); this.server = server; this.jdbcConf = new JdbcConfig(props); diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseParameterMetaData.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseParameterMetaData.java index f31ef9fe9..35b8b5a7b 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseParameterMetaData.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseParameterMetaData.java @@ -64,7 +64,7 @@ public int getScale(int param) throws SQLException { @Override public int getParameterType(int param) throws SQLException { ClickHouseColumn p = getParameter(param); - return p != null ? JdbcTypeMapping.toJdbcType(p) : Types.OTHER; + return p != null ? JdbcTypeMapping.toJdbcType(null, p) : Types.OTHER; } @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 28b1ea9c0..944b144fb 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 @@ -7,6 +7,7 @@ import java.sql.SQLWarning; import java.sql.Statement; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.UUID; @@ -86,25 +87,6 @@ private ClickHouseResponse getLastResponse(Map o return response; } - protected ClickHouseSqlStatement applyFormat(ClickHouseSqlStatement stmt, ClickHouseFormat preferredFormat) { - /* - * if (ClickHouseChecker.nonNull(stmt, "ParsedStatement").isQuery() && - * !stmt.hasFormat()) { String sql = stmt.getSQL(); String format = - * ClickHouseChecker.nonNull(preferredFormat, "Format").name(); - * - * Map positions = new HashMap<>(); - * positions.putAll(stmt.getPositions()); - * positions.put(ClickHouseSqlStatement.KEYWORD_FORMAT, sql.length()); - * - * sql = new StringBuilder(sql).append("\nFORMAT ").append(format).toString(); - * stmt = new ClickHouseSqlStatement(sql, stmt.getStatementType(), - * stmt.getCluster(), stmt.getDatabase(), stmt.getTable(), format, - * stmt.getOutfile(), stmt.getParameters(), positions); } - */ - - return stmt; - } - protected void ensureOpen() throws SQLException { if (closed) { throw SqlExceptionUtils.clientError("Cannot operate on a closed statement"); @@ -160,7 +142,6 @@ protected ClickHouseResponse executeStatement(String stmt, protected ClickHouseResponse executeStatement(ClickHouseSqlStatement stmt, Map options, List tables, Map settings) throws SQLException { - // stmt = applyFormat(stmt, request.getFormat()); return executeStatement(stmt.getSQL(), options, tables, settings); } @@ -205,13 +186,7 @@ protected ClickHouseSqlStatement parseSqlStatements(String sql) { throw new IllegalArgumentException("Failed to parse given SQL: " + sql); } - ClickHouseSqlStatement lastStmt = getLastStatement(); - ClickHouseSqlStatement formattedStmt = applyFormat(lastStmt, request.getFormat()); - if (formattedStmt != lastStmt) { - setLastStatement(lastStmt = formattedStmt); - } - - return lastStmt; + return getLastStatement(); } protected ResultSet updateResult(ClickHouseSqlStatement stmt, ClickHouseResponse response) throws SQLException { diff --git a/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/domain/ClickHouseDataType.java b/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/domain/ClickHouseDataType.java index 182120b90..2c3e4e41e 100644 --- a/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/domain/ClickHouseDataType.java +++ b/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/domain/ClickHouseDataType.java @@ -8,6 +8,7 @@ import java.sql.Timestamp; import java.util.Collections; import java.util.HashMap; +import java.util.Locale; import java.util.Map; import java.util.UUID; @@ -30,73 +31,73 @@ public enum ClickHouseDataType { // https://github.com/ClickHouse/ClickHouse/blob/master/src/DataTypes/DataTypeFixedString.cpp // https://github.com/ClickHouse/ClickHouse/blob/master/src/DataTypes/DataTypesNumber.cpp // https://github.com/ClickHouse/ClickHouse/blob/master/src/DataTypes/DataTypeString.cpp - IntervalYear (JDBCType.INTEGER, Integer.class, true, 19, 0), - IntervalQuarter (JDBCType.INTEGER, Integer.class, true, 19, 0), - IntervalMonth (JDBCType.INTEGER, Integer.class, true, 19, 0), - IntervalWeek (JDBCType.INTEGER, Integer.class, true, 19, 0), - IntervalDay (JDBCType.INTEGER, Integer.class, true, 19, 0), - IntervalHour (JDBCType.INTEGER, Integer.class, true, 19, 0), - IntervalMinute (JDBCType.INTEGER, Integer.class, true, 19, 0), - IntervalSecond (JDBCType.INTEGER, Integer.class, true, 19, 0), - UInt256 (JDBCType.NUMERIC, BigInteger.class, true, 39, 0), - UInt128 (JDBCType.NUMERIC, BigInteger.class, true, 20, 0), - UInt64 (JDBCType.BIGINT, BigInteger.class, false, 19, 0, - "BIGINT UNSIGNED"), - UInt32 (JDBCType.BIGINT, Long.class, false, 10, 0, - "INT UNSIGNED", "INTEGER UNSIGNED", "MEDIUMINT UNSIGNED"), - UInt16 (JDBCType.SMALLINT, Integer.class, false, 5, 0, - "SMALLINT UNSIGNED"), - UInt8 (JDBCType.TINYINT, Integer.class, false, 3, 0, - "TINYINT UNSIGNED", "INT1 UNSIGNED"), - Int256 (JDBCType.NUMERIC, BigInteger.class, true, 40, 0), - Int128 (JDBCType.NUMERIC, BigInteger.class, true, 20, 0), - Int64 (JDBCType.BIGINT, Long.class, true, 20, 0, - "BIGINT", "BIGINT SIGNED"), - Int32 (JDBCType.INTEGER, Integer.class, true, 11, 0, - "INT", "INTEGER", "MEDIUMINT", "INT SIGNED", "INTEGER SIGNED", "MEDIUMINT SIGNED"), - Int16 (JDBCType.SMALLINT, Integer.class, true, 6, 0, - "SMALLINT", "SMALLINT SIGNED"), - Int8 (JDBCType.TINYINT, Integer.class, true, 4, 0, - "TINYINT", "BOOL", "BOOLEAN", "INT1", "BYTE", "TINYINT SIGNED", "INT1 SIGNED"), - Date (JDBCType.DATE, Date.class, false, 10, 0), - DateTime (JDBCType.TIMESTAMP, Timestamp.class, false, 19, 0, - "TIMESTAMP"), - DateTime32 (JDBCType.TIMESTAMP, Timestamp.class, false, 19, 0), - DateTime64 (JDBCType.TIMESTAMP, Timestamp.class, false, 38, 3), // scale up to 18 - Enum8 (JDBCType.VARCHAR, String.class, false, 0, 0, - "ENUM"), - Enum16 (JDBCType.VARCHAR, String.class, false, 0, 0), - Float32 (JDBCType.REAL, Float.class, true, 8, 8, - "SINGLE", "REAL"), - Float64 (JDBCType.DOUBLE, Double.class, true, 17, 17, - "DOUBLE", "DOUBLE PRECISION"), - Decimal32 (JDBCType.DECIMAL, BigDecimal.class, true, 9, 9), - Decimal64 (JDBCType.DECIMAL, BigDecimal.class, true, 18, 18), - Decimal128 (JDBCType.DECIMAL, BigDecimal.class, true, 38, 38), - Decimal256 (JDBCType.DECIMAL, BigDecimal.class, true, 76, 20), - Decimal (JDBCType.DECIMAL, BigDecimal.class, true, 0, 0, - "DEC", "NUMERIC", "FIXED"), - UUID (JDBCType.OTHER, UUID.class, false, 36, 0), - IPv4 (JDBCType.VARCHAR, String.class, false, 10, 0), - IPv6 (JDBCType.VARCHAR, String.class, false, 0, 0), - String (JDBCType.VARCHAR, String.class, false, 0, 0, - "CHAR", "NCHAR", "CHARACTER", "VARCHAR", "NVARCHAR", "VARCHAR2", - "TEXT", "TINYTEXT", "MEDIUMTEXT", "LONGTEXT", - "BLOB", "CLOB", "TINYBLOB", "MEDIUMBLOB", "LONGBLOB", "BYTEA", - "CHARACTER LARGE OBJECT", "CHARACTER VARYING", "CHAR LARGE OBJECT", "CHAR VARYING", - "NATIONAL CHAR", "NATIONAL CHARACTER", "NATIONAL CHARACTER LARGE OBJECT", - "NATIONAL CHARACTER VARYING", "NATIONAL CHAR VARYING", - "NCHAR VARYING", "NCHAR LARGE OBJECT", "BINARY LARGE OBJECT", "BINARY VARYING"), - FixedString (JDBCType.CHAR, String.class, false, -1, 0, - "BINARY"), - Nothing (JDBCType.NULL, Object.class, false, 0, 0), - Nested (JDBCType.STRUCT, String.class, false, 0, 0), + IntervalYear(JDBCType.INTEGER, Integer.class, true, 19, 0), + IntervalQuarter(JDBCType.INTEGER, Integer.class, true, 19, 0), + IntervalMonth(JDBCType.INTEGER, Integer.class, true, 19, 0), + IntervalWeek(JDBCType.INTEGER, Integer.class, true, 19, 0), + IntervalDay(JDBCType.INTEGER, Integer.class, true, 19, 0), + IntervalHour(JDBCType.INTEGER, Integer.class, true, 19, 0), + IntervalMinute(JDBCType.INTEGER, Integer.class, true, 19, 0), + IntervalSecond(JDBCType.INTEGER, Integer.class, true, 19, 0), + UInt256(JDBCType.NUMERIC, BigInteger.class, true, 39, 0), + UInt128(JDBCType.NUMERIC, BigInteger.class, true, 20, 0), + UInt64(JDBCType.BIGINT, BigInteger.class, false, 19, 0, + "BIGINT UNSIGNED"), + UInt32(JDBCType.BIGINT, Long.class, false, 10, 0, + "INT UNSIGNED", "INTEGER UNSIGNED", "MEDIUMINT UNSIGNED"), + UInt16(JDBCType.SMALLINT, Integer.class, false, 5, 0, + "SMALLINT UNSIGNED"), + UInt8(JDBCType.TINYINT, Integer.class, false, 3, 0, + "TINYINT UNSIGNED", "INT1 UNSIGNED"), + Int256(JDBCType.NUMERIC, BigInteger.class, true, 40, 0), + Int128(JDBCType.NUMERIC, BigInteger.class, true, 20, 0), + Int64(JDBCType.BIGINT, Long.class, true, 20, 0, + "BIGINT", "BIGINT SIGNED"), + Int32(JDBCType.INTEGER, Integer.class, true, 11, 0, + "INT", "INTEGER", "MEDIUMINT", "INT SIGNED", "INTEGER SIGNED", "MEDIUMINT SIGNED"), + Int16(JDBCType.SMALLINT, Integer.class, true, 6, 0, + "SMALLINT", "SMALLINT SIGNED"), + Int8(JDBCType.TINYINT, Integer.class, true, 4, 0, + "TINYINT", "BOOL", "BOOLEAN", "INT1", "BYTE", "TINYINT SIGNED", "INT1 SIGNED"), + Date(JDBCType.DATE, Date.class, false, 10, 0), + DateTime(JDBCType.TIMESTAMP, Timestamp.class, false, 19, 0, + "TIMESTAMP"), + DateTime32(JDBCType.TIMESTAMP, Timestamp.class, false, 19, 0), + DateTime64(JDBCType.TIMESTAMP, Timestamp.class, false, 38, 3), // scale up to 18 + Enum8(JDBCType.VARCHAR, String.class, false, 0, 0, + "ENUM"), + Enum16(JDBCType.VARCHAR, String.class, false, 0, 0), + Float32(JDBCType.REAL, Float.class, true, 8, 8, + "SINGLE", "REAL"), + Float64(JDBCType.DOUBLE, Double.class, true, 17, 17, + "DOUBLE", "DOUBLE PRECISION"), + Decimal32(JDBCType.DECIMAL, BigDecimal.class, true, 9, 9), + Decimal64(JDBCType.DECIMAL, BigDecimal.class, true, 18, 18), + Decimal128(JDBCType.DECIMAL, BigDecimal.class, true, 38, 38), + Decimal256(JDBCType.DECIMAL, BigDecimal.class, true, 76, 20), + Decimal(JDBCType.DECIMAL, BigDecimal.class, true, 0, 0, + "DEC", "NUMERIC", "FIXED"), + UUID(JDBCType.OTHER, UUID.class, false, 36, 0), + IPv4(JDBCType.VARCHAR, String.class, false, 10, 0), + IPv6(JDBCType.VARCHAR, String.class, false, 0, 0), + String(JDBCType.VARCHAR, String.class, false, 0, 0, + "CHAR", "NCHAR", "CHARACTER", "VARCHAR", "NVARCHAR", "VARCHAR2", + "TEXT", "TINYTEXT", "MEDIUMTEXT", "LONGTEXT", + "BLOB", "CLOB", "TINYBLOB", "MEDIUMBLOB", "LONGBLOB", "BYTEA", + "CHARACTER LARGE OBJECT", "CHARACTER VARYING", "CHAR LARGE OBJECT", "CHAR VARYING", + "NATIONAL CHAR", "NATIONAL CHARACTER", "NATIONAL CHARACTER LARGE OBJECT", + "NATIONAL CHARACTER VARYING", "NATIONAL CHAR VARYING", + "NCHAR VARYING", "NCHAR LARGE OBJECT", "BINARY LARGE OBJECT", "BINARY VARYING"), + FixedString(JDBCType.CHAR, String.class, false, -1, 0, + "BINARY"), + Nothing(JDBCType.NULL, Object.class, false, 0, 0), + Nested(JDBCType.STRUCT, String.class, false, 0, 0), // TODO use list/collection for Tuple - Tuple (JDBCType.OTHER, String.class, false, 0, 0), - Array (JDBCType.ARRAY, Array.class, false, 0, 0), - Map (JDBCType.OTHER, Map.class, false, 0, 0), - AggregateFunction (JDBCType.OTHER, String.class, false, 0, 0), - Unknown (JDBCType.OTHER, String.class, false, 0, 0); + Tuple(JDBCType.OTHER, String.class, false, 0, 0), + Array(JDBCType.ARRAY, Array.class, false, 0, 0), + Map(JDBCType.OTHER, Map.class, false, 0, 0), + AggregateFunction(JDBCType.OTHER, String.class, false, 0, 0), + Unknown(JDBCType.OTHER, String.class, false, 0, 0); private static final Map name2type; @@ -109,15 +110,15 @@ public enum ClickHouseDataType { if (used != null) { throw new IllegalStateException(java.lang.String.format(errorMsg, t.name(), used.name())); } - String nameInUpperCase = t.name().toUpperCase(); + String nameInUpperCase = t.name().toUpperCase(Locale.ROOT); if (!nameInUpperCase.equals(t.name())) { used = map.put(nameInUpperCase, t); if (used != null) { throw new IllegalStateException(java.lang.String.format(errorMsg, nameInUpperCase, used.name())); } } - for (String alias: t.aliases) { - used = map.put(alias.toUpperCase(), t); + for (String alias : t.aliases) { + used = map.put(alias.toUpperCase(Locale.ROOT), t); if (used != null) { throw new IllegalStateException(java.lang.String.format(errorMsg, alias, used.name())); } @@ -134,8 +135,8 @@ public enum ClickHouseDataType { private final String[] aliases; ClickHouseDataType(JDBCType jdbcType, Class javaClass, - boolean signed, int defaultPrecision, int defaultScale, - String... aliases) { + boolean signed, int defaultPrecision, int defaultScale, + String... aliases) { this.jdbcType = jdbcType; this.javaClass = javaClass; this.signed = signed; @@ -169,7 +170,7 @@ public int getDefaultScale() { } public static ClickHouseDataType fromTypeString(String typeString) { - return name2type.getOrDefault(typeString.trim().toUpperCase(), ClickHouseDataType.Unknown); + return name2type.getOrDefault(typeString.trim().toUpperCase(Locale.ROOT), ClickHouseDataType.Unknown); } public static ClickHouseDataType resolveDefaultArrayDataType(String typeName) { diff --git a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseStatementTest.java b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseStatementTest.java index 8f2234be9..c80823e91 100644 --- a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseStatementTest.java +++ b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseStatementTest.java @@ -18,6 +18,7 @@ import java.util.TimeZone; import java.util.UUID; +import com.clickhouse.client.ClickHouseDataType; import com.clickhouse.client.ClickHouseValues; import com.clickhouse.client.config.ClickHouseClientOption; import com.clickhouse.client.data.ClickHouseDateTimeValue; @@ -51,7 +52,8 @@ public void testLogComment() throws SQLException { String sql = "-- select something " + uuid + "\nselect 12345"; stmt.execute(sql + "; system flush logs;"); ResultSet rs = stmt.executeQuery( - "select distinct query from system.query_log where log_comment = 'select something " + uuid + "'"); + "select distinct query, type from system.query_log where log_comment = 'select something " + uuid + + "'"); Assert.assertTrue(rs.next()); Assert.assertEquals(rs.getString(1), sql); Assert.assertFalse(rs.next()); @@ -130,6 +132,35 @@ public void testTimestamp() throws SQLException { } } + @Test(groups = "integration") + public void testCustomTypeMappings() throws SQLException { + Properties props = new Properties(); + try (ClickHouseConnection conn = newConnection(props); + ClickHouseStatement stmt = conn.createStatement()) { + ResultSet rs = stmt.executeQuery("select cast('a' as Enum('a'=1,'b'=2))"); + Assert.assertTrue(rs.next()); + Assert.assertEquals(rs.getObject(1), "a"); + Assert.assertEquals(rs.getString(1), "a"); + Assert.assertEquals(rs.getInt(1), 1); + Assert.assertFalse(rs.next()); + } + + props.setProperty("typeMappings", + "Enum8=java.lang.Byte,DateTime64=java.lang.String, String=com.clickhouse.client.ClickHouseDataType"); + try (ClickHouseConnection conn = newConnection(props); + ClickHouseStatement stmt = conn.createStatement()) { + ResultSet rs = stmt.executeQuery( + "select cast('a' as Enum('a'=1,'b'=2)), toDateTime64('2021-12-21 12:34:56.789',3), 'Float64'"); + Assert.assertTrue(rs.next()); + Assert.assertEquals(rs.getObject(1), (byte) 1); + Assert.assertEquals(rs.getObject(2), "2021-12-21 12:34:56.789"); + Assert.assertEquals(rs.getObject(3), ClickHouseDataType.Float64); + Assert.assertEquals(rs.getString(1), "a"); + Assert.assertEquals(rs.getInt(1), 1); + Assert.assertFalse(rs.next()); + } + } + @Test(groups = "integration") public void testTimeZone() throws SQLException { String dateType = "DateTime32"; 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 0e3ffcc4b..854c2a691 100644 --- a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/JdbcIntegrationTest.java +++ b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/JdbcIntegrationTest.java @@ -4,6 +4,7 @@ import java.sql.DriverManager; import java.sql.SQLException; import java.sql.Statement; +import java.util.Locale; import java.util.Properties; import com.clickhouse.client.BaseIntegrationTest; @@ -52,7 +53,7 @@ public JdbcIntegrationTest() { className = className.substring(0, className.length() - CLASS_SUFFIX.length()); } - this.dbName = "test_" + className.toLowerCase(); + this.dbName = "test_" + className.toLowerCase(Locale.ROOT); } public String getServerAddress(ClickHouseProtocol protocol) { diff --git a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/parser/ClickHouseSqlParserTest.java b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/parser/ClickHouseSqlParserTest.java index f87f625d8..ecd3cbfef 100644 --- a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/parser/ClickHouseSqlParserTest.java +++ b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/parser/ClickHouseSqlParserTest.java @@ -66,9 +66,11 @@ public void testParseNonSql() throws ParseException { String sql; assertEquals(parse(sql = null), - new ClickHouseSqlStatement[] { new ClickHouseSqlStatement(sql, StatementType.UNKNOWN) }); + new ClickHouseSqlStatement[] { + new ClickHouseSqlStatement(sql, StatementType.UNKNOWN) }); assertEquals(parse(sql = ""), - new ClickHouseSqlStatement[] { new ClickHouseSqlStatement(sql, StatementType.UNKNOWN) }); + new ClickHouseSqlStatement[] { + new ClickHouseSqlStatement(sql, StatementType.UNKNOWN) }); checkSingleStatement(parse(sql = "invalid sql"), sql); checkSingleStatement(parse(sql = "-- some comments"), sql); @@ -86,14 +88,16 @@ public void testParseNonSql() throws ParseException { public void testAlterStatement() { String sql; - checkSingleStatement(parse(sql = "ALTER TABLE alter_test ADD COLUMN Added0 UInt32"), sql, StatementType.ALTER, + checkSingleStatement(parse(sql = "ALTER TABLE alter_test ADD COLUMN Added0 UInt32"), sql, + StatementType.ALTER, "system", "alter_test"); checkSingleStatement( parse(sql = "ALTER TABLE test_db.test_table UPDATE a = 1, \"b\" = '2', `c`=3.3 WHERE d=123 and e=456"), sql, StatementType.ALTER_UPDATE, "test_db", "test_table"); checkSingleStatement(parse(sql = "ALTER TABLE tTt on cluster 'cc' delete WHERE d=123 and e=456"), sql, StatementType.ALTER_DELETE, "system", "tTt"); - checkSingleStatement(parse(sql = "ALTER USER user DEFAULT ROLE role1, role2"), sql, StatementType.ALTER); + checkSingleStatement(parse(sql = "ALTER USER user DEFAULT ROLE role1, role2"), sql, + StatementType.ALTER); } @Test(groups = "unit") @@ -124,7 +128,8 @@ public void testDeleteStatement() { String sql; checkSingleStatement(parse(sql = "delete from a"), sql, StatementType.DELETE, "system", "a"); - checkSingleStatement(parse(sql = "delete from c.a where upper(a)=upper(lower(b))"), sql, StatementType.DELETE, + checkSingleStatement(parse(sql = "delete from c.a where upper(a)=upper(lower(b))"), sql, + StatementType.DELETE, "c", "a"); } @@ -139,7 +144,8 @@ public void testDescribeStatement() { // fix issue #614 checkSingleStatement(parse(sql = "desc t1 t2"), sql, StatementType.DESCRIBE, "system", "columns"); checkSingleStatement(parse(sql = "desc table t1 t2"), sql, StatementType.DESCRIBE, "system", "columns"); - checkSingleStatement(parse(sql = "desc table t1 as `t2`"), sql, StatementType.DESCRIBE, "system", "columns"); + checkSingleStatement(parse(sql = "desc table t1 as `t2`"), sql, StatementType.DESCRIBE, "system", + "columns"); } @Test(groups = "unit") @@ -147,7 +153,8 @@ public void testDetachStatement() { String sql; checkSingleStatement(parse(sql = "detach TABLE t"), sql, StatementType.DETACH); - checkSingleStatement(parse(sql = "detach TABLE if exists t.t on cluster 'cc'"), sql, StatementType.DETACH); + checkSingleStatement(parse(sql = "detach TABLE if exists t.t on cluster 'cc'"), sql, + StatementType.DETACH); } @Test(groups = "unit") @@ -205,20 +212,24 @@ public void testInsertStatement() throws ParseException { } assertEquals(s.getStartPosition("values"), valuePosition); - checkSingleStatement(parse(sql = "insert into function null('a UInt8') values(1)"), sql, StatementType.INSERT); + checkSingleStatement(parse(sql = "insert into function null('a UInt8') values(1)"), sql, + StatementType.INSERT); checkSingleStatement(parse(sql = "insert into function null('a UInt8') values(1)(2)"), sql, StatementType.INSERT); checkSingleStatement(parse(sql = "insert into function null('a UInt8') select * from number(10)"), sql, StatementType.INSERT); - checkSingleStatement(parse(sql = "insert into test2(a,b) values('values(',',')"), sql, StatementType.INSERT, + checkSingleStatement(parse(sql = "insert into test2(a,b) values('values(',',')"), sql, + StatementType.INSERT, "system", "test2"); checkSingleStatement(parse(sql = "INSERT INTO table t(a, b, c) values('1', ',', 'ccc')"), sql, StatementType.INSERT, "system", "t"); checkSingleStatement(parse(sql = "INSERT INTO table t(a, b, c) values('1', 2, 'ccc') (3,2,1)"), sql, StatementType.INSERT, "system", "t"); - checkSingleStatement(parse(sql = "INSERT INTO table s.t select * from ttt"), sql, StatementType.INSERT, "s", + checkSingleStatement(parse(sql = "INSERT INTO table s.t select * from ttt"), sql, StatementType.INSERT, + "s", "t"); - checkSingleStatement(parse(sql = "INSERT INTO insert_select_testtable (* EXCEPT(b)) Values (2, 2)"), sql, + checkSingleStatement(parse(sql = "INSERT INTO insert_select_testtable (* EXCEPT(b)) Values (2, 2)"), + sql, StatementType.INSERT, "system", "insert_select_testtable"); } @@ -238,7 +249,9 @@ public void testKillStatement() { public void testOptimizeStatement() { String sql; - checkSingleStatement(parse(sql = "OPTIMIZE TABLE a ON CLUSTER cluster PARTITION ID 'partition_id' FINAL"), sql, + checkSingleStatement( + parse(sql = "OPTIMIZE TABLE a ON CLUSTER cluster PARTITION ID 'partition_id' FINAL"), + sql, StatementType.OPTIMIZE); } @@ -246,7 +259,8 @@ public void testOptimizeStatement() { public void testRenameStatement() { String sql; - checkSingleStatement(parse(sql = "RENAME TABLE table1 TO table2, table3 TO table4 ON CLUSTER cluster"), sql, + checkSingleStatement(parse(sql = "RENAME TABLE table1 TO table2, table3 TO table4 ON CLUSTER cluster"), + sql, StatementType.RENAME); checkSingleStatement(parse( sql = "RENAME TABLE db1.table1 TO db2.table2, db2.table3 to db2.table4, db3.table5 to db2.table6 ON CLUSTER 'c'"), @@ -258,7 +272,8 @@ public void testRevokeStatement() { String sql; checkSingleStatement(parse(sql = "REVOKE SELECT ON accounts.* FROM john"), sql, StatementType.REVOKE); - checkSingleStatement(parse(sql = "REVOKE SELECT(wage) ON accounts.staff FROM mira"), sql, StatementType.REVOKE); + checkSingleStatement(parse(sql = "REVOKE SELECT(wage) ON accounts.staff FROM mira"), sql, + StatementType.REVOKE); } @Test(groups = "unit") @@ -266,14 +281,18 @@ public void testSelectStatement() { String sql; assertEquals(parse(sql = "select\n1"), new ClickHouseSqlStatement[] { new ClickHouseSqlStatement(sql, - StatementType.SELECT, null, null, "unknown", null, null, null, null, null, null, null) }); + StatementType.SELECT, null, null, "unknown", null, null, null, null, null, null, + null) }); assertEquals(parse(sql = "select\r\n1"), new ClickHouseSqlStatement[] { new ClickHouseSqlStatement(sql, - StatementType.SELECT, null, null, "unknown", null, null, null, null, null, null, null) }); + StatementType.SELECT, null, null, "unknown", null, null, null, null, null, null, + null) }); assertEquals(parse(sql = "select 314 limit 5\nFORMAT JSONCompact;"), - new ClickHouseSqlStatement[] { new ClickHouseSqlStatement("select 314 limit 5\nFORMAT JSONCompact", - StatementType.SELECT, null, null, "unknown", null, "JSONCompact", null, null, null, null, - null) }); + new ClickHouseSqlStatement[] { + new ClickHouseSqlStatement("select 314 limit 5\nFORMAT JSONCompact", + StatementType.SELECT, null, null, "unknown", null, + "JSONCompact", null, null, null, null, + null) }); checkSingleStatement(parse(sql = "select (())"), sql, StatementType.SELECT); checkSingleStatement(parse(sql = "select []"), sql, StatementType.SELECT); @@ -281,15 +300,19 @@ public void testSelectStatement() { checkSingleStatement(parse(sql = "select *"), sql, StatementType.SELECT); checkSingleStatement(parse(sql = "select timezone()"), sql, StatementType.SELECT); checkSingleStatement(parse(sql = "select @@version, $version"), sql, StatementType.SELECT); - checkSingleStatement(parse(sql = "select * from jdbc('db', 'schema', 'select 1')"), sql, StatementType.SELECT, + checkSingleStatement(parse(sql = "select * from jdbc('db', 'schema', 'select 1')"), sql, + StatementType.SELECT, "system", "jdbc"); - checkSingleStatement(parse(sql = "select 1 as a1, a.a as a2, aa(a1, a2) a3, length(a3) as a4 from x"), sql, + checkSingleStatement(parse(sql = "select 1 as a1, a.a as a2, aa(a1, a2) a3, length(a3) as a4 from x"), + sql, StatementType.SELECT, "system", "x"); checkSingleStatement(parse(sql = "select x.* from (select [1,2] a, (1,2,3) b, a[1], b.2) x"), sql, StatementType.SELECT, "system", "x"); - checkSingleStatement(parse(sql = "select (3, [[1,2],[3,4]]) as a, (a.2)[2][1]"), sql, StatementType.SELECT); + checkSingleStatement(parse(sql = "select (3, [[1,2],[3,4]]) as a, (a.2)[2][1]"), sql, + StatementType.SELECT); checkSingleStatement( - parse(sql = "select 1,1.1,'\"''`a' a, \"'`\"\"a\" as b, (1 + `a`.a) c, null, inf i, nan as n"), sql, + parse(sql = "select 1,1.1,'\"''`a' a, \"'`\"\"a\" as b, (1 + `a`.a) c, null, inf i, nan as n"), + sql, StatementType.SELECT); checkSingleStatement(parse(sql = "select 1 as select"), sql, StatementType.SELECT); checkSingleStatement(parse(sql = "select 1, 2 a, 3 as b, 1+1-2*3/4, *, c.* from c a"), sql, @@ -298,43 +321,51 @@ public void testSelectStatement() { checkSingleStatement(parse( sql = " -- cc\nselect 1 as `a.b`, a, 1+1, b from \"a\".`b` inner join a on a.abb/* \n\r\n1*/\n=2 and a.abb = c.a and a=1 and (k is null and j not in(1,2))"), sql, StatementType.SELECT, "a", "b"); - checkSingleStatement(parse(sql = "SELECT idx, s FROM test.mymetadata WHERE idx = ?"), sql, StatementType.SELECT, + checkSingleStatement(parse(sql = "SELECT idx, s FROM test.mymetadata WHERE idx = ?"), sql, + StatementType.SELECT, "test", "mymetadata"); checkSingleStatement(parse(sql = "WITH 2 AS two SELECT two * two"), sql, StatementType.SELECT); checkSingleStatement(parse( sql = "SELECT i, array(toUnixTimestamp(dt_server[1])), array(toUnixTimestamp(dt_berlin[1])), array(toUnixTimestamp(dt_lax[1])) FROM test.fun_with_timezones_array"), sql, StatementType.SELECT, "test", "fun_with_timezones_array"); - checkSingleStatement(parse(sql = "SELECT SUM(x) FROM t WHERE y = ? GROUP BY ?"), sql, StatementType.SELECT, + checkSingleStatement(parse(sql = "SELECT SUM(x) FROM t WHERE y = ? GROUP BY ?"), sql, + StatementType.SELECT, "system", "t"); assertEquals(parse(sql = loadSql("issue-441_with-totals.sql")), - new ClickHouseSqlStatement[] { new ClickHouseSqlStatement(sql, StatementType.SELECT, null, null, + new ClickHouseSqlStatement[] { new ClickHouseSqlStatement(sql, StatementType.SELECT, + null, null, "unknown", null, null, null, null, new HashMap() { { put("TOTALS", 208); } }, null, null) }); assertEquals(parse(sql = loadSql("issue-555_custom-format.sql")), - new ClickHouseSqlStatement[] { new ClickHouseSqlStatement(sql, StatementType.SELECT, null, null, "wrd", - null, "CSVWithNames", null, null, null, null, null) }); + new ClickHouseSqlStatement[] { + new ClickHouseSqlStatement(sql, StatementType.SELECT, null, null, "wrd", + null, "CSVWithNames", null, null, null, null, null) }); assertEquals(parse(sql = loadSql("with-clause.sql")), - new ClickHouseSqlStatement[] { new ClickHouseSqlStatement(sql, StatementType.SELECT, null, null, - "unknown", null, null, null, null, null, null, null) }); + new ClickHouseSqlStatement[] { + new ClickHouseSqlStatement(sql, StatementType.SELECT, null, null, + "unknown", null, null, null, null, null, null, null) }); } @Test(groups = "unit") public void testSetStatement() { String sql; - checkSingleStatement(parse(sql = "SET profile = 'my-profile', mutations_sync=1"), sql, StatementType.SET); - checkSingleStatement(parse(sql = "SET DEFAULT ROLE role1, role2, role3 TO user"), sql, StatementType.SET); + checkSingleStatement(parse(sql = "SET profile = 'my-profile', mutations_sync=1"), sql, + StatementType.SET); + checkSingleStatement(parse(sql = "SET DEFAULT ROLE role1, role2, role3 TO user"), sql, + StatementType.SET); } @Test(groups = "unit") public void testShowStatement() { String sql; - checkSingleStatement(parse(sql = "SHOW DATABASES LIKE '%de%'"), sql, StatementType.SHOW, "system", "databases"); + checkSingleStatement(parse(sql = "SHOW DATABASES LIKE '%de%'"), sql, StatementType.SHOW, "system", + "databases"); checkSingleStatement(parse(sql = "show tables from db"), sql, StatementType.SHOW, "system", "tables"); checkSingleStatement(parse(sql = "show dictionaries from db"), sql, StatementType.SHOW, "system", "dictionaries"); @@ -344,9 +375,12 @@ public void testShowStatement() { public void testSystemStatement() { String sql; - checkSingleStatement(parse(sql = "SYSTEM DROP REPLICA 'replica_name' FROM ZKPATH '/path/to/table/in/zk'"), sql, + checkSingleStatement( + parse(sql = "SYSTEM DROP REPLICA 'replica_name' FROM ZKPATH '/path/to/table/in/zk'"), + sql, StatementType.SYSTEM); - checkSingleStatement(parse(sql = "SYSTEM RESTART REPLICA db.replicated_merge_tree_family_table_name"), sql, + checkSingleStatement(parse(sql = "SYSTEM RESTART REPLICA db.replicated_merge_tree_family_table_name"), + sql, StatementType.SYSTEM); } @@ -382,27 +416,33 @@ public void testWatchStatement() throws ParseException { @Test(groups = "unit") public void testComments() throws ParseException { String sql; - checkSingleStatement(parse(sql = "select\n--something\n//else\n1/*2*/ from a.b"), sql, StatementType.SELECT, + checkSingleStatement(parse(sql = "select\n--something\n//else\n1/*2*/ from a.b"), sql, + StatementType.SELECT, "a", "b"); checkSingleStatement(parse(sql = "select 1/*/**/*/ from a.b"), sql, StatementType.SELECT, "a", "b"); checkSingleStatement(parse(sql = "select 1/*/1/**/*2*/ from a.b"), sql, StatementType.SELECT, "a", "b"); checkSingleStatement(parse(sql = "SELECT /*/**/*/ 1 from a.b"), sql, StatementType.SELECT, "a", "b"); checkSingleStatement(parse(sql = "SELECT /*a/*b*/c*/ 1 from a.b"), sql, StatementType.SELECT, "a", "b"); - checkSingleStatement(parse(sql = "SELECT /*ab/*cd*/ef*/ 1 from a.b"), sql, StatementType.SELECT, "a", "b"); + checkSingleStatement(parse(sql = "SELECT /*ab/*cd*/ef*/ 1 from a.b"), sql, StatementType.SELECT, "a", + "b"); } @Test(groups = "unit") public void testMultipleStatements() throws ParseException { assertEquals(parse("use ab;;;select 1; ;\t;\r;\n"), - new ClickHouseSqlStatement[] { new ClickHouseSqlStatement("use ab", StatementType.USE, null, "ab", null, - null, null, null, null, null, null, null), + new ClickHouseSqlStatement[] { + new ClickHouseSqlStatement("use ab", StatementType.USE, null, "ab", + null, + null, null, null, null, null, null, null), new ClickHouseSqlStatement("select 1", StatementType.SELECT) }); assertEquals(parse("select * from \"a;1\".`b;c`;;;select 1 as `a ; a`; ;\t;\r;\n"), new ClickHouseSqlStatement[] { - new ClickHouseSqlStatement("select * from \"a;1\".`b;c`", StatementType.SELECT, null, "a;1", + new ClickHouseSqlStatement("select * from \"a;1\".`b;c`", + StatementType.SELECT, null, "a;1", "b;c", null, null, null, null, null, null, null), - new ClickHouseSqlStatement("select 1 as `a ; a`", StatementType.SELECT) }); + new ClickHouseSqlStatement("select 1 as `a ; a`", + StatementType.SELECT) }); } @Test(groups = "unit") @@ -411,8 +451,10 @@ public void testAlias() throws ParseException { checkSingleStatement(parse(sql = "select 1 as c, 2 b"), sql, StatementType.SELECT); checkSingleStatement(parse(sql = "select 1 from a.b c"), sql, StatementType.SELECT, "a", "b"); checkSingleStatement(parse(sql = "select 1 select from a.b c"), sql, StatementType.SELECT, "a", "b"); - checkSingleStatement(parse(sql = "select 1 from (select 2) b"), sql, StatementType.SELECT, "system", "b"); - checkSingleStatement(parse(sql = "select 1 from (select 2) as from"), sql, StatementType.SELECT, "system", + checkSingleStatement(parse(sql = "select 1 from (select 2) b"), sql, StatementType.SELECT, "system", + "b"); + checkSingleStatement(parse(sql = "select 1 from (select 2) as from"), sql, StatementType.SELECT, + "system", "from"); checkSingleStatement(parse(sql = "select 1 from a.b c1, b.a c2"), sql, StatementType.SELECT, "a", "b"); } @@ -421,14 +463,17 @@ public void testAlias() throws ParseException { public void testExpression() throws ParseException { String sql; checkSingleStatement(parse(sql = "SELECT a._ from a.b"), sql, StatementType.SELECT, "a", "b"); - checkSingleStatement(parse(sql = "SELECT 2 BETWEEN 1 + 1 AND 3 - 1 from a.b"), sql, StatementType.SELECT, "a", + checkSingleStatement(parse(sql = "SELECT 2 BETWEEN 1 + 1 AND 3 - 1 from a.b"), sql, + StatementType.SELECT, "a", "b"); checkSingleStatement(parse(sql = "SELECT CASE WHEN 1 THEN 2 WHEN 3 THEN 4 ELSE 5 END from a.b"), sql, StatementType.SELECT, "a", "b"); - checkSingleStatement(parse(sql = "select (1,2) a1, a1.1, a1 .1, a1 . 1 from a.b"), sql, StatementType.SELECT, + checkSingleStatement(parse(sql = "select (1,2) a1, a1.1, a1 .1, a1 . 1 from a.b"), sql, + StatementType.SELECT, "a", "b"); checkSingleStatement(parse(sql = "select -.0, +.0, -a from a.b"), sql, StatementType.SELECT, "a", "b"); - checkSingleStatement(parse(sql = "select 1 and `a`.\"b\" c1, c1 or (c2 and c3), c4 ? c5 : c6 from a.b"), sql, + checkSingleStatement(parse(sql = "select 1 and `a`.\"b\" c1, c1 or (c2 and c3), c4 ? c5 : c6 from a.b"), + sql, StatementType.SELECT, "a", "b"); checkSingleStatement(parse(sql = "select [[[1,2],[3,4],[5,6]]] a, a[1][1][2] from a.b"), sql, StatementType.SELECT, "a", "b"); @@ -437,21 +482,28 @@ public void testExpression() throws ParseException { sql, StatementType.SELECT, "a", "b"); checkSingleStatement(parse(sql = "select c.c1, c.c2 c, c.c3 as cc, c.c4.1.2 from a.b"), sql, StatementType.SELECT, "a", "b"); - checkSingleStatement(parse(sql = "select - (select (1,).1) from a.b"), sql, StatementType.SELECT, "a", "b"); - checkSingleStatement(parse(sql = "select 1.1e1,(1) . 1 , ((1,2)).1 .2 . 3 from a.b"), sql, StatementType.SELECT, + checkSingleStatement(parse(sql = "select - (select (1,).1) from a.b"), sql, StatementType.SELECT, "a", + "b"); + checkSingleStatement(parse(sql = "select 1.1e1,(1) . 1 , ((1,2)).1 .2 . 3 from a.b"), sql, + StatementType.SELECT, "a", "b"); - checkSingleStatement(parse(sql = "select a.b.c1, c1, b.c1 from a.b"), sql, StatementType.SELECT, "a", "b"); + checkSingleStatement(parse(sql = "select a.b.c1, c1, b.c1 from a.b"), sql, StatementType.SELECT, "a", + "b"); checkSingleStatement(parse(sql = "select date'2020-02-04', timestamp '2020-02-04' from a.b"), sql, StatementType.SELECT, "a", "b"); checkSingleStatement(parse(sql = "select count (), sum(c1), fake(a1, count(), (1+1)) from a.b"), sql, StatementType.SELECT, "a", "b"); - checkSingleStatement(parse(sql = "select {}, {'a':'b', 'c':'1'} from a.b"), sql, StatementType.SELECT, "a", + checkSingleStatement(parse(sql = "select {}, {'a':'b', 'c':'1'} from a.b"), sql, StatementType.SELECT, + "a", "b"); - checkSingleStatement(parse(sql = "select [], [1,2], [ [1,2], [3,4] ] from a.b"), sql, StatementType.SELECT, "a", + checkSingleStatement(parse(sql = "select [], [1,2], [ [1,2], [3,4] ] from a.b"), sql, + StatementType.SELECT, "a", "b"); checkSingleStatement(parse(sql = "select 1+1-1*1/1 from a.b"), sql, StatementType.SELECT, "a", "b"); - checkSingleStatement(parse(sql = "select (1+(1-1)*1/1)-1 from a.b"), sql, StatementType.SELECT, "a", "b"); - checkSingleStatement(parse(sql = "select (1+(1+(-1))*1/1)-(select (1,).1) from a.b"), sql, StatementType.SELECT, + checkSingleStatement(parse(sql = "select (1+(1-1)*1/1)-1 from a.b"), sql, StatementType.SELECT, "a", + "b"); + checkSingleStatement(parse(sql = "select (1+(1+(-1))*1/1)-(select (1,).1) from a.b"), sql, + StatementType.SELECT, "a", "b"); } @@ -556,35 +608,46 @@ public void testExtractDBAndTableName() { String sql; checkSingleStatement(parse(sql = "SELECT 1 from table"), sql, StatementType.SELECT, "system", "table"); - checkSingleStatement(parse(sql = "SELECT 1 from table a"), sql, StatementType.SELECT, "system", "table"); - checkSingleStatement(parse(sql = "SELECT 1 from\ntable a"), sql, StatementType.SELECT, "system", "table"); - checkSingleStatement(parse(sql = "SELECT 1\nfrom\ntable a"), sql, StatementType.SELECT, "system", "table"); - checkSingleStatement(parse(sql = "SELECT 1\nFrom\ntable a"), sql, StatementType.SELECT, "system", "table"); + checkSingleStatement(parse(sql = "SELECT 1 from table a"), sql, StatementType.SELECT, "system", + "table"); + checkSingleStatement(parse(sql = "SELECT 1 from\ntable a"), sql, StatementType.SELECT, "system", + "table"); + checkSingleStatement(parse(sql = "SELECT 1\nfrom\ntable a"), sql, StatementType.SELECT, "system", + "table"); + checkSingleStatement(parse(sql = "SELECT 1\nFrom\ntable a"), sql, StatementType.SELECT, "system", + "table"); checkSingleStatement(parse(sql = "SELECT 1 from db.table a"), sql, StatementType.SELECT, "db", "table"); checkSingleStatement(parse(sql = " SELECT 1 from \"db.table\" a"), sql, StatementType.SELECT, "system", "db.table"); checkSingleStatement(parse(sql = "SELECT 1 from `db.table` a"), sql, StatementType.SELECT, "system", "db.table"); checkSingleStatement(parse(sql = "from `db.table` a"), sql, StatementType.UNKNOWN, "system", "unknown"); - checkSingleStatement(parse(sql = " from `db.table` a"), sql, StatementType.UNKNOWN, "system", "unknown"); - checkSingleStatement(parse(sql = "ELECT from `db.table` a"), sql, StatementType.UNKNOWN, "system", "unknown"); + checkSingleStatement(parse(sql = " from `db.table` a"), sql, StatementType.UNKNOWN, "system", + "unknown"); + checkSingleStatement(parse(sql = "ELECT from `db.table` a"), sql, StatementType.UNKNOWN, "system", + "unknown"); checkSingleStatement(parse(sql = "SHOW tables"), sql, StatementType.SHOW, "system", "tables"); checkSingleStatement(parse(sql = "desc table1"), sql, StatementType.DESCRIBE, "system", "columns"); checkSingleStatement(parse(sql = "DESC table1"), sql, StatementType.DESCRIBE, "system", "columns"); - checkSingleStatement(parse(sql = "SELECT 'from db.table a' from tab"), sql, StatementType.SELECT, "system", + checkSingleStatement(parse(sql = "SELECT 'from db.table a' from tab"), sql, StatementType.SELECT, + "system", "tab"); checkSingleStatement(parse(sql = "SELECT"), sql, StatementType.UNKNOWN, "system", "unknown"); checkSingleStatement(parse(sql = "S"), sql, StatementType.UNKNOWN, "system", "unknown"); checkSingleStatement(parse(sql = ""), sql, StatementType.UNKNOWN, "system", "unknown"); - checkSingleStatement(parse(sql = " SELECT 1 from table from"), sql, StatementType.SELECT, "system", "table"); - checkSingleStatement(parse(sql = " SELECT 1 from table from"), sql, StatementType.SELECT, "system", "table"); + checkSingleStatement(parse(sql = " SELECT 1 from table from"), sql, StatementType.SELECT, "system", + "table"); + checkSingleStatement(parse(sql = " SELECT 1 from table from"), sql, StatementType.SELECT, "system", + "table"); checkSingleStatement(parse(sql = "SELECT fromUnixTimestamp64Milli(time) as x from table"), sql, StatementType.SELECT, "system", "table"); - checkSingleStatement(parse(sql = " SELECT fromUnixTimestamp64Milli(time)from table"), sql, StatementType.SELECT, + checkSingleStatement(parse(sql = " SELECT fromUnixTimestamp64Milli(time)from table"), sql, + StatementType.SELECT, "system", "table"); checkSingleStatement(parse(sql = "/*qq*/ SELECT fromUnixTimestamp64Milli(time)from table"), sql, StatementType.SELECT, "system", "table"); - checkSingleStatement(parse(sql = " SELECTfromUnixTimestamp64Milli(time)from table"), sql, StatementType.UNKNOWN, + checkSingleStatement(parse(sql = " SELECTfromUnixTimestamp64Milli(time)from table"), sql, + StatementType.UNKNOWN, "system", "unknown"); checkSingleStatement(parse(sql = " SELECT fromUnixTimestamp64Milli(time)from \".inner.a\""), sql, StatementType.SELECT, "system", ".inner.a"); @@ -631,6 +694,15 @@ public void testJdbcEscapeSyntax() { assertEquals(stmts[0].getSQL(), "select "); } + @Test(groups = "unit") + public void testNewParameterSyntax() { + String sql = "select {column_a:String}"; + ClickHouseSqlStatement[] stmts = parse(sql); + assertEquals(stmts.length, 1); + assertEquals(stmts[0].isQuery(), true); + assertEquals(stmts[0].getSQL(), sql); + } + // known issue public void testTernaryOperator() { String sql = "select x > 2 ? 'a' : 'b' from (select number as x from system.numbers limit ?)"; From 7ddc6e09d11cfb5de632d592bb143d5bd700ab22 Mon Sep 17 00:00:00 2001 From: Zhichun Wu Date: Tue, 21 Dec 2021 22:59:20 +0800 Subject: [PATCH 6/6] Fix benchmark build failure --- .../java/com/clickhouse/benchmark/misc/FactoryBenchmark.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/misc/FactoryBenchmark.java b/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/misc/FactoryBenchmark.java index db7c03c10..094d447bc 100644 --- a/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/misc/FactoryBenchmark.java +++ b/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/misc/FactoryBenchmark.java @@ -28,6 +28,7 @@ import org.openjdk.jmh.annotations.Warmup; import org.openjdk.jmh.infra.Blackhole; import com.clickhouse.benchmark.BaseState; +import com.clickhouse.client.ClickHouseEnum; import com.clickhouse.client.ClickHouseValue; import com.clickhouse.client.data.ClickHouseBigDecimalValue; import com.clickhouse.client.data.ClickHouseBigIntegerValue; @@ -82,7 +83,7 @@ public void setupSamples() { add(map, list, Double.class, () -> ClickHouseDoubleValue.ofNull()); add(map, list, BigInteger.class, () -> ClickHouseBigIntegerValue.ofNull()); add(map, list, BigDecimal.class, () -> ClickHouseBigDecimalValue.ofNull()); - add(map, list, Enum.class, () -> ClickHouseEnumValue.ofNull()); + add(map, list, Enum.class, () -> ClickHouseEnumValue.ofNull(ClickHouseEnum.EMPTY)); add(map, list, Inet4Address.class, () -> ClickHouseIpv4Value.ofNull()); add(map, list, Inet6Address.class, () -> ClickHouseIpv6Value.ofNull()); @@ -121,7 +122,7 @@ ClickHouseValue newValue(Class clazz) { } else if (BigDecimal.class.equals(clazz)) { return ClickHouseBigDecimalValue.ofNull(); } else if (Enum.class.equals(clazz)) { - return ClickHouseEnumValue.ofNull(); + return ClickHouseEnumValue.ofNull(ClickHouseEnum.EMPTY); } else if (Inet4Address.class.equals(clazz)) { return ClickHouseIpv4Value.ofNull(); } else if (Inet6Address.class.equals(clazz)) {