From eb1f1ed48f1403264a2a7cb815d90124a58f0f8f Mon Sep 17 00:00:00 2001 From: Zhichun Wu Date: Tue, 30 Nov 2021 18:29:05 +0800 Subject: [PATCH 1/8] Reformat code --- .../config/ClickHouseConfigOptionTest.java | 202 +++++++++--------- 1 file changed, 101 insertions(+), 101 deletions(-) diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/config/ClickHouseConfigOptionTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/config/ClickHouseConfigOptionTest.java index e93f30889..c6e0fe8bc 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/config/ClickHouseConfigOptionTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/config/ClickHouseConfigOptionTest.java @@ -8,110 +8,110 @@ import com.clickhouse.client.ClickHouseFormat; public class ClickHouseConfigOptionTest { - static enum ClickHouseTestOption implements ClickHouseOption { - STR("string_option", "string", "string option"), - STR0("string_option0", "string0", "string option without environment variable support"), - STR1("string_option0", "string1", - "string option without environment variable and system property support"), - INT("integer_option", 2333, "integer option"), - INT0("integer_option0", 23330, "integer option without environment variable support"), - INT1("integer_option1", 23331, - "integer option without environment variable and system property support"), - BOOL("boolean_option", false, "boolean option"), - BOOL0("boolean_option0", true, "boolean option without environment variable support"), - BOOL1("boolean_option1", false, - "boolean option without environment variable and system property support"); - - private final String key; - private final Serializable defaultValue; - private final Class clazz; - private final String description; - - ClickHouseTestOption(String key, T defaultValue, String description) { - this.key = ClickHouseChecker.nonNull(key, "key"); - this.defaultValue = defaultValue; - this.clazz = defaultValue.getClass(); - this.description = ClickHouseChecker.nonNull(description, "description"); - } - - @Override - public String getKey() { - return key; - } - - @Override - public Serializable getDefaultValue() { - return defaultValue; - } - - @Override - public Class getValueType() { - return clazz; - } - - @Override - public String getDescription() { - return description; - } + static enum ClickHouseTestOption implements ClickHouseOption { + STR("string_option", "string", "string option"), + STR0("string_option0", "string0", "string option without environment variable support"), + STR1("string_option0", "string1", + "string option without environment variable and system property support"), + INT("integer_option", 2333, "integer option"), + INT0("integer_option0", 23330, "integer option without environment variable support"), + INT1("integer_option1", 23331, + "integer option without environment variable and system property support"), + BOOL("boolean_option", false, "boolean option"), + BOOL0("boolean_option0", true, "boolean option without environment variable support"), + BOOL1("boolean_option1", false, + "boolean option without environment variable and system property support"); + + private final String key; + private final Serializable defaultValue; + private final Class clazz; + private final String description; + + ClickHouseTestOption(String key, T defaultValue, String description) { + this.key = ClickHouseChecker.nonNull(key, "key"); + this.defaultValue = defaultValue; + this.clazz = defaultValue.getClass(); + this.description = ClickHouseChecker.nonNull(description, "description"); } - @Test(groups = { "unit" }) - public void testFromString() { - Assert.assertThrows(IllegalArgumentException.class, - () -> ClickHouseOption.fromString(null, String.class)); - Assert.assertEquals(ClickHouseOption.fromString("", String.class), ""); - - Assert.assertEquals(ClickHouseOption.fromString("", Boolean.class), Boolean.FALSE); - Assert.assertEquals(ClickHouseOption.fromString("Yes", Boolean.class), Boolean.FALSE); - Assert.assertEquals(ClickHouseOption.fromString("1", Boolean.class), Boolean.TRUE); - Assert.assertEquals(ClickHouseOption.fromString("true", Boolean.class), Boolean.TRUE); - Assert.assertEquals(ClickHouseOption.fromString("True", Boolean.class), Boolean.TRUE); - - Assert.assertEquals(ClickHouseOption.fromString("", Integer.class), Integer.valueOf(0)); - Assert.assertEquals(ClickHouseOption.fromString("0", Integer.class), Integer.valueOf(0)); - Assert.assertThrows(IllegalArgumentException.class, - () -> ClickHouseOption.fromString(null, Integer.class)); - - Assert.assertEquals(ClickHouseOption.fromString("0.1", Float.class), Float.valueOf(0.1F)); - Assert.assertEquals(ClickHouseOption.fromString("NaN", Float.class), Float.valueOf(Float.NaN)); - - Assert.assertEquals(ClickHouseOption.fromString("Map", ClickHouseDataType.class), - ClickHouseDataType.Map); - Assert.assertEquals(ClickHouseOption.fromString("RowBinary", ClickHouseFormat.class), - ClickHouseFormat.RowBinary); - Assert.assertThrows(IllegalArgumentException.class, - () -> ClickHouseOption.fromString("NonExistFormat", ClickHouseFormat.class)); + @Override + public String getKey() { + return key; } - @Test(groups = { "unit" }) - public void testGetEffectiveDefaultValue() { - // environment variables are set in pom.xml - Assert.assertEquals(ClickHouseTestOption.STR.getEffectiveDefaultValue(), - ClickHouseTestOption.STR.getDefaultValueFromEnvVar().get()); - Assert.assertEquals(ClickHouseTestOption.INT.getEffectiveDefaultValue(), - Integer.parseInt(ClickHouseTestOption.INT.getDefaultValueFromEnvVar().get())); - Assert.assertEquals(ClickHouseTestOption.BOOL.getEffectiveDefaultValue(), - Boolean.valueOf(ClickHouseTestOption.BOOL.getDefaultValueFromEnvVar().get())); - - String sv = "system.property"; - int iv = 12345; - boolean bv = true; - System.setProperty(ClickHouseTestOption.STR0.getPrefix().toLowerCase() + "_" - + ClickHouseTestOption.STR0.name().toLowerCase(), sv); - System.setProperty(ClickHouseTestOption.INT0.getPrefix().toLowerCase() + "_" - + ClickHouseTestOption.INT0.name().toLowerCase(), String.valueOf(iv)); - System.setProperty(ClickHouseTestOption.BOOL0.getPrefix().toLowerCase() + "_" - + ClickHouseTestOption.BOOL0.name().toLowerCase(), String.valueOf(bv)); - - Assert.assertEquals(ClickHouseTestOption.STR0.getEffectiveDefaultValue(), sv); - Assert.assertEquals(ClickHouseTestOption.INT0.getEffectiveDefaultValue(), iv); - Assert.assertEquals(ClickHouseTestOption.BOOL0.getEffectiveDefaultValue(), bv); - - Assert.assertEquals(ClickHouseTestOption.STR1.getEffectiveDefaultValue(), - ClickHouseTestOption.STR1.getDefaultValue()); - Assert.assertEquals(ClickHouseTestOption.INT1.getEffectiveDefaultValue(), - ClickHouseTestOption.INT1.getDefaultValue()); - Assert.assertEquals(ClickHouseTestOption.BOOL1.getEffectiveDefaultValue(), - ClickHouseTestOption.BOOL1.getDefaultValue()); + @Override + public Serializable getDefaultValue() { + return defaultValue; } + + @Override + public Class getValueType() { + return clazz; + } + + @Override + public String getDescription() { + return description; + } + } + + @Test(groups = { "unit" }) + public void testFromString() { + Assert.assertThrows(IllegalArgumentException.class, + () -> ClickHouseOption.fromString(null, String.class)); + Assert.assertEquals(ClickHouseOption.fromString("", String.class), ""); + + Assert.assertEquals(ClickHouseOption.fromString("", Boolean.class), Boolean.FALSE); + Assert.assertEquals(ClickHouseOption.fromString("Yes", Boolean.class), Boolean.FALSE); + Assert.assertEquals(ClickHouseOption.fromString("1", Boolean.class), Boolean.TRUE); + Assert.assertEquals(ClickHouseOption.fromString("true", Boolean.class), Boolean.TRUE); + Assert.assertEquals(ClickHouseOption.fromString("True", Boolean.class), Boolean.TRUE); + + Assert.assertEquals(ClickHouseOption.fromString("", Integer.class), Integer.valueOf(0)); + Assert.assertEquals(ClickHouseOption.fromString("0", Integer.class), Integer.valueOf(0)); + Assert.assertThrows(IllegalArgumentException.class, + () -> ClickHouseOption.fromString(null, Integer.class)); + + Assert.assertEquals(ClickHouseOption.fromString("0.1", Float.class), Float.valueOf(0.1F)); + Assert.assertEquals(ClickHouseOption.fromString("NaN", Float.class), Float.valueOf(Float.NaN)); + + Assert.assertEquals(ClickHouseOption.fromString("Map", ClickHouseDataType.class), + ClickHouseDataType.Map); + Assert.assertEquals(ClickHouseOption.fromString("RowBinary", ClickHouseFormat.class), + ClickHouseFormat.RowBinary); + Assert.assertThrows(IllegalArgumentException.class, + () -> ClickHouseOption.fromString("NonExistFormat", ClickHouseFormat.class)); + } + + @Test(groups = { "unit" }) + public void testGetEffectiveDefaultValue() { + // environment variables are set in pom.xml + Assert.assertEquals(ClickHouseTestOption.STR.getEffectiveDefaultValue(), + ClickHouseTestOption.STR.getDefaultValueFromEnvVar().get()); + Assert.assertEquals(ClickHouseTestOption.INT.getEffectiveDefaultValue(), + Integer.parseInt(ClickHouseTestOption.INT.getDefaultValueFromEnvVar().get())); + Assert.assertEquals(ClickHouseTestOption.BOOL.getEffectiveDefaultValue(), + Boolean.valueOf(ClickHouseTestOption.BOOL.getDefaultValueFromEnvVar().get())); + + String sv = "system.property"; + int iv = 12345; + boolean bv = true; + System.setProperty(ClickHouseTestOption.STR0.getPrefix().toLowerCase() + "_" + + ClickHouseTestOption.STR0.name().toLowerCase(), sv); + System.setProperty(ClickHouseTestOption.INT0.getPrefix().toLowerCase() + "_" + + ClickHouseTestOption.INT0.name().toLowerCase(), String.valueOf(iv)); + System.setProperty(ClickHouseTestOption.BOOL0.getPrefix().toLowerCase() + "_" + + ClickHouseTestOption.BOOL0.name().toLowerCase(), String.valueOf(bv)); + + Assert.assertEquals(ClickHouseTestOption.STR0.getEffectiveDefaultValue(), sv); + Assert.assertEquals(ClickHouseTestOption.INT0.getEffectiveDefaultValue(), iv); + Assert.assertEquals(ClickHouseTestOption.BOOL0.getEffectiveDefaultValue(), bv); + + Assert.assertEquals(ClickHouseTestOption.STR1.getEffectiveDefaultValue(), + ClickHouseTestOption.STR1.getDefaultValue()); + Assert.assertEquals(ClickHouseTestOption.INT1.getEffectiveDefaultValue(), + ClickHouseTestOption.INT1.getDefaultValue()); + Assert.assertEquals(ClickHouseTestOption.BOOL1.getEffectiveDefaultValue(), + ClickHouseTestOption.BOOL1.getDefaultValue()); + } } From 5db1b84f6dea942dcefadcbe7d94b4f890c66809 Mon Sep 17 00:00:00 2001 From: Zhichun Wu Date: Tue, 30 Nov 2021 23:54:35 +0800 Subject: [PATCH 2/8] Set error code in grpc response --- .../client/grpc/ClickHouseGrpcClient.java | 29 ++++++++++--------- .../client/grpc/ClickHouseGrpcClientTest.java | 13 +++++++++ 2 files changed, 28 insertions(+), 14 deletions(-) diff --git a/clickhouse-grpc-client/src/main/java/com/clickhouse/client/grpc/ClickHouseGrpcClient.java b/clickhouse-grpc-client/src/main/java/com/clickhouse/client/grpc/ClickHouseGrpcClient.java index 779451288..7d79ffe0d 100644 --- a/clickhouse-grpc-client/src/main/java/com/clickhouse/client/grpc/ClickHouseGrpcClient.java +++ b/clickhouse-grpc-client/src/main/java/com/clickhouse/client/grpc/ClickHouseGrpcClient.java @@ -57,19 +57,19 @@ protected static Compression getResultCompression(ClickHouseConfig config) { CompressionAlgorithm algorithm = CompressionAlgorithm.DEFLATE; CompressionLevel level = CompressionLevel.COMPRESSION_MEDIUM; switch (config.getDecompressAlgorithmForClientRequest()) { - case NONE: - algorithm = CompressionAlgorithm.NO_COMPRESSION; - break; - case DEFLATE: - break; - case GZIP: - algorithm = CompressionAlgorithm.GZIP; - break; - // case STREAM_GZIP: - default: - log.warn("Unsupported algorithm [%s], change to [%s]", config.getDecompressAlgorithmForClientRequest(), - algorithm); - break; + case NONE: + algorithm = CompressionAlgorithm.NO_COMPRESSION; + break; + case DEFLATE: + break; + case GZIP: + algorithm = CompressionAlgorithm.GZIP; + break; + // case STREAM_GZIP: + default: + log.warn("Unsupported algorithm [%s], change to [%s]", config.getDecompressAlgorithmForClientRequest(), + algorithm); + break; } int l = config.getDecompressLevelForClientRequest(); @@ -269,7 +269,8 @@ protected CompletableFuture executeSync(ClickHouseRequest sealedRequest.getSettings(), result); return result.hasException() - ? failedResponse(ClickHouseException.of(result.getException().getDisplayText(), server)) + ? failedResponse(new ClickHouseException(result.getException().getCode(), + result.getException().getDisplayText(), server)) : CompletableFuture.completedFuture(response); } catch (IOException e) { throw new CompletionException(ClickHouseException.of(e, server)); 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 d44e8a9c4..a45edbbb9 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 @@ -277,6 +277,19 @@ public void testReadWriteDateTimeTypes() throws Exception { } } + @Test(groups = "integration") + public void testDropNonExistDb() throws Exception { + ClickHouseNode server = getServer(ClickHouseProtocol.GRPC); + + try { + ClickHouseClient.send(server, "drop database non_exist_db").get(); + Assert.fail("Exception is excepted"); + } catch (ExecutionException e) { + ClickHouseException ce = (ClickHouseException) e.getCause(); + Assert.assertEquals(ce.getErrorCode(), 81); + } + } + @Test(groups = "integration") public void testReadWriteDomains() throws Exception { ClickHouseNode server = getServer(ClickHouseProtocol.GRPC); From e353f0ce502073ad3dd6a4e5b37546c9d7125561 Mon Sep 17 00:00:00 2001 From: Zhichun Wu Date: Wed, 8 Dec 2021 20:11:16 +0800 Subject: [PATCH 3/8] enhance http client and jdbc driver --- README.md | 116 +++--- clickhouse-benchmark/pom.xml | 4 +- .../benchmark/jdbc/DriverState.java | 3 + .../com/clickhouse/benchmark/jdbc/Query.java | 23 +- .../client/ClickHouseClientBuilder.java | 2 +- .../client/ClickHouseInputStream.java | 348 +++++++++++++++++- .../client/ClickHouseParameterizedQuery.java | 82 +++-- .../clickhouse/client/ClickHouseRequest.java | 36 +- .../clickhouse/client/ClickHouseUtils.java | 2 +- .../clickhouse/client/ClickHouseValues.java | 1 - .../client/data/BinaryStreamUtils.java | 148 ++------ .../client/data/ClickHouseByteValue.java | 2 +- .../client/data/ClickHouseExternalTable.java | 22 +- .../client/data/ClickHouseLZ4InputStream.java | 185 +++++++--- .../client/data/ClickHousePipedStream.java | 170 +-------- .../data/ClickHouseRowBinaryProcessor.java | 9 +- .../client/data/ClickHouseShortValue.java | 2 +- .../client/data/ClickHouseTupleValue.java | 3 +- .../ClickHouseParameterizedQueryTest.java | 18 +- .../client/data/BinaryStreamUtilsTest.java | 8 + .../data/ClickHousePipedStreamTest.java | 13 +- clickhouse-grpc-client/pom.xml | 6 +- clickhouse-http-client/pom.xml | 2 +- .../client/http/ClickHouseHttpConnection.java | 34 +- .../client/http/ClickHouseHttpResponse.java | 50 ++- .../client/http/DefaultHttpConnection.java | 50 ++- .../http/ClickHouseResponseHandler.java | 87 +++++ .../client/http/DefaultHttpConnection.java | 246 +++++++++++++ .../src/main/java11/module-info.java | 12 + .../client/http/ClickHouseHttpClientTest.java | 3 +- .../http/ClickHouseHttpConnectionTest.java | 5 - clickhouse-jdbc/pom.xml | 12 +- .../clickhouse/jdbc/AbstractResultSet.java | 2 +- .../clickhouse/jdbc/ClickHouseConnection.java | 81 +++- .../clickhouse/jdbc/ClickHouseDataSource.java | 41 +-- .../jdbc/ClickHouseDatabaseMetaData.java | 116 +++--- .../com/clickhouse/jdbc/ClickHouseDriver.java | 7 +- .../jdbc/ClickHousePreparedStatement.java | 223 +++++++++++ .../clickhouse/jdbc/ClickHouseResultSet.java | 19 +- .../jdbc/ClickHouseResultSetMetaData.java | 2 +- .../java/com/clickhouse/jdbc/JdbcConfig.java | 138 +++++++ .../jdbc/JdbcParameterizedQuery.java | 9 +- .../com/clickhouse/jdbc/JdbcParseHandler.java | 15 +- .../com/clickhouse/jdbc/JdbcTypeMapping.java | 298 +++++++-------- .../jdbc/{Wrapper.java => JdbcWrapper.java} | 2 +- .../internal/ClickHouseConnectionImpl.java | 189 +++++----- .../internal/ClickHouseJdbcUrlParser.java | 41 ++- .../internal/ClickHouseParameterMetaData.java | 4 +- .../internal/ClickHouseStatementImpl.java | 95 +++-- ....java => InputBasedPreparedStatement.java} | 299 ++------------- .../internal/SqlBasedPreparedStatement.java | 325 +++------------- .../internal/TableBasedPreparedStatement.java | 262 +++++++++++++ .../jdbc/parser/ClickHouseSqlStatement.java | 63 +++- .../clickhouse/jdbc/parser/ParseHandler.java | 4 +- .../clickhouse/ClickHouseStatementImpl.java | 2 +- .../src/main/java9/module-info.java | 4 +- .../src/main/javacc/ClickHouseSqlParser.jj | 64 +++- .../jdbc/ClickHouseConnectionTest.java | 18 + .../jdbc/ClickHousePreparedStatementTest.java | 119 +++++- .../jdbc/ClickHouseResultSetTest.java | 81 ++++ .../jdbc/ClickHouseStatementTest.java | 36 ++ .../jdbc/CombinedResultSetTest.java | 16 +- .../clickhouse/jdbc/JdbcIntegrationTest.java | 8 +- .../jdbc/JdbcParameterizedQueryTest.java | 11 +- .../clickhouse/jdbc/JdbcParseHandlerTest.java | 155 +++++--- .../internal/ClickHouseJdbcUrlParserTest.java | 27 +- .../jdbc/parser/ClickHouseSqlParserTest.java | 57 ++- 67 files changed, 2905 insertions(+), 1632 deletions(-) create mode 100644 clickhouse-http-client/src/main/java11/com/clickhouse/client/http/ClickHouseResponseHandler.java create mode 100644 clickhouse-http-client/src/main/java11/com/clickhouse/client/http/DefaultHttpConnection.java create mode 100644 clickhouse-http-client/src/main/java11/module-info.java create mode 100644 clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHousePreparedStatement.java create mode 100644 clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcConfig.java rename clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/{Wrapper.java => JdbcWrapper.java} (92%) rename clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/{StreamBasedPreparedStatement.java => InputBasedPreparedStatement.java} (55%) create mode 100644 clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/TableBasedPreparedStatement.java create mode 100644 clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseConnectionTest.java create mode 100644 clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseResultSetTest.java diff --git a/README.md b/README.md index c06d9cebd..b7ba58216 100644 --- a/README.md +++ b/README.md @@ -1,86 +1,96 @@ # ClickHouse Java Client & JDBC Driver -[![clickhouse-jdbc](https://maven-badges.herokuapp.com/maven-central/ru.yandex.clickhouse/clickhouse-jdbc/badge.svg)](https://maven-badges.herokuapp.com/maven-central/ru.yandex.clickhouse/clickhouse-jdbc) ![Build Status(https://github.com/ClickHouse/clickhouse-jdbc/workflows/Build/badge.svg)](https://github.com/ClickHouse/clickhouse-jdbc/workflows/Build/badge.svg) [![Coverage](https://sonarcloud.io/api/project_badges/measure?project=ClickHouse_clickhouse-jdbc&metric=coverage)](https://sonarcloud.io/dashboard?id=ClickHouse_clickhouse-jdbc) +[![clickhouse-jdbc](https://maven-badges.herokuapp.com/maven-central/com.clickhouse/clickhouse-jdbc/badge.svg)](https://maven-badges.herokuapp.com/maven-central/com.clickhouse/clickhouse-jdbc) ![Build Status(https://github.com/ClickHouse/clickhouse-jdbc/workflows/Build/badge.svg)](https://github.com/ClickHouse/clickhouse-jdbc/workflows/Build/badge.svg) [![Coverage](https://sonarcloud.io/api/project_badges/measure?project=ClickHouse_clickhouse-jdbc&metric=coverage)](https://sonarcloud.io/dashboard?id=ClickHouse_clickhouse-jdbc) -Java client and JDBC driver for ClickHouse. +Java client and JDBC driver for ClickHouse. Java client is async and light weight library for accessing ClickHouse in Java; while JDBC driver is built on top of the Java client with more dependencies and extensions for JDBC-compliance. ## Usage ### Java Client -Use Java client when you prefer async and more "direct" way to communicate with ClickHouse. JDBC driver is actually a thin wrapper of the Java client. - ```xml com.clickhouse - - clickhouse-grpc-client + + clickhouse-http-client 0.3.2 ``` -Example: - -```Java -// declare a server to connect to -ClickHouseNode server = ClickHouseNode.of("server.domain", ClickHouseProtocol.GRPC, 9100, "my_db"); - -// run multiple queries in one go and wait until it's finished -ClickHouseClient.send(server, - "create database if not exists test", - "use test", // change current database from my_db to test - "create table if not exists test_table(s String) engine=Memory", - "insert into test_table values('1')('2')('3')", - "select * from test_table limit 1", - "truncate table test_table", - "drop table if exists test_table").get(); - -// query with named parameters -try (ClickHouseClient client = ClickHouseClient.newInstance(ClickHouseProtocol.GRPC); - ClickHouseResponse resp = client.connect(server) - .format(ClickHouseFormat.RowBinaryWithNamesAndTypes).set("send_logs_level", "trace") - .query("select id, name from some_table where id in :ids and name like :name").params(Arrays.asList(1,2,3), "%key%").execute().get()) { - // you can also use resp.stream() as well - for (ClickHouseRecord record : resp.records()) { - int id = record.getValue(0).asInteger(); - String name = record.getValue(1).asString(); +
+ Expand to see example... + + ```Java + // declare a server to connect to + ClickHouseNode server = ClickHouseNode.of("server.domain", ClickHouseProtocol.HTTP, 8123, "my_db"); + + // run multiple queries in one go and wait until they're completed + ClickHouseClient.send(server, "create database if not exists test", + "use test", // change current database from my_db to test + "create table if not exists test_table(s String) engine=Memory", + "insert into test_table values('1')('2')('3')", + "select * from test_table limit 1", + "truncate table test_table", + "drop table if exists test_table").get(); + + // query with named parameters + try (ClickHouseClient client = ClickHouseClient.newInstance(ClickHouseProtocol.GRPC); + ClickHouseResponse resp = client.connect(server) + .format(ClickHouseFormat.RowBinaryWithNamesAndTypes).set("send_logs_level", "trace") + .query("select id, name from some_table where id in :ids and name like :name").params(Arrays.asList(1,2,3), "%key%").execute().get()) { + // you can also use resp.stream() as well + for (ClickHouseRecord record : resp.records()) { + int id = record.getValue(0).asInteger(); + String name = record.getValue(1).asString(); + } + + ClickHouseResponseSummary summary = resp.getSummary(); + long totalRows = summary.getRows(); } - ClickHouseResponseSummary summary = resp.getSummary(); - long totalRows = summary.getRows(); -} + // load data with custom writer + ClickHouseClient.load(server, "target_table", ClickHouseFormat.TabSeparated, + ClickHouseCompression.NONE, new ClickHouseWriter() { + @Override + public void write(OutputStream output) throws IOException { + output.write("1\t\\N\n".getBytes()); + output.write("2\t123".getBytes()); + } + }).get(); + ``` +
-// load data with custom writer -ClickHouseClient.load(server, "target_table", ClickHouseFormat.TabSeparated, - ClickHouseCompression.NONE, new ClickHouseWriter() { - @Override - public void write(OutputStream output) throws IOException { - output.write("1\t\\N\n".getBytes()); - output.write("2\t123".getBytes()); - } - }).get(); -``` ### JDBC Driver ```xml - - ru.yandex.clickhouse + + com.clickhouse clickhouse-jdbc 0.3.2 + + http + + + * + * + + ``` -URL syntax: `jdbc:clickhouse://:[/[?param1=value1¶m2=value2]]`, e.g. `jdbc:clickhouse://localhost:8123/test?socket_timeout=120000` +URL Syntax: `jdbc:(clickhouse|ch)[:(grpc|http)]://:[][/[?param1=value1¶m2=value2]]` + - `jdbc:ch:grpc://localhost` is same as `jdbc:clickhouse:grpc://localhost:9100` + - `jdbc:ch://localhost/test?socket_timeout=120000` is same as `jdbc:clickhouse:http://localhost:8123/test?socket_timeout=120000` -JDBC Driver Class: `ru.yandex.clickhouse.ClickHouseDriver` (will be changed to `com.clickhouse.jdbc.ClickHouseDriver` starting from 0.4.0) +JDBC Driver Class: `com.clickhouse.jdbc.ClickHouseDriver` (will remove `ru.yandex.clickhouse.ClickHouseDriver` starting from 0.4.0) For example: ```java -String url = "jdbc:clickhouse://localhost:8123/test"; -ClickHouseProperties properties = new ClickHouseProperties(); +String url = "jdbc:ch://localhost/test"; +Properties properties = new Properties(); // set connection options - see more defined in ClickHouseConnectionSettings properties.setClientName("Agent #1"); ... @@ -96,13 +106,11 @@ additionalDBParams.put(ClickHouseQueryParam.SESSION_ID, "new-session-id"); ... try (ClickHouseConnection conn = dataSource.getConnection(); ClickHouseStatement stmt = conn.createStatement(); - ResultSet rs = stmt.executeQuery(sql, additionalDBParams)) { + ResultSet rs = stmt.executeQuery(sql)) { ... } ``` -Additionally, if you have a few instances, you can use `BalancedClickhouseDataSource`. - ### Extended API In order to provide non-JDBC complaint data manipulation functionality, proprietary API exists. @@ -174,7 +182,7 @@ Java 8 or higher is required in order to use Java client and/or JDBC driver. | \*String | Y | Y | | | UUID | Y | Y | | | AggregatedFunction | N | N | Partially supported | -| Array | Y | N | | +| Array | Y | Y | | | Map | Y | Y | | | Nested | Y | N | | | Tuple | Y | N | | diff --git a/clickhouse-benchmark/pom.xml b/clickhouse-benchmark/pom.xml index fe975208c..18d27c64c 100644 --- a/clickhouse-benchmark/pom.xml +++ b/clickhouse-benchmark/pom.xml @@ -17,7 +17,7 @@ 1.4.4 - 2.6.0 + 2.6.1 UTF-8 1.33 benchmarks @@ -138,7 +138,7 @@ *:* - module-info.class + **/module-info.class META-INF/MANIFEST.MF META-INF/*.SF META-INF/*.DSA diff --git a/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/jdbc/DriverState.java b/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/jdbc/DriverState.java index 681695f48..f3d38b3df 100644 --- a/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/jdbc/DriverState.java +++ b/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/jdbc/DriverState.java @@ -15,6 +15,7 @@ import com.clickhouse.benchmark.BaseState; import com.clickhouse.benchmark.Constants; import com.clickhouse.benchmark.ServerState; +// import com.github.housepower.settings.ClickHouseDefines; @State(Scope.Thread) public class DriverState extends BaseState { @@ -49,6 +50,8 @@ public void doSetup(ServerState serverState) throws Exception { url = String.format(jdbcDriver.getUrlTemplate(), serverState.getHost(), serverState.getPort(jdbcDriver.getDefaultPort()), serverState.getDatabase(), serverState.getUser(), serverState.getPassword(), compression); + // ClickHouseDefines.WRITE_COMPRESS = false; + // ClickHouseDefines.READ_DECOMPRESS = Boolean.parseBoolean(compression); conn = driver.connect(url, new Properties()); try (Statement s = conn.createStatement()) { diff --git a/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/jdbc/Query.java b/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/jdbc/Query.java index 8436ae4cb..09d2554a7 100644 --- a/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/jdbc/Query.java +++ b/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/jdbc/Query.java @@ -7,7 +7,7 @@ public class Query extends DriverBenchmark { @Benchmark - public void selectArrayOfInts(Blackhole blackhole, DriverState state) throws Throwable { + public void selectArrayOfUInt16(Blackhole blackhole, DriverState state) throws Throwable { int num = state.getRandomNumber(); int rows = state.getSampleSize() + num; ConsumeValueFunction func = state.getConsumeFunction((b, r, i) -> b.consume(r.getArray(i))); @@ -21,7 +21,7 @@ public void selectArrayOfInts(Blackhole blackhole, DriverState state) throws Thr } @Benchmark - public void selectMapOfInts(Blackhole blackhole, DriverState state) throws Throwable { + public void selectMapOfInt32(Blackhole blackhole, DriverState state) throws Throwable { int num = state.getRandomNumber(); int rows = state.getSampleSize() + num; ConsumeValueFunction func = state.getConsumeFunction((b, r, i) -> b.consume(r.getObject(i))); @@ -36,12 +36,12 @@ public void selectMapOfInts(Blackhole blackhole, DriverState state) throws Throw } @Benchmark - public void selectTupleOfInts(Blackhole blackhole, DriverState state) throws Throwable { + public void selectTupleOfInt16(Blackhole blackhole, DriverState state) throws Throwable { int num = state.getRandomNumber(); int rows = state.getSampleSize() + num; - ConsumeValueFunction func = state.getConsumeFunction((b, r, i) -> b.consume(r.getArray(i))); + ConsumeValueFunction func = state.getConsumeFunction((b, r, i) -> b.consume(r.getObject(i))); try (Statement stmt = executeQuery(state, - "select tuple(range(100, number % 600)) as v from numbers(?)", rows)) { + "select tuple(arrayMap(x -> cast(x as Int16), range(100, number % 600))) as v from numbers(?)", rows)) { ResultSet rs = stmt.getResultSet(); while (rs.next()) { func.consume(blackhole, rs, 1); @@ -103,6 +103,19 @@ public void selectUInt8(Blackhole blackhole, DriverState state) throws Throwable } } + @Benchmark + public void selectUuid(Blackhole blackhole, DriverState state) throws Throwable { + int num = state.getRandomNumber(); + int rows = state.getSampleSize() + num; + ConsumeValueFunction func = state.getConsumeFunction((b, r, i) -> b.consume(r.getString(i))); + try (Statement stmt = executeQuery(state, "select generateUUIDv4() as v from numbers(?)", rows)) { + ResultSet rs = stmt.getResultSet(); + while (rs.next()) { + func.consume(blackhole, rs, 1); + } + } + } + @Benchmark public void selectInt32(Blackhole blackhole, DriverState state) throws Throwable { int num = state.getRandomNumber(); 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 4008a86cf..01c191d4e 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseClientBuilder.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseClientBuilder.java @@ -97,7 +97,7 @@ public ClickHouseClient build() { boolean noSelector = nodeSelector == null || nodeSelector == ClickHouseNodeSelector.EMPTY; int counter = 0; - for (ClickHouseClient c : ServiceLoader.load(ClickHouseClient.class)) { + for (ClickHouseClient c : ServiceLoader.load(ClickHouseClient.class, getClass().getClassLoader())) { counter++; if (noSelector || nodeSelector.match(c)) { client = c; 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 1f4601b50..b956f040e 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseInputStream.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseInputStream.java @@ -3,24 +3,198 @@ 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 java.util.concurrent.BlockingQueue; +import java.util.concurrent.TimeUnit; /** * Extended input stream. */ public abstract class ClickHouseInputStream extends InputStream { - static final class SimpleInputStream extends ClickHouseInputStream { - private final InputStream in; + /** + * Empty and read-only byte buffer. + */ + public static final ByteBuffer EMPTY = ByteBuffer.wrap(new byte[0]).asReadOnlyBuffer(); + static final class BlockingInputStream extends ClickHouseInputStream { + private final BlockingQueue queue; + private final int timeout; + + // too much to maintain a 2-level buffer for reading? + private ByteBuffer buffer; private boolean closed; - protected SimpleInputStream(InputStream input) { - this.in = ClickHouseChecker.nonNull(input, "InputStream"); + BlockingInputStream(BlockingQueue queue, int timeout) { + this.queue = queue; + this.timeout = timeout; + + this.buffer = null; this.closed = false; } + private void ensureOpen() throws IOException { + if (closed) { + throw new IOException("Stream has been closed"); + } + + if (buffer == null || (buffer != EMPTY && buffer.limit() > 0 && !buffer.hasRemaining())) { + updateBuffer(); + } + } + + private int updateBuffer() throws IOException { + try { + if (timeout > 0) { + buffer = queue.poll(timeout, TimeUnit.MILLISECONDS); + if (buffer == null) { + throw new IOException(ClickHouseUtils.format("Read timed out after %d ms", timeout)); + } + } else { + buffer = queue.take(); + } + + return buffer.remaining(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException("Thread was interrupted when getting next buffer from queue", e); + } + } + @Override public int available() throws IOException { - return in.available(); + if (closed || buffer == EMPTY) { + return 0; + } + + int available = 0; + if (buffer == null || (buffer.limit() > 0 && !buffer.hasRemaining())) { + available = updateBuffer(); + } else { + available = buffer.remaining(); + } + return available; + } + + @Override + public boolean isClosed() { + return closed; + } + + @Override + public void close() throws IOException { + // it's caller's responsiblity to consume all data in the queue, which will + // unblock writer + closed = true; + buffer = null; + } + + @Override + public byte readByte() throws IOException { + ensureOpen(); + + if (buffer == EMPTY || buffer.limit() == 0) { + close(); + throw new EOFException(); + } + + return buffer.get(); + } + + @Override + public int read() throws IOException { + ensureOpen(); + + if (buffer == EMPTY || buffer.limit() == 0) { + return -1; + } + + return 0xFF & buffer.get(); + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + ensureOpen(); + + int counter = 0; + while (len > 0) { + if (buffer == EMPTY || buffer.limit() == 0) { + return counter > 0 ? counter : -1; + } + + int remain = buffer.remaining(); + if (remain >= len) { + buffer.get(b, off, len); + counter += len; + len = 0; + } else { + buffer.get(b, off, remain); + counter += remain; + off += remain; + len -= remain; + + updateBuffer(); + } + } + + return counter; + } + + @Override + public String readString(int byteLength, Charset charset) throws IOException { + ensureOpen(); + + if (byteLength < 1) { + return ""; + } + + if (charset == null) { + charset = StandardCharsets.UTF_8; + } + + if (!buffer.isReadOnly() && byteLength > 8 && buffer.remaining() > byteLength) { + int pos = buffer.position(); + ((Buffer) buffer).position(pos + byteLength); + return charset.decode(ByteBuffer.wrap(buffer.array(), pos, byteLength)).toString(); + } + + return new String(readBytes(byteLength), charset); + } + + @Override + public long skip(long n) throws IOException { + ensureOpen(); + + // peforms better but this is a bit tricky + if (n == Long.MAX_VALUE) { + long counter = buffer.remaining(); + while (buffer != EMPTY && buffer.limit() > 0) { + counter += buffer.limit(); + updateBuffer(); + } + + return counter; + } + + return super.skip(n); + } + } + + static final class WrappedInputStream extends ClickHouseInputStream { + private final InputStream in; + + private boolean closed; + + WrappedInputStream(InputStream input) { + in = ClickHouseChecker.nonNull(input, "InputStream"); + closed = false; + } + + @Override + public int available() throws IOException { + return !closed ? in.available() : 0; } @Override @@ -41,8 +215,11 @@ public boolean isClosed() { @Override public void close() throws IOException { - in.close(); - closed = true; + try { + in.close(); + } finally { + closed = true; + } } @Override @@ -61,8 +238,25 @@ public long skip(long n) throws IOException { } } + /** + * Wraps the given blocking queue. + * + * @param queue non-null blocking queue + * @param timeout read timeout in milliseconds + * @return wrapped input + */ + public static ClickHouseInputStream of(BlockingQueue queue, int timeout) { + return new BlockingInputStream(ClickHouseChecker.nonNull(queue, "queue"), timeout); + } + + /** + * Wraps the given input stream. + * + * @param input non-null input stream + * @return wrapped input + */ public static ClickHouseInputStream of(InputStream input) { - return input instanceof ClickHouseInputStream ? (ClickHouseInputStream) input : new SimpleInputStream(input); + return input instanceof ClickHouseInputStream ? (ClickHouseInputStream) input : new WrappedInputStream(input); } /** @@ -77,7 +271,7 @@ public int readUnsignedByte() throws IOException { } /** - * Reads one single byte from the input stream. This is faster than + * Reads one single byte from the input stream. It's supposed to be faster than * {@link #read()}. * * @return byte value if present @@ -86,6 +280,142 @@ public int readUnsignedByte() throws IOException { */ public abstract byte readByte() throws IOException; + /** + * Reads {@code length} bytes from the input stream. It behaves in the same + * way as {@link java.io.DataInput#readFully(byte[])}. + * + * @param length number of bytes to read + * @return byte array and its length should be {@code length} + * @throws IOException when failed to read value from input stream, not able to + * retrieve all bytes, or reached end of the stream + */ + public byte[] readBytes(int length) throws IOException { + byte[] bytes = new byte[length]; + + for (int l = length, c = 0, n = 0; l > 0; l -= n) { + n = read(bytes, c, l); + if (n != -1) { + c += n; + } else { + try { + close(); + } catch (IOException e) { + // ignore + } + + throw c == 0 ? new EOFException() + : new IOException(ClickHouseUtils + .format("Reached end of input stream after reading %d of %d bytes", c, length)); + } + } + + return bytes; + } + + /** + * Reads string from the input stream. {@link #readVarInt()} will be called + * automatically to understand byte length of the string. + * + * @param charset charset, null is treated as {@link StandardCharsets#UTF_8} + * @return non-null string + * @throws IOException when failed to read value from input stream, not able to + * retrieve all bytes, or reached end of the stream + */ + public String readString(Charset charset) throws IOException { + return readString(readVarInt(), charset); + } + + /** + * Reads string from the input stream. When {@code byteLength} is zero or + * negative number, this method will always return empty string. + * + * @param byteLength length in byte + * @param charset charset, null is treated as {@link StandardCharsets#UTF_8} + * @return non-null string + * @throws IOException when failed to read value from input stream, not able to + * retrieve all bytes, or reached end of the stream + */ + public String readString(int byteLength, Charset charset) throws IOException { + if (byteLength < 1) { + return ""; + } + + return new String(readBytes(byteLength), charset != null ? charset : StandardCharsets.UTF_8); + } + + /** + * Reads ascii string from input stream. {@link #readVarInt()} will be called + * automatically to understand byte length of the string. + * + * @return non-null ascii string + * @throws IOException when failed to read value from input stream, not able to + * retrieve all bytes, or reached end of the stream + */ + public String readAsciiString() throws IOException { + return readString(readVarInt(), StandardCharsets.US_ASCII); + } + + /** + * Reads ascii string from input stream. Similar as + * {@code readString(byteLength, StandardCharsets.US_ASCII)}. + * + * @param byteLength length in byte + * @return non-null ascii string + * @throws IOException when failed to read value from input stream, not able to + * retrieve all bytes, or reached end of the stream + */ + public String readAsciiString(int byteLength) throws IOException { + return readString(byteLength, StandardCharsets.US_ASCII); + } + + /** + * Reads unicode string from input stream. + * + * @return non-null unicode string + * @throws IOException when failed to read value from input stream, not able to + * retrieve all bytes, or reached end of the stream + */ + public String readUnicodeString() throws IOException { + return readString(readVarInt(), StandardCharsets.UTF_8); + } + + /** + * Reads unicode string from input stream. Similar as + * {@code readString(byteLength, null)}. + * + * @param byteLength length in byte + * @return non-null unicode string + * @throws IOException when failed to read value from input stream, not able to + * retrieve all bytes, or reached end of the stream + */ + public String readUnicodeString(int byteLength) throws IOException { + return readString(byteLength, StandardCharsets.UTF_8); + } + + /** + * Read varint from input stream. + * + * @return varint + * @throws IOException when failed to read value from input stream or reached + * end of the stream + */ + public int readVarInt() throws IOException { + // https://github.com/ClickHouse/ClickHouse/blob/abe314feecd1647d7c2b952a25da7abf5c19f352/src/IO/VarInt.h#L126 + long result = 0L; + int shift = 0; + for (int i = 0; i < 9; i++) { + // gets 7 bits from next byte + byte b = readByte(); + result |= (b & 0x7F) << shift; + if ((b & 0x80) == 0) { + break; + } + shift += 7; + } + + return (int) result; + } + /** * Checks if the input stream has been closed or not. * 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 51278a7da..fd29077a0 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseParameterizedQuery.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseParameterizedQuery.java @@ -2,16 +2,14 @@ import java.io.Serializable; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.Iterator; -import java.util.LinkedHashSet; +import java.util.LinkedHashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Set; /** * A parameterized query is a parsed query with parameters being extracted for @@ -30,17 +28,23 @@ public class ClickHouseParameterizedQuery implements Serializable { * A part of query. */ protected static class QueryPart implements Serializable { - protected final String part; - protected final int paramIndex; - protected final String paramName; - protected final ClickHouseColumn paramType; + public final String part; + public final int paramIndex; + public final String paramName; + public final ClickHouseColumn paramType; - protected QueryPart(String part, int paramIndex, String paramName, String paramType) { + protected QueryPart(String part, int paramIndex, String paramName, String paramType, + Map map) { this.part = part; this.paramIndex = paramIndex; this.paramName = paramName != null ? paramName : String.valueOf(paramIndex); - // what should be default? ClickHouseAnyValue(simply convert object to string)? - this.paramType = paramType != null ? ClickHouseColumn.of("", paramType) : null; + if (paramType != null) { + this.paramType = ClickHouseColumn.of("", paramType); + map.put(paramName, ClickHouseValues.newValue(this.paramType)); + } else { + this.paramType = null; + map.putIfAbsent(paramName, null); + } } @Override @@ -74,7 +78,8 @@ public boolean equals(Object obj) { * Substitute named parameters in given SQL. * * @param sql SQL containing named parameters - * @param params mapping between parameter name and correspoding SQL expression + * @param params mapping between parameter name and correspoding SQL + * expression(NOT raw value) * @return substituted SQL, or the given sql if one of {@code sql} and * {@code params} is null or empty */ @@ -152,7 +157,7 @@ public static ClickHouseParameterizedQuery of(String query) { protected final String originalQuery; private final List parts; - private final Set names; + private final Map names; private final String lastPart; /** @@ -164,7 +169,7 @@ protected ClickHouseParameterizedQuery(String query) { originalQuery = ClickHouseChecker.nonBlank(query, "query"); parts = new LinkedList<>(); - names = new LinkedHashSet<>(); + names = new LinkedHashMap<>(); lastPart = parse(); } @@ -193,8 +198,8 @@ protected void addPart(String part, int paramIndex, String paramName, String par if (paramName == null) { paramName = String.valueOf(paramIndex); } - parts.add(new QueryPart(part, paramIndex, paramName, paramType)); - names.add(paramName); + + parts.add(new QueryPart(part, paramIndex, paramName, paramType, names)); } /** @@ -252,12 +257,12 @@ protected String parse() { if (builder.length() > 0) { paramName = builder.toString(); - if (names.add(paramName)) { + if (!names.containsKey(paramName)) { paramIndex++; } } - parts.add(new QueryPart(part, paramIndex, paramName, paramType)); + parts.add(new QueryPart(part, paramIndex, paramName, paramType, names)); } } } @@ -266,6 +271,19 @@ protected String parse() { return partIndex < len ? originalQuery.substring(partIndex, len) : null; } + /** + * Converts given raw value to SQL expression. + * + * @param paramName name of the parameter + * @param value raw value, could be null + * @return non-null SQL expression + */ + protected String toSqlExpression(String paramName, Object value) { + ClickHouseValue template = names.get(paramName); + return template != null ? template.update(value).toSqlExpression() + : ClickHouseValues.convertToSqlExpression(value); + } + /** * Applies stringified parameters to the query. * @@ -341,7 +359,7 @@ public String apply(Object param, Object... more) { if (index > 0) { param = index < len ? more[index - 1] : null; } - builder.append(ClickHouseValues.convertToSqlExpression(param)); + builder.append(toSqlExpression(p.paramName, param)); index++; } @@ -370,7 +388,7 @@ public String apply(Object[] values) { for (QueryPart p : parts) { builder.append(p.part); builder.append( - index < len ? ClickHouseValues.convertToSqlExpression(values[index]) : ClickHouseValues.NULL_EXPR); + index < len ? toSqlExpression(p.paramName, values[index]) : ClickHouseValues.NULL_EXPR); index++; } @@ -441,8 +459,16 @@ public String apply(String[] values) { * * @return list of named parameters */ - public List getNamedParameters() { - return names.isEmpty() ? Collections.emptyList() : Arrays.asList(names.toArray(new String[0])); + public List getParameters() { + if (names.isEmpty()) { + return Collections.emptyList(); + } + + List list = new ArrayList<>(names.size()); + for (String n : names.keySet()) { + list.add(n); + } + return Collections.unmodifiableList(list); } /** @@ -474,6 +500,20 @@ public List getQueryParts() { return queryParts; } + /** + * Gets parameter templates for converting value to SQL expression. + * + * @return parameter templates + */ + public ClickHouseValue[] getParameterTemplates() { + int i = 0; + ClickHouseValue[] tempaltes = new ClickHouseValue[names.size()]; + for (ClickHouseValue v : names.values()) { + tempaltes[i++] = v; + } + return tempaltes; + } + /** * Checks if the query has at least one parameter or not. * diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseRequest.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseRequest.java index eb14a9fd1..8e85458ee 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseRequest.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseRequest.java @@ -229,14 +229,6 @@ protected ClickHouseClient getClient() { return client; } - protected ClickHouseParameterizedQuery getPreparedQuery() { - if (preparedQuery == null) { - preparedQuery = ClickHouseParameterizedQuery.of(getQuery()); - } - - return preparedQuery; - } - /** * Gets query, either set by {@code query()} or {@code table()}. * @@ -367,6 +359,20 @@ public Optional getQueryId() { return ClickHouseChecker.isNullOrEmpty(queryId) ? Optional.empty() : Optional.of(queryId); } + /** + * Gets prepared query, which is a loosely parsed query with the origianl query + * and list of parameters. + * + * @return prepared query + */ + public ClickHouseParameterizedQuery getPreparedQuery() { + if (preparedQuery == null) { + preparedQuery = ClickHouseParameterizedQuery.of(getQuery()); + } + + return preparedQuery; + } + /** * Gets immutable settings. * @@ -731,7 +737,7 @@ public SelfT params(Collection values) { namedParameters.clear(); if (values != null && !values.isEmpty()) { - List names = getPreparedQuery().getNamedParameters(); + List names = getPreparedQuery().getParameters(); int size = names.size(); int index = 0; for (String v : values) { @@ -763,7 +769,7 @@ public SelfT params(ClickHouseValue value, ClickHouseValue... more) { namedParameters.clear(); if (value != null) { // it doesn't make sense to pass null as first parameter - List names = getPreparedQuery().getNamedParameters(); + List names = getPreparedQuery().getParameters(); int size = names.size(); int index = 0; @@ -800,7 +806,7 @@ public SelfT params(ClickHouseValue[] values) { namedParameters.clear(); if (values != null && values.length > 0) { - List names = getPreparedQuery().getNamedParameters(); + List names = getPreparedQuery().getParameters(); int size = names.size(); int index = 0; for (ClickHouseValue v : values) { @@ -832,7 +838,7 @@ public SelfT params(String value, String... more) { namedParameters.clear(); - List names = getPreparedQuery().getNamedParameters(); + List names = getPreparedQuery().getParameters(); int size = names.size(); int index = 0; namedParameters.put(names.get(index++), value); @@ -867,7 +873,7 @@ public SelfT params(String[] values) { namedParameters.clear(); if (values != null && values.length > 0) { - List names = getPreparedQuery().getNamedParameters(); + List names = getPreparedQuery().getParameters(); int size = names.size(); int index = 0; for (String v : values) { @@ -899,7 +905,7 @@ public SelfT params(Object value, Object... more) { namedParameters.clear(); - List names = getPreparedQuery().getNamedParameters(); + List names = getPreparedQuery().getParameters(); int size = names.size(); int index = 0; namedParameters.put(names.get(index++), ClickHouseValues.convertToSqlExpression(value)); @@ -934,7 +940,7 @@ public SelfT params(Object[] values) { namedParameters.clear(); if (values != null && values.length > 0) { - List names = getPreparedQuery().getNamedParameters(); + List names = getPreparedQuery().getParameters(); int size = names.size(); int index = 0; for (Object v : values) { 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 da06b8a4e..7e21e9dbc 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseUtils.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseUtils.java @@ -99,7 +99,7 @@ private static T findFirstService(Class serviceInterface) { T service = null; - for (T s : ServiceLoader.load(serviceInterface)) { + for (T s : ServiceLoader.load(serviceInterface, ClickHouseUtils.class.getClassLoader())) { if (s != null) { service = s; break; diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseValues.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseValues.java index 746302f84..6f55d18e1 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseValues.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseValues.java @@ -1,7 +1,6 @@ package com.clickhouse.client; import java.io.IOException; -import java.io.InputStream; import java.lang.reflect.Array; import java.math.BigDecimal; import java.math.BigInteger; 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 49d9941ce..e1725e3f7 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 @@ -20,6 +20,7 @@ import java.time.LocalTime; import java.time.ZoneOffset; import java.util.TimeZone; +import java.util.UUID; import java.util.concurrent.TimeUnit; import com.clickhouse.client.ClickHouseChecker; import com.clickhouse.client.ClickHouseDataType; @@ -86,6 +87,18 @@ private static > T toEnum(int value, Class enumType) { ClickHouseUtils.format("Enum [%s] does not contain value [%d]", enumType, value)); } + 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); + } + + 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); + } + /** * Reverse the given byte array. * @@ -93,11 +106,13 @@ private static > T toEnum(int value, Class enumType) { * @return same byte array but reserved */ public static byte[] reverse(byte[] bytes) { - if (bytes != null && bytes.length > 1) { - for (int i = 0, len = bytes.length / 2; i < len; i++) { + int l = bytes != null ? bytes.length : 0; + if (l > 1) { + for (int i = 0, len = l / 2; i < len; i++) { byte b = bytes[i]; - bytes[i] = bytes[bytes.length - 1 - i]; - bytes[bytes.length - 1 - i] = b; + --l; + bytes[i] = bytes[l]; + bytes[l] = b; } } @@ -136,38 +151,6 @@ public static int getVarLongSize(long value) { return result; } - /** - * Reads {@code length} bytes from given input stream. It behaves in the same - * way as {@link java.io.DataInput#readFully(byte[])}. - * - * @param input non-null input stream - * @param length number of bytes to read - * @return byte array and its length should be {@code length} - * @throws IOException when failed to read value from input stream, not able to - * retrieve all bytes, or reached end of the stream - */ - public static byte[] readBytes(ClickHouseInputStream input, int length) throws IOException { - int count = 0; - byte[] bytes = new byte[length]; - while (count < length) { - int n = input.read(bytes, count, length - count); - if (n < 0) { - try { - input.close(); - } catch (IOException e) { - // ignore - } - - throw count == 0 ? new EOFException() - : new IOException(ClickHouseUtils - .format("Reached end of input stream after reading %d of %d bytes", count, length)); - } - count += n; - } - - return bytes; - } - /** * Writes bytes into given output stream. * @@ -579,7 +562,7 @@ public static void writeNonNull(OutputStream output) throws IOException { * end of the stream */ public static Inet4Address readInet4Address(ClickHouseInputStream input) throws IOException { - return (Inet4Address) InetAddress.getByAddress(reverse(readBytes(input, 4))); + return (Inet4Address) InetAddress.getByAddress(reverse(input.readBytes(4))); } /** @@ -603,7 +586,7 @@ public static void writeInet4Address(OutputStream output, Inet4Address value) th * end of the stream */ public static Inet6Address readInet6Address(ClickHouseInputStream input) throws IOException { - return Inet6Address.getByAddress(null, readBytes(input, 16), null); + return Inet6Address.getByAddress(null, input.readBytes(16), null); } /** @@ -688,8 +671,7 @@ public static void writeUnsignedInt8(OutputStream output, int value) throws IOEx * end of the stream */ public static short readInt16(ClickHouseInputStream input) throws IOException { - byte[] bytes = readBytes(input, 2); - return (short) ((0xFF & bytes[0]) | (bytes[1] << 8)); + return (short) (input.readUnsignedByte() | (input.readByte() << 8)); } /** @@ -751,9 +733,8 @@ public static void writeUnsignedInt16(OutputStream output, int value) throws IOE * end of the stream */ public static int readInt32(ClickHouseInputStream input) throws IOException { - byte[] bytes = readBytes(input, 4); - - return (0xFF & bytes[0]) | ((0xFF & bytes[1]) << 8) | ((0xFF & bytes[2]) << 16) | (bytes[3] << 24); + return input.readUnsignedByte() | (input.readUnsignedByte() << 8) | (input.readUnsignedByte() << 16) + | (input.readByte() << 24); } /** @@ -803,11 +784,7 @@ public static void writeUnsignedInt32(OutputStream output, long value) throws IO * end of the stream */ public static long readInt64(ClickHouseInputStream input) throws IOException { - byte[] bytes = readBytes(input, 8); - - return (0xFFL & bytes[0]) | ((0xFFL & bytes[1]) << 8) | ((0xFFL & bytes[2]) << 16) | ((0xFFL & bytes[3]) << 24) - | ((0xFFL & bytes[4]) << 32) | ((0xFFL & bytes[5]) << 40) | ((0xFFL & bytes[6]) << 48) - | ((0xFFL & bytes[7]) << 56); + return toInt64(input.readBytes(8), 0); } /** @@ -839,7 +816,7 @@ public static void writeInt64(OutputStream output, long value) throws IOExceptio * end of the stream */ public static BigInteger readUnsignedInt64(ClickHouseInputStream input) throws IOException { - return new BigInteger(1, reverse(readBytes(input, 8))); + return new BigInteger(1, reverse(input.readBytes(8))); } /** @@ -879,7 +856,7 @@ public static void writeUnsignedInt64(OutputStream output, BigInteger value) thr * end of the stream */ public static BigInteger readInt128(ClickHouseInputStream input) throws IOException { - return new BigInteger(reverse(readBytes(input, 16))); + return new BigInteger(reverse(input.readBytes(16))); } /** @@ -903,7 +880,7 @@ public static void writeInt128(OutputStream output, BigInteger value) throws IOE * end of the stream */ public static BigInteger readUnsignedInt128(ClickHouseInputStream input) throws IOException { - return new BigInteger(1, reverse(readBytes(input, 16))); + return new BigInteger(1, reverse(input.readBytes(16))); } /** @@ -928,7 +905,7 @@ public static void writeUnsignedInt128(OutputStream output, BigInteger value) th * end of the stream */ public static BigInteger readInt256(ClickHouseInputStream input) throws IOException { - return new BigInteger(reverse(readBytes(input, 32))); + return new BigInteger(reverse(input.readBytes(32))); } /** @@ -952,7 +929,7 @@ public static void writeInt256(OutputStream output, BigInteger value) throws IOE * end of the stream */ public static BigInteger readUnsignedInt256(ClickHouseInputStream input) throws IOException { - return new BigInteger(1, reverse(readBytes(input, 32))); + return new BigInteger(1, reverse(input.readBytes(32))); } /** @@ -1024,8 +1001,9 @@ public static void writeFloat64(OutputStream output, double value) throws IOExce * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static java.util.UUID readUuid(ClickHouseInputStream input) throws IOException { - return new java.util.UUID(readInt64(input), readInt64(input)); + public static UUID readUuid(ClickHouseInputStream input) throws IOException { + byte[] bytes = input.readBytes(16); + return new UUID(toInt64(bytes, 0), toInt64(bytes, 8)); } /** @@ -1517,7 +1495,7 @@ public static String readFixedString(ClickHouseInputStream input, int length) th * end of the stream */ public static String readFixedString(ClickHouseInputStream input, int length, Charset charset) throws IOException { - byte[] bytes = readBytes(input, length); + byte[] bytes = input.readBytes(length); return new String(bytes, charset == null ? StandardCharsets.UTF_8 : charset); } @@ -1556,45 +1534,6 @@ public static void writeFixedString(OutputStream output, String value, int lengt output.write(bytes); } - /** - * Read string from given input stream. - * - * @param input non-null input stream - * @return string value - * @throws IOException when failed to read value from input stream or reached - * end of the stream - */ - public static String readString(ClickHouseInputStream input) throws IOException { - return readString(input, readVarInt(input), null); - } - - /** - * Read string from given input stream. - * - * @param input non-null input stream - * @param charset charset used to convert byte array to string, null means UTF-8 - * @return string value - * @throws IOException when failed to read value from input stream or reached - * end of the stream - */ - public static String readString(ClickHouseInputStream input, Charset charset) throws IOException { - return readString(input, readVarInt(input), charset); - } - - /** - * Reads fixed string from given input stream. - * - * @param input non-null input stream - * @param length length in byte - * @param charset charset used to convert byte array to string, null means UTF-8 - * @return string value - * @throws IOException when failed to read value from input stream or reached - * end of the stream - */ - public static String readString(ClickHouseInputStream input, int length, Charset charset) throws IOException { - return new String(readBytes(input, length), charset == null ? StandardCharsets.UTF_8 : charset); - } - /** * Reads characters from given reader. * @@ -1644,23 +1583,6 @@ public static void writeString(OutputStream output, String value, Charset charse * @throws IOException when failed to read value from input stream or reached * end of the stream */ - public static int readVarInt(ClickHouseInputStream input) throws IOException { - // https://github.com/ClickHouse/ClickHouse/blob/abe314feecd1647d7c2b952a25da7abf5c19f352/src/IO/VarInt.h#L126 - long result = 0L; - int shift = 0; - for (int i = 0; i < 9; i++) { - // gets 7 bits from next byte - int b = input.readUnsignedByte(); - result |= (b & 0x7F) << shift; - if ((b & 0x80) == 0) { - break; - } - shift += 7; - } - - return (int) result; - } - public static int readVarInt(InputStream input) throws IOException { // https://github.com/ClickHouse/ClickHouse/blob/abe314feecd1647d7c2b952a25da7abf5c19f352/src/IO/VarInt.h#L126 long result = 0L; @@ -1697,7 +1619,7 @@ public static int readVarInt(ByteBuffer buffer) { int shift = 0; for (int i = 0; i < 9; i++) { // gets 7 bits from next byte - int b = buffer.get(); + byte b = buffer.get(); result |= (b & 0x7F) << shift; if ((b & 0x80) == 0) { break; diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseByteValue.java b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseByteValue.java index 3e9717612..12005dad5 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseByteValue.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseByteValue.java @@ -84,7 +84,7 @@ protected ClickHouseByteValue(boolean isNull, byte value) { protected ClickHouseByteValue set(boolean isNull, byte value) { this.isNull = isNull; - this.value = isNull ? 0 : value; + this.value = isNull ? (byte) 0 : value; return this; } diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseExternalTable.java b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseExternalTable.java index e7af11102..5f40235b5 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseExternalTable.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseExternalTable.java @@ -21,6 +21,7 @@ public static class Builder { private CompletableFuture content; private ClickHouseFormat format; private List columns; + private boolean asTempTable; protected Builder() { columns = new LinkedList<>(); @@ -88,8 +89,18 @@ public Builder columns(Collection columns) { return this; } + public Builder asTempTable() { + asTempTable = true; + return this; + } + + public Builder asExternalTable() { + asTempTable = false; + return this; + } + public ClickHouseExternalTable build() { - return new ClickHouseExternalTable(name, content, format, columns); + return new ClickHouseExternalTable(name, content, format, columns, asTempTable); } } @@ -101,11 +112,12 @@ public static Builder builder() { private final CompletableFuture content; private final ClickHouseFormat format; private final List columns; + private final boolean asTempTable; private final String structure; protected ClickHouseExternalTable(String name, CompletableFuture content, ClickHouseFormat format, - Collection columns) { + Collection columns, boolean asTempTable) { this.name = name == null ? "" : name.trim(); this.content = ClickHouseChecker.nonNull(content, "content"); this.format = format == null ? ClickHouseFormat.TabSeparated : format; @@ -124,6 +136,8 @@ protected ClickHouseExternalTable(String name, CompletableFuture co this.columns = Collections.unmodifiableList(list); this.structure = builder.deleteCharAt(builder.length() - 1).toString(); } + + this.asTempTable = asTempTable; } public boolean hasName() { @@ -153,6 +167,10 @@ public List getColumns() { return columns; } + public boolean isTempTable() { + return asTempTable; + } + public String getStructure() { return structure; } 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 6bb62f356..cf835187d 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,8 +3,13 @@ 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 net.jpountz.lz4.LZ4Factory; import net.jpountz.lz4.LZ4FastDecompressor; @@ -12,15 +17,63 @@ /** * Reader from clickhouse in lz4. */ -public class ClickHouseLZ4InputStream extends InputStream { +public class ClickHouseLZ4InputStream extends ClickHouseInputStream { private static final LZ4Factory factory = LZ4Factory.fastestInstance(); static final int MAGIC = 0x82; private final InputStream stream; - private byte[] currentBlock; - private int pointer; + private ByteBuffer currentBlock; + private boolean closed; + + private boolean checkNext() throws IOException { + if (currentBlock == null) { + currentBlock = readNextBlock(); + } + return currentBlock != null && currentBlock.hasRemaining(); + } + + // every block is: + private ByteBuffer readNextBlock() throws IOException { + int read = stream.read(); + if (read < 0) { + return null; + } + + 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); + } + + readFully(bytes, 0, 8); + // 4 bytes - size of the compressed data including 9 bytes of the header + int compressedSizeWithHeader = BinaryStreamUtils.toInt32(bytes, 0); + // 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)) { + 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); + } private void readFully(byte b[], int off, int len) throws IOException { if (len < 0) { @@ -30,15 +83,25 @@ private void readFully(byte b[], int off, int len) throws IOException { while (n < len) { int count = stream.read(b, off + n, len - n); if (count < 0) { + try { + close(); + } catch (IOException e) { + // ignore + } throw new EOFException(); } n += count; } } - private int readUnsignedByte() throws IOException { + private int readUnsignedByteFromInput() throws IOException { int ch = stream.read(); if (ch < 0) { + try { + close(); + } catch (IOException e) { + // ignore + } throw new EOFException(); } return ch; @@ -46,20 +109,39 @@ private int readUnsignedByte() throws IOException { public ClickHouseLZ4InputStream(InputStream stream) { this.stream = ClickHouseChecker.nonNull(stream, "InputStream"); + this.closed = false; + } + + @Override + public byte readByte() throws IOException { + if (!checkNext()) { + try { + close(); + } catch (IOException e) { + // ignore + } + throw new EOFException(); + } + + return currentBlock.get(); } @Override public int available() throws IOException { + if (closed) { + return 0; + } + int estimated = stream.available(); - if (estimated == 0 && currentBlock != null) { - estimated = currentBlock.length - pointer; + if (estimated == 0 && checkNext()) { + estimated = currentBlock.remaining(); } return estimated; } @Override public int read() throws IOException { - return checkNext() ? 0xFF & currentBlock[pointer++] : -1; + return checkNext() ? 0xFF & currentBlock.get() : -1; } @Override @@ -72,80 +154,63 @@ public int read(byte[] b, int off, int len) throws IOException { return 0; } - if (!checkNext()) + if (!checkNext()) { return -1; + } int copied = 0; - int targetPointer = off; while (copied != len) { - int toCopy = Math.min(currentBlock.length - pointer, len - copied); - System.arraycopy(currentBlock, pointer, b, targetPointer, toCopy); - targetPointer += toCopy; - pointer += toCopy; + int toCopy = Math.min(currentBlock.remaining(), len - copied); + currentBlock.get(b, off, toCopy); + off += toCopy; copied += toCopy; - if (!checkNext()) { // finished + + if (!checkNext()) { break; } + } + return copied; } @Override public void close() throws IOException { - stream.close(); - } - - private boolean checkNext() throws IOException { - if (currentBlock == null || pointer == currentBlock.length) { - currentBlock = readNextBlock(); - pointer = 0; + try { + stream.close(); + } finally { + closed = true; } - return currentBlock != null && pointer < currentBlock.length; } - private int readInt() throws IOException { - byte b1 = (byte) readUnsignedByte(); - byte b2 = (byte) readUnsignedByte(); - byte b3 = (byte) readUnsignedByte(); - byte b4 = (byte) readUnsignedByte(); - - return b4 << 24 | (b3 & 0xFF) << 16 | (b2 & 0xFF) << 8 | (b1 & 0xFF); + @Override + public boolean isClosed() { + return closed; } - // every block is: - private byte[] readNextBlock() throws IOException { - int read = stream.read(); - if (read < 0) - return null; + @Override + public String readString(int byteLength, Charset charset) throws IOException { + if (byteLength < 1) { + return ""; + } else if (!checkNext()) { + try { + close(); + } catch (IOException e) { + // ignore + } + throw new EOFException(); + } - byte[] checksum = new byte[16]; - checksum[0] = (byte) read; - // checksum - 16 bytes. - readFully(checksum, 1, 15); - ClickHouseBlockChecksum expected = ClickHouseBlockChecksum.fromBytes(checksum); - // header: - // 1 byte - 0x82 (shows this is LZ4) - int magic = readUnsignedByte(); - if (magic != MAGIC) - throw new IOException("Magic is not correct: " + magic); - // 4 bytes - size of the compressed data including 9 bytes of the header - int compressedSizeWithHeader = readInt(); - // 4 bytes - size of uncompressed data - int uncompressedSize = readInt(); - int compressedSize = compressedSizeWithHeader - 9; // header - byte[] block = new byte[compressedSize]; - // compressed data: compressed_size - 9 байт. - readFully(block, 0, block.length); + if (charset == null) { + charset = StandardCharsets.UTF_8; + } - ClickHouseBlockChecksum real = ClickHouseBlockChecksum.calculateForBlock((byte) magic, compressedSizeWithHeader, - uncompressedSize, block, compressedSize); - if (!real.equals(expected)) { - throw new IllegalArgumentException("Checksum doesn't match: corrupted data."); + 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(); } - byte[] decompressed = new byte[uncompressedSize]; - LZ4FastDecompressor decompressor = factory.fastDecompressor(); - decompressor.decompress(block, 0, decompressed, 0, uncompressedSize); - return decompressed; + return new String(readBytes(byteLength), charset); } } 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 fcb5790be..de4ca2f02 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 @@ -1,6 +1,5 @@ package com.clickhouse.client.data; -import java.io.EOFException; import java.io.IOException; import java.io.OutputStream; import java.nio.Buffer; @@ -20,171 +19,6 @@ * reader are on two separate threads. */ public class ClickHousePipedStream extends OutputStream { - protected static final ByteBuffer EMPTY = ByteBuffer.wrap(new byte[0]); - - static class Input extends ClickHouseInputStream { - private final BlockingQueue queue; - private final int timeout; - - // too much to maintain a 2-level buffer for reading? - private ByteBuffer buffer; - private boolean closed; - - Input(BlockingQueue queue, int timeout) { - this.queue = queue; - this.timeout = timeout; - - this.buffer = null; - this.closed = false; - } - - private void ensureOpen() throws IOException { - if (this.closed) { - throw new IOException("Stream has been closed"); - } - - if (this.buffer == null) { - updateBuffer(); - } - } - - private int updateBuffer() throws IOException { - try { - if (timeout > 0) { - buffer = queue.poll(timeout, TimeUnit.MILLISECONDS); - if (buffer == null) { - throw new IOException(ClickHouseUtils.format("Read timed out after %d ms", timeout)); - } - } else { - buffer = queue.take(); - } - - return buffer.remaining(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new IOException("Thread was interrupted when getting next buffer from queue", e); - } - } - - @Override - public int available() throws IOException { - ensureOpen(); - - if (buffer == EMPTY || buffer.limit() == 0) { - return 0; - } - - int available = buffer.remaining(); - if (available == 0) { - available = updateBuffer(); - } - - return available; - } - - @Override - public boolean isClosed() { - return this.closed; - } - - @Override - public void close() throws IOException { - // it's caller's responsiblity to consume all data in the queue, which will - // unblock writer - this.closed = true; - this.buffer = null; - } - - @Override - public byte readByte() throws IOException { - ensureOpen(); - - if (buffer == EMPTY || buffer.limit() == 0) { - close(); - throw new EOFException(); - } - - if (buffer.hasRemaining()) { - return buffer.get(); - } else { - updateBuffer(); - - return readByte(); - } - } - - @Override - public int read() throws IOException { - ensureOpen(); - - if (buffer == EMPTY || buffer.limit() == 0) { - return -1; - } - - if (buffer.hasRemaining()) { - return 0xFF & buffer.get(); - } else { - updateBuffer(); - - return read(); - } - } - - @Override - public int read(byte[] b, int off, int len) throws IOException { - ensureOpen(); - - if (buffer == EMPTY || buffer.limit() == 0) { - return -1; - } - - int counter = 0; - while (len > 0) { - if (buffer == EMPTY || buffer.limit() == 0) { - return counter; - } - - int remain = buffer.remaining(); - if (remain > 0) { - if (remain >= len) { - buffer.get(b, off, len); - counter += len; - len = 0; - } else { - buffer.get(b, off, remain); - counter += remain; - off += remain; - len -= remain; - - updateBuffer(); - } - } else { - updateBuffer(); - } - } - - return counter; - } - - @Override - public long skip(long n) throws IOException { - ensureOpen(); - - // peforms better but this is a bit tricky - if (n == Long.MAX_VALUE) { - long counter = buffer.remaining(); - while (buffer != EMPTY && buffer.limit() > 0) { - counter += buffer.limit(); - updateBuffer(); - } - - return counter; - } - - return super.skip(n); - } - } - protected final BlockingQueue queue; private final int bufferSize; @@ -236,7 +70,7 @@ private void updateBuffer() throws IOException { } public ClickHouseInputStream getInput() { - return new Input(queue, timeout); + return ClickHouseInputStream.of(queue, timeout); } @Override @@ -247,7 +81,7 @@ public void close() throws IOException { flush(); - buffer = EMPTY; + buffer = ClickHouseInputStream.EMPTY; try { if (timeout > 0) { if (!queue.offer(buffer, timeout, TimeUnit.MILLISECONDS)) { 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 e768d2ec4..79c2c8294 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 @@ -293,7 +293,7 @@ private MappedFunctions() { c.getPrecision()), ClickHouseDataType.FixedString); buildMappings(deserializers, serializers, - (r, f, c, i) -> ClickHouseStringValue.of(r, BinaryStreamUtils.readString(i)), + (r, f, c, i) -> ClickHouseStringValue.of(r, i.readUnicodeString()), (v, f, c, o) -> BinaryStreamUtils.writeString(o, v.asString()), ClickHouseDataType.String); buildMappings(deserializers, serializers, (r, f, c, i) -> ClickHouseUuidValue.of(r, BinaryStreamUtils.readUuid(i)), @@ -521,7 +521,7 @@ protected List readColumns() throws IOException { int size = 0; try { - size = BinaryStreamUtils.readVarInt(input); + size = input.readVarInt(); } catch (EOFException e) { // no result returned return Collections.emptyList(); @@ -529,12 +529,13 @@ protected List readColumns() throws IOException { String[] names = new String[ClickHouseChecker.between(size, "size", 0, Integer.MAX_VALUE)]; for (int i = 0; i < size; i++) { - names[i] = BinaryStreamUtils.readString(input); + names[i] = input.readUnicodeString(); } List columns = new ArrayList<>(size); for (int i = 0; i < size; i++) { - columns.add(ClickHouseColumn.of(names[i], BinaryStreamUtils.readString(input))); + // a bit risky here - what if ClickHouse support user type? + columns.add(ClickHouseColumn.of(names[i], input.readAsciiString())); } return columns; diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseShortValue.java b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseShortValue.java index 5a8fe9f2a..8efddd6f8 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseShortValue.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseShortValue.java @@ -84,7 +84,7 @@ protected ClickHouseShortValue(boolean isNull, short value) { protected ClickHouseShortValue set(boolean isNull, short value) { this.isNull = isNull; - this.value = isNull ? 0 : value; + this.value = isNull ? (short) 0 : value; return this; } diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseTupleValue.java b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseTupleValue.java index a75f6a04b..9694d285d 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseTupleValue.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseTupleValue.java @@ -81,8 +81,7 @@ public Object[] asArray() { return ClickHouseValues.EMPTY_OBJECT_ARRAY; } - List value = getValue(); - return value.toArray(new Object[value.size()]); + return getValue().toArray(new Object[0]); } @Override 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 1288495ca..3335d5516 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseParameterizedQueryTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/ClickHouseParameterizedQueryTest.java @@ -1,5 +1,7 @@ package com.clickhouse.client; +import java.time.LocalDateTime; +import java.time.ZoneOffset; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -148,7 +150,7 @@ public String nextElement() { Assert.assertTrue(q.apply("test") == query); Assert.assertTrue(q.apply("test1", "test2") == query); Assert.assertFalse(q.hasParameter()); - Assert.assertTrue((Object) q.getNamedParameters() == Collections.emptyList()); + Assert.assertTrue((Object) q.getParameters() == Collections.emptyList()); Assert.assertEquals(q.getQueryParts().toArray(new String[0][]), new String[][] { new String[] { query, null } }); } @@ -182,4 +184,18 @@ public void testApplyNamedParameters() { Assert.assertEquals(ClickHouseParameterizedQuery.apply(sql, params), "select 2>1?3:2, name, value, value::Decimal64(3) from my_table where value != ':ccc' and num in (1,2,3 ) and value = 's t r'"); } + + @Test(groups = { "unit" }) + public void testApplyTypedParameters() { + LocalDateTime ts = LocalDateTime.ofEpochSecond(10000, 123456789, ZoneOffset.UTC); + String sql = "select :ts1 ts1, :ts2(DateTime32) ts2, :ts2 ts3"; + ClickHouseParameterizedQuery pq = ClickHouseParameterizedQuery.of(sql); + ClickHouseValue[] templates = pq.getParameterTemplates(); + Assert.assertEquals(templates.length, pq.getParameters().size()); + 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 + "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-client/src/test/java/com/clickhouse/client/data/BinaryStreamUtilsTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/data/BinaryStreamUtilsTest.java index 446fc5265..aedf5b240 100644 --- a/clickhouse-client/src/test/java/com/clickhouse/client/data/BinaryStreamUtilsTest.java +++ b/clickhouse-client/src/test/java/com/clickhouse/client/data/BinaryStreamUtilsTest.java @@ -8,6 +8,7 @@ import java.net.Inet4Address; import java.net.Inet6Address; import java.net.InetAddress; +import java.nio.charset.StandardCharsets; import java.time.Instant; import java.time.LocalDate; import java.time.LocalDateTime; @@ -1877,6 +1878,13 @@ public void testWriteInet6Address() throws IOException { generateBytes(0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0xFF, 0xFF, 0xC0, 0xA8, 5, 1)); } + @Test(groups = { "unit" }) + public void testReadString() throws IOException { + Assert.assertEquals(new String(new byte[0]), ""); + Assert.assertEquals(new String(new byte[0], StandardCharsets.UTF_8), ""); + Assert.assertEquals(new String(new byte[0], StandardCharsets.UTF_16), ""); + } + @Test(groups = { "unit" }) public void testReadUuid() throws IOException { byte[][] arr = generateBytes(Long.BYTES * 2, false); 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 ffe751a57..9b2328c0b 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 @@ -8,6 +8,9 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; + +import com.clickhouse.client.ClickHouseInputStream; + import org.testng.Assert; import org.testng.annotations.Test; @@ -47,16 +50,16 @@ public void testRead() throws Exception { } stream.queue.clear(); - stream.queue.put(ClickHousePipedStream.EMPTY); + stream.queue.put(ClickHouseInputStream.EMPTY); Assert.assertEquals(stream.queue.size(), 1); try (InputStream in = stream.getInput()) { Assert.assertEquals(in.read(), -1); } stream.queue.put((ByteBuffer) ((Buffer) buf).rewind()); - stream.queue.put(buf); - stream.queue.put(ClickHousePipedStream.EMPTY); - Assert.assertEquals(stream.queue.size(), 3); + // stream.queue.put(buf); + stream.queue.put(ClickHouseInputStream.EMPTY); + Assert.assertEquals(stream.queue.size(), 2); try (InputStream in = stream.getInput()) { Assert.assertEquals(in.read(), 3); Assert.assertEquals(in.read(), 4); @@ -119,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(ClickHousePipedStream.EMPTY); + stream.queue.put(ClickHouseInputStream.EMPTY); Assert.assertEquals(stream.queue.size(), 2); try (InputStream in = stream.getInput()) { Assert.assertEquals(in.read(bytes, 0, 3), 2); diff --git a/clickhouse-grpc-client/pom.xml b/clickhouse-grpc-client/pom.xml index d03afc990..bcc28f1e3 100644 --- a/clickhouse-grpc-client/pom.xml +++ b/clickhouse-grpc-client/pom.xml @@ -138,10 +138,10 @@ google/** javax/** org/** + **/module-info.class META-INF/MANIFEST.MF META-INF/maven/** META-INF/native-image/** - META-INF/versions/** @@ -192,10 +192,10 @@ javax/** okio/** org/** + **/module-info.class META-INF/MANIFEST.MF META-INF/maven/** META-INF/native-image/** - META-INF/versions/** @@ -252,11 +252,11 @@ io/grpc/netty/** javax/** org/** + **/module-info.class META-INF/MANIFEST.MF META-INF/maven/** META-INF/native/** META-INF/native-image/** - META-INF/versions/** diff --git a/clickhouse-http-client/pom.xml b/clickhouse-http-client/pom.xml index 368647e93..36e0dbea7 100644 --- a/clickhouse-http-client/pom.xml +++ b/clickhouse-http-client/pom.xml @@ -116,10 +116,10 @@ **/darwin/** **/linux/** **/win32/** + **/module-info.class META-INF/MANIFEST.MF META-INF/maven/** META-INF/native-image/** - META-INF/versions/** 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 7e166b451..f362066e1 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 @@ -20,6 +20,7 @@ import com.clickhouse.client.ClickHouseCompression; import com.clickhouse.client.ClickHouseConfig; import com.clickhouse.client.ClickHouseCredentials; +import com.clickhouse.client.ClickHouseInputStream; import com.clickhouse.client.ClickHouseNode; import com.clickhouse.client.ClickHouseRequest; import com.clickhouse.client.ClickHouseUtils; @@ -231,28 +232,25 @@ protected OutputStream getRequestOutputStream(OutputStream out) throws IOExcepti return out; } - protected InputStream getResponseInputStream(InputStream in) throws IOException { - if (!config.isCompressServerResponse()) { - return in; + protected ClickHouseInputStream getResponseInputStream(InputStream in) throws IOException { + if (config.isCompressServerResponse()) { + // TODO support more algorithms + ClickHouseCompression algorithm = config.getCompressAlgorithmForServerResponse(); + switch (algorithm) { + case GZIP: + in = ClickHouseInputStream.of(new GZIPInputStream(in)); + break; + case LZ4: + in = new ClickHouseLZ4InputStream(in); + break; + default: + throw new UnsupportedOperationException("Unsupported compression algorithm: " + algorithm); + } } - // TODO support more algorithms - ClickHouseCompression algorithm = config.getCompressAlgorithmForServerResponse(); - switch (algorithm) { - case GZIP: - in = new GZIPInputStream(in); - break; - case LZ4: - in = new ClickHouseLZ4InputStream(in); - break; - default: - throw new UnsupportedOperationException("Unsupported compression algorithm: " + algorithm); - } - return in; + return in instanceof ClickHouseInputStream ? (ClickHouseInputStream) in : ClickHouseInputStream.of(in); } - protected abstract String getResponseHeader(String header, String defaultValue); - /** * Creates a merged map. * diff --git a/clickhouse-http-client/src/main/java/com/clickhouse/client/http/ClickHouseHttpResponse.java b/clickhouse-http-client/src/main/java/com/clickhouse/client/http/ClickHouseHttpResponse.java index 19d5a03db..5abbc1259 100644 --- a/clickhouse-http-client/src/main/java/com/clickhouse/client/http/ClickHouseHttpResponse.java +++ b/clickhouse-http-client/src/main/java/com/clickhouse/client/http/ClickHouseHttpResponse.java @@ -1,9 +1,8 @@ package com.clickhouse.client.http; -import java.io.EOFException; import java.io.IOException; -import java.io.InputStream; import java.io.Serializable; +import java.nio.charset.Charset; import java.util.HashMap; import java.util.Map; import java.util.TimeZone; @@ -32,7 +31,7 @@ private static long getLongValue(Map map, String key) { } private final ClickHouseHttpConnection connection; - private final InputStream input; + private final ClickHouseInputStream input; protected final String serverDisplayName; protected final String queryId; @@ -55,7 +54,8 @@ protected ClickHouseConfig getConfig(ClickHouseRequest request) { return config; } - public ClickHouseHttpResponse(ClickHouseHttpConnection connection, InputStream input) { + public ClickHouseHttpResponse(ClickHouseHttpConnection connection, ClickHouseInputStream input, + String serverDisplayName, String queryId, String summary, ClickHouseFormat format, TimeZone timeZone) { if (connection == null || input == null) { throw new IllegalArgumentException("Non-null connection and input stream are required"); } @@ -63,13 +63,12 @@ public ClickHouseHttpResponse(ClickHouseHttpConnection connection, InputStream i this.connection = connection; this.input = input; - this.serverDisplayName = connection.getResponseHeader("X-ClickHouse-Server-Display-Name", - connection.server.getHost()); - // queryId, format and timeZone are only available for queries - this.queryId = connection.getResponseHeader("X-ClickHouse-Query-Id", ""); + this.serverDisplayName = !ClickHouseChecker.isNullOrEmpty(serverDisplayName) ? serverDisplayName + : connection.server.getHost(); + this.queryId = !ClickHouseChecker.isNullOrEmpty(queryId) ? queryId : ""; // {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} Map map = (Map) ClickHouseUtils - .parseJson(connection.getResponseHeader("X-ClickHouse-Summary", "{}")); + .parseJson(!ClickHouseChecker.isNullOrEmpty(summary) ? summary : "{}"); // discard those X-ClickHouse-Progress headers this.summary = new ClickHouseResponseSummary( new ClickHouseResponseSummary.Progress(getLongValue(map, "read_rows"), getLongValue(map, "read_bytes"), @@ -77,32 +76,15 @@ public ClickHouseHttpResponse(ClickHouseHttpConnection connection, InputStream i getLongValue(map, "written_bytes")), null); - if (ClickHouseChecker.isNullOrEmpty(this.queryId)) { - this.format = connection.config.getFormat(); - this.timeZone = connection.config.getServerTimeZone(); - // better to close input stream since there's no response to read? - // input.close(); - } else { - String value = connection.getResponseHeader("X-ClickHouse-Format", ""); - this.format = !ClickHouseChecker.isNullOrEmpty(value) ? ClickHouseFormat.valueOf(value) - : connection.config.getFormat(); - value = connection.getResponseHeader("X-ClickHouse-Timezone", ""); - this.timeZone = !ClickHouseChecker.isNullOrEmpty(value) ? TimeZone.getTimeZone(value) - : connection.config.getServerTimeZone(); - } + this.format = format != null ? format : connection.config.getFormat(); + this.timeZone = timeZone != null ? timeZone : connection.config.getServerTimeZone(); closed = false; } @Override public byte readByte() throws IOException { - int v = input.read(); - if (v == -1) { - close(); - throw new EOFException(); - } - - return (byte) v; + return input.readByte(); } @Override @@ -158,4 +140,14 @@ public int read(byte[] b, int off, int len) throws IOException { public long skip(long n) throws IOException { return input.skip(n); } + + @Override + public byte[] readBytes(int length) throws IOException { + return input.readBytes(length); + } + + @Override + public String readString(int byteLength, Charset charset) throws IOException { + return input.readString(byteLength, charset); + } } 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 905d0611b..afa4369d4 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 @@ -1,5 +1,7 @@ package com.clickhouse.client.http; +import com.clickhouse.client.ClickHouseChecker; +import com.clickhouse.client.ClickHouseFormat; import com.clickhouse.client.ClickHouseNode; import com.clickhouse.client.ClickHouseRequest; import com.clickhouse.client.ClickHouseSslContextProvider; @@ -23,6 +25,7 @@ import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; +import java.util.TimeZone; import java.util.UUID; import java.util.Map.Entry; @@ -35,6 +38,33 @@ public class DefaultHttpConnection extends ClickHouseHttpConnection { private final HttpURLConnection conn; + private ClickHouseHttpResponse buildResponse() throws IOException { + // X-ClickHouse-Server-Display-Name: xxx + // X-ClickHouse-Query-Id: xxx + // X-ClickHouse-Format: RowBinaryWithNamesAndTypes + // X-ClickHouse-Timezone: UTC + // X-ClickHouse-Summary: + // {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} + String displayName = getResponseHeader("X-ClickHouse-Server-Display-Name", server.getHost()); + String queryId = getResponseHeader("X-ClickHouse-Query-Id", ""); + String summary = getResponseHeader("X-ClickHouse-Summary", "{}"); + + ClickHouseFormat format = config.getFormat(); + TimeZone timeZone = config.getServerTimeZone(); + // queryId, format and timeZone are only available for queries + if (!ClickHouseChecker.isNullOrEmpty(queryId)) { + String value = getResponseHeader("X-ClickHouse-Format", ""); + format = !ClickHouseChecker.isNullOrEmpty(value) ? ClickHouseFormat.valueOf(value) + : format; + value = getResponseHeader("X-ClickHouse-Timezone", ""); + timeZone = !ClickHouseChecker.isNullOrEmpty(value) ? TimeZone.getTimeZone(value) + : timeZone; + } + + return new ClickHouseHttpResponse(this, getResponseInputStream(conn.getInputStream()), + displayName, queryId, summary, format, timeZone); + } + private HttpURLConnection newConnection(String url, boolean post) throws IOException { HttpURLConnection newConn = (HttpURLConnection) new URL(url).openConnection(); @@ -64,6 +94,11 @@ private HttpURLConnection newConnection(String url, boolean post) throws IOExcep return newConn; } + private String getResponseHeader(String header, String defaultValue) { + String value = conn.getHeaderField(header); + return value != null ? value : defaultValue; + } + private void setHeaders(HttpURLConnection conn, Map headers) { headers = mergeHeaders(headers); @@ -106,12 +141,6 @@ protected boolean isReusable() { return false; } - @Override - protected String getResponseHeader(String header, String defaultValue) { - String value = conn.getHeaderField(header); - return value != null ? value : defaultValue; - } - @Override protected ClickHouseHttpResponse post(String sql, InputStream data, List tables, Map headers) throws IOException { @@ -168,14 +197,7 @@ protected ClickHouseHttpResponse post(String sql, InputStream data, List { + // An immutable ByteBuffer sentinel to mark that the last byte was received. + private static final List LAST_LIST = List.of(ClickHouseInputStream.EMPTY); + + private final BlockingQueue buffers; + private final ClickHouseInputStream in; + private final AtomicBoolean subscribed; + + ClickHouseResponseHandler(int bufferSize, int timeout) { + buffers = new LinkedBlockingDeque<>(); + in = ClickHouseInputStream.of(buffers, timeout); + subscribed = new AtomicBoolean(); + } + + @Override + public void onSubscribe(Subscription s) { + try { + if (!subscribed.compareAndSet(false, true)) { + s.cancel(); + } else { + if (in.isClosed()) { + s.cancel(); + return; + } + s.request(Long.MAX_VALUE); + } + } catch (Throwable t) { + try { + in.close(); + } catch (IOException x) { + // ignore + } finally { + onError(t); + } + } + } + + @Override + public void onNext(List item) { + try { + if (!buffers.addAll(item)) { + // should never happen + throw new IllegalStateException("Queue is full"); + } + } catch (Throwable t) { + try { + in.close(); + } catch (IOException e) { + // ignore + } finally { + onError(t); + } + } + } + + @Override + public void onError(Throwable throwable) { + buffers.offer(ClickHouseInputStream.EMPTY); + } + + @Override + public void onComplete() { + onNext(LAST_LIST); + } + + @Override + public CompletionStage getBody() { + return CompletableFuture.completedStage(in); + } +} 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 new file mode 100644 index 000000000..bd96b11b8 --- /dev/null +++ b/clickhouse-http-client/src/main/java11/com/clickhouse/client/http/DefaultHttpConnection.java @@ -0,0 +1,246 @@ +package com.clickhouse.client.http; + +import com.clickhouse.client.ClickHouseChecker; +import com.clickhouse.client.ClickHouseClient; +import com.clickhouse.client.ClickHouseFormat; +import com.clickhouse.client.ClickHouseNode; +import com.clickhouse.client.ClickHouseRequest; +import com.clickhouse.client.ClickHouseSslContextProvider; +import com.clickhouse.client.data.ClickHouseExternalTable; +import com.clickhouse.client.data.ClickHousePipedStream; +import com.clickhouse.client.http.config.ClickHouseHttpOption; +import com.clickhouse.client.logging.Logger; +import com.clickhouse.client.logging.LoggerFactory; + +import java.io.BufferedReader; +import java.io.BufferedWriter; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.OutputStreamWriter; +import java.io.Reader; +import java.net.HttpURLConnection; +import java.net.URI; +import java.net.http.HttpClient; +import java.net.http.HttpHeaders; +import java.net.http.HttpRequest; +import java.net.http.HttpResponse; +import java.net.http.HttpClient.Redirect; +import java.net.http.HttpClient.Version; +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.TimeZone; +import java.util.UUID; +import java.util.Map.Entry; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; + +import javax.net.ssl.SSLContext; + +public class DefaultHttpConnection extends ClickHouseHttpConnection { + private static final Logger log = LoggerFactory.getLogger(DefaultHttpConnection.class); + + private final HttpClient httpClient; + + private ClickHouseHttpResponse buildResponse(HttpResponse r) throws IOException { + HttpHeaders headers = r.headers(); + String displayName = headers.firstValue("X-ClickHouse-Server-Display-Name").orElse(server.getHost()); + String queryId = headers.firstValue("X-ClickHouse-Query-Id").orElse(""); + String summary = headers.firstValue("X-ClickHouse-Summary").orElse("{}"); + + ClickHouseFormat format = config.getFormat(); + TimeZone timeZone = config.getServerTimeZone(); + // queryId, format and timeZone are only available for queries + if (!ClickHouseChecker.isNullOrEmpty(queryId)) { + String value = headers.firstValue("X-ClickHouse-Format").orElse(""); + format = !ClickHouseChecker.isNullOrEmpty(value) ? ClickHouseFormat.valueOf(value) + : format; + value = headers.firstValue("X-ClickHouse-Timezone").orElse(""); + timeZone = !ClickHouseChecker.isNullOrEmpty(value) ? TimeZone.getTimeZone(value) + : timeZone; + } + + return new ClickHouseHttpResponse(this, getResponseInputStream(checkResponse(r).body()), + displayName, queryId, summary, format, timeZone); + } + + private HttpResponse checkResponse(HttpResponse r) throws IOException { + if (r.statusCode() != HttpURLConnection.HTTP_OK) { + // TODO get exception from response header, for example: + // X-ClickHouse-Exception-Code: 47 + StringBuilder builder = new StringBuilder(); + try (Reader reader = new BufferedReader( + new InputStreamReader(getResponseInputStream(r.body()), StandardCharsets.UTF_8))) { + int c = 0; + while ((c = reader.read()) != -1) { + builder.append((char) c); + } + } catch (IOException e) { + log.warn("Error while reading error message", e); + } + + throw new IOException(builder.toString()); + } + + return r; + } + + private HttpRequest newRequest(String url) { + return HttpRequest.newBuilder() + .uri(URI.create(url)) + .timeout(Duration.ofMillis(config.getSocketTimeout())).build(); + } + + protected DefaultHttpConnection(ClickHouseNode server, ClickHouseRequest request) throws IOException { + super(server, request); + + HttpClient.Builder builder = HttpClient.newBuilder() + .connectTimeout(Duration.ofMillis(config.getConnectionTimeout())) + .followRedirects(Redirect.ALWAYS) + .version(Version.HTTP_1_1); + if (config.isAsync()) { + builder.executor(ClickHouseClient.getExecutorService()); + } + if (config.isSsl()) { + builder.sslContext(ClickHouseSslContextProvider.getProvider().getSslContext(SSLContext.class, config) + .orElse(null)); + } + + httpClient = builder.build(); + } + + @Override + protected boolean isReusable() { + return true; + } + + 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())); + try (BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(stream, StandardCharsets.UTF_8))) { + if (boundary != null) { + String line = "\r\n--" + boundary + "\r\n"; + writer.write(line); + writer.write("Content-Disposition: form-data; name=\"query\"\r\n\r\n"); + writer.write(sql); + + for (ClickHouseExternalTable t : tables) { + String tableName = t.getName(); + StringBuilder builder = new StringBuilder(); + builder.append(line).append("Content-Disposition: form-data; name=\"").append(tableName) + .append("_format\"\r\n\r\n").append(t.getFormat().name()); + builder.append(line).append("Content-Disposition: form-data; name=\"").append(tableName) + .append("_structure\"\r\n\r\n").append(t.getStructure()); + builder.append(line).append("Content-Disposition: form-data; name=\"").append(tableName) + .append("\"; filename=\"").append(tableName).append("\"\r\n") + .append("Content-Type: application/octet-stream\r\n") + .append("Content-Transfer-Encoding: binary\r\n\r\n"); + writer.write(builder.toString()); + writer.flush(); + + pipe(t.getContent(), stream, DEFAULT_BUFFER_SIZE); + } + + writer.write("\r\n--" + boundary + "--\r\n"); + writer.flush(); + } else { + writer.write(sql); + writer.flush(); + + if (data.available() > 0) { + // append \n + if (sql.charAt(sql.length() - 1) != '\n') { + stream.write(10); + } + + pipe(data, stream, DEFAULT_BUFFER_SIZE); + } + } + } + + HttpResponse r; + try { + r = f.get(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException("Thread was interrupted when posting request or receiving response", e); + } catch (ExecutionException e) { + throw new IOException("Failed to post request", e); + } + + return buildResponse(r); + } + + private ClickHouseHttpResponse postString(HttpRequest.Builder reqBuilder, String sql) throws IOException { + reqBuilder.POST(HttpRequest.BodyPublishers.ofString(sql)); + HttpResponse r; + try { + // r = httpClient.send(reqBuilder.build(), responseInfo -> new + // ExtendedResponseInputStream()); + r = httpClient.send(reqBuilder.build(), + responseInfo -> new ClickHouseResponseHandler(config.getMaxBufferSize(), + config.getSocketTimeout())); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException("Thread was interrupted when posting request or receiving response", e); + } + return buildResponse(r); + } + + @Override + protected ClickHouseHttpResponse post(String sql, InputStream data, List tables, + Map headers) throws IOException { + HttpRequest.Builder reqBuilder = HttpRequest.newBuilder() + .uri(URI.create(url)) + .timeout(Duration.ofMillis(config.getSocketTimeout())); + String boundary = null; + if (tables != null && !tables.isEmpty()) { + boundary = UUID.randomUUID().toString(); + reqBuilder.setHeader("Content-Type", "multipart/form-data; boundary=" + boundary); + } else { + reqBuilder.setHeader("Content-Type", "text/plain; charset=UTF-8"); + } + + headers = mergeHeaders(headers); + if (headers != null && !headers.isEmpty()) { + for (Entry header : headers.entrySet()) { + reqBuilder.setHeader(header.getKey(), header.getValue()); + } + } + + return boundary != null || data != null ? postStream(reqBuilder, boundary, sql, data, tables) + : postString(reqBuilder, sql); + } + + @Override + public boolean ping(int timeout) { + String response = (String) config.getOption(ClickHouseHttpOption.DEFAULT_RESPONSE); + try { + HttpResponse 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()); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } catch (IOException e) { + log.debug("Failed to ping server: ", e.getMessage()); + } + + return false; + } + + @Override + public void close() { + } +} diff --git a/clickhouse-http-client/src/main/java11/module-info.java b/clickhouse-http-client/src/main/java11/module-info.java new file mode 100644 index 000000000..7d1cb83e4 --- /dev/null +++ b/clickhouse-http-client/src/main/java11/module-info.java @@ -0,0 +1,12 @@ +module com.clickhouse.client.http { + exports com.clickhouse.client.http; + exports com.clickhouse.client.http.config; + + provides com.clickhouse.client.ClickHouseClient with com.clickhouse.client.http.ClickHouseHttpClient; + + requires java.net.http; + + requires static com.google.gson; + + requires transitive com.clickhouse.client; +} diff --git a/clickhouse-http-client/src/test/java/com/clickhouse/client/http/ClickHouseHttpClientTest.java b/clickhouse-http-client/src/test/java/com/clickhouse/client/http/ClickHouseHttpClientTest.java index 17eb4f37f..73af1b33e 100644 --- a/clickhouse-http-client/src/test/java/com/clickhouse/client/http/ClickHouseHttpClientTest.java +++ b/clickhouse-http-client/src/test/java/com/clickhouse/client/http/ClickHouseHttpClientTest.java @@ -198,7 +198,8 @@ public void testPost() throws Exception { try (ClickHouseClient client = ClickHouseClient.builder() .defaultCredentials(ClickHouseCredentials.fromUserAndPassword("foo", "bar")).build()) { // why no detailed error message for this: "select 1,2" - try (ClickHouseResponse resp = client.connect(server).format(ClickHouseFormat.RowBinaryWithNamesAndTypes) + try (ClickHouseResponse resp = client.connect(server).compressServerResponse(false) + .format(ClickHouseFormat.RowBinaryWithNamesAndTypes) .query("select 1,2").execute().get()) { int count = 0; for (ClickHouseRecord r : resp.records()) { diff --git a/clickhouse-http-client/src/test/java/com/clickhouse/client/http/ClickHouseHttpConnectionTest.java b/clickhouse-http-client/src/test/java/com/clickhouse/client/http/ClickHouseHttpConnectionTest.java index e72fb6c2a..6026c29ff 100644 --- a/clickhouse-http-client/src/test/java/com/clickhouse/client/http/ClickHouseHttpConnectionTest.java +++ b/clickhouse-http-client/src/test/java/com/clickhouse/client/http/ClickHouseHttpConnectionTest.java @@ -35,11 +35,6 @@ public boolean ping(int timeout) { @Override public void close() throws Exception { } - - @Override - protected String getResponseHeader(String header, String defaultValue) { - return defaultValue; - } } @Test(groups = { "unit" }) diff --git a/clickhouse-jdbc/pom.xml b/clickhouse-jdbc/pom.xml index 9a2957f9d..43c272497 100644 --- a/clickhouse-jdbc/pom.xml +++ b/clickhouse-jdbc/pom.xml @@ -238,6 +238,7 @@ + com.clickhouse.jdbc ${spec.title} ${spec.version} @@ -261,11 +262,11 @@ **/darwin/** **/linux/** **/win32/** + **/module-info.class META-INF/DEPENDENCIES META-INF/MANIFEST.MF META-INF/maven/** META-INF/native-image/** - META-INF/versions/** META-INF/*.xml @@ -303,6 +304,7 @@ + com.clickhouse.jdbc ${spec.title} ${spec.version} @@ -320,11 +322,11 @@ **/darwin/** **/linux/** **/win32/** + **/module-info.class META-INF/DEPENDENCIES META-INF/MANIFEST.MF META-INF/maven/** META-INF/native-image/** - META-INF/versions/** META-INF/*.xml @@ -358,6 +360,7 @@ + com.clickhouse.jdbc ${spec.title} ${spec.version} @@ -387,11 +390,11 @@ **/darwin/** **/linux/** **/win32/** + **/module-info.class META-INF/DEPENDENCIES META-INF/MANIFEST.MF META-INF/maven/** META-INF/native-image/** - META-INF/versions/** META-INF/*.xml @@ -415,6 +418,7 @@ + com.clickhouse.jdbc ${spec.title} ${spec.version} @@ -458,11 +462,11 @@ **/darwin/** **/linux/** **/win32/** + **/module-info.class META-INF/DEPENDENCIES META-INF/MANIFEST.MF META-INF/maven/** META-INF/native-image/** - META-INF/versions/** META-INF/*.xml diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/AbstractResultSet.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/AbstractResultSet.java index d4be1244e..b1ca6c1c3 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/AbstractResultSet.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/AbstractResultSet.java @@ -18,7 +18,7 @@ import java.sql.Time; import java.sql.Timestamp; -public abstract class AbstractResultSet extends Wrapper implements ResultSet { +public abstract class AbstractResultSet extends JdbcWrapper implements ResultSet { protected void ensureOpen() throws SQLException { if (isClosed()) { throw SqlExceptionUtils.clientError("Cannot operate on a closed ResultSet"); diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseConnection.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseConnection.java index 648054c69..327b80f16 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseConnection.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseConnection.java @@ -8,12 +8,62 @@ import java.sql.SQLException; import java.sql.Statement; import java.util.Calendar; +import java.util.Collections; import java.util.Optional; import java.util.TimeZone; +import com.clickhouse.client.ClickHouseColumn; +import com.clickhouse.client.ClickHouseConfig; +import com.clickhouse.client.ClickHouseValue; +import com.clickhouse.client.ClickHouseValues; import com.clickhouse.client.ClickHouseVersion; +import com.clickhouse.client.data.ClickHouseSimpleResponse; +import com.clickhouse.jdbc.parser.ClickHouseSqlStatement; public interface ClickHouseConnection extends Connection { + // The name of the application currently utilizing the connection + static final String PROP_APPLICATION_NAME = "ApplicationName"; + static final String PROP_CUSTOM_HTTP_HEADERS = "CustomHttpHeaders"; + static final String PROP_CUSTOM_HTTP_PARAMS = "CustomHttpParameters"; + // The name of the user that the application using the connection is performing + // work for. This may not be the same as the user name that was used in + // establishing the connection. + // private static final String PROP_CLIENT_USER = "ClientUser"; + // The hostname of the computer the application using the connection is running + // on. + // private static final String PROP_CLIENT_HOST = "ClientHostname"; + + @Override + default ClickHouseArray createArrayOf(String typeName, Object[] elements) throws SQLException { + ClickHouseColumn column = ClickHouseColumn.of("", typeName); + ClickHouseValue v = ClickHouseValues.newValue(column).update(elements); + ClickHouseResultSet rs = new ClickHouseResultSet("", "", createStatement(), + ClickHouseSimpleResponse.of(Collections.singletonList(column), + new Object[][] { new Object[] { v.asObject() } })); + rs.next(); + return new ClickHouseArray(rs, 1); + } + + @Override + default ClickHouseBlob createBlob() throws SQLException { + return new ClickHouseBlob(); + } + + @Override + default ClickHouseClob createClob() throws SQLException { + return new ClickHouseClob(); + } + + @Override + default ClickHouseStruct createStruct(String typeName, Object[] attributes) throws SQLException { + return new ClickHouseStruct(typeName, attributes); + } + + @Override + default ClickHouseXml createSQLXML() throws SQLException { + return new ClickHouseXml(); + } + @Override default ClickHouseStatement createStatement() throws SQLException { return createStatement(ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY, @@ -130,11 +180,40 @@ default PreparedStatement prepareStatement(String sql, int resultSetType, int re */ TimeZone getServerTimeZone(); + /** + * Gets server version. + * + * @return non-null server version + */ ClickHouseVersion getServerVersion(); + /** + * Gets URI of the connection. + * + * @return URI of the connection + */ URI getUri(); - boolean isJdbcCompliant(); + /** + * Gets JDBC-specific configuration. + * + * @return non-null JDBC-specific configuration + */ + JdbcConfig getJdbcConfig(); + /** + * Creates a new query ID. + * + * @return universal unique query ID + */ String newQueryId(); + + /** + * Parses the given sql. + * + * @param sql sql to parse + * @param config configuration which might be used for parsing, could be null + * @return non-null parsed sql statements + */ + ClickHouseSqlStatement[] parse(String sql, ClickHouseConfig config); } diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseDataSource.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseDataSource.java index 2c4344cdd..956311a33 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseDataSource.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseDataSource.java @@ -4,6 +4,7 @@ import com.clickhouse.client.ClickHouseNode; import com.clickhouse.client.config.ClickHouseDefaults; +import com.clickhouse.jdbc.internal.ClickHouseConnectionImpl; import com.clickhouse.jdbc.internal.ClickHouseJdbcUrlParser; import com.clickhouse.jdbc.internal.ClickHouseJdbcUrlParser.ConnectionInfo; @@ -15,41 +16,32 @@ import java.util.Properties; import java.util.logging.Logger; -public class ClickHouseDataSource extends Wrapper implements DataSource { +public class ClickHouseDataSource extends JdbcWrapper implements DataSource { private final String url; - protected final ClickHouseDriver driver = new ClickHouseDriver(); - - protected final Properties properties; - protected final ClickHouseNode server; - protected final URI uri; + protected final ClickHouseDriver driver; + protected final ConnectionInfo connInfo; protected PrintWriter printWriter; protected int loginTimeoutSeconds = 0; - public ClickHouseDataSource(String url) { + public ClickHouseDataSource(String url) throws SQLException { this(url, new Properties()); } - public ClickHouseDataSource(String url, Properties properties) { + public ClickHouseDataSource(String url, Properties properties) throws SQLException { if (url == null) { throw new IllegalArgumentException("Incorrect ClickHouse jdbc url. It must be not null"); } this.url = url; - try { - ConnectionInfo connInfo = ClickHouseJdbcUrlParser.parse(url, properties); - this.properties = connInfo.getProperties(); - this.server = connInfo.getServer(); - this.uri = connInfo.getUri(); - } catch (URISyntaxException e) { - throw new IllegalArgumentException(e); - } + this.driver = new ClickHouseDriver(); + this.connInfo = ClickHouseJdbcUrlParser.parse(url, properties); } @Override public ClickHouseConnection getConnection() throws SQLException { - return driver.connect(url, properties); + return new ClickHouseConnectionImpl(connInfo); } @Override @@ -62,31 +54,32 @@ public ClickHouseConnection getConnection(String username, String password) thro password = ""; } - Properties props = new Properties(properties); + Properties props = new Properties(connInfo.getProperties()); props.setProperty(ClickHouseDefaults.USER.getKey(), username); props.setProperty(ClickHouseDefaults.PASSWORD.getKey(), password); return driver.connect(url, props); } public String getHost() { - return server.getHost(); + return connInfo.getServer().getHost(); } public int getPort() { - return server.getPort(); + return connInfo.getServer().getPort(); } public String getDatabase() { - return server.getDatabase().orElse((String) ClickHouseDefaults.DATABASE.getEffectiveDefaultValue()); + return connInfo.getServer().getDatabase() + .orElse((String) ClickHouseDefaults.DATABASE.getEffectiveDefaultValue()); } // public String getUrl() { // return url; // } - public Properties getProperties() { - return properties; - } + // public Properties getProperties() { + // return connInfo.getProperties(); + // } @Override public PrintWriter getLogWriter() throws SQLException { 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 6aee3f297..4d354300f 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseDatabaseMetaData.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseDatabaseMetaData.java @@ -24,7 +24,7 @@ import com.clickhouse.client.logging.Logger; import com.clickhouse.client.logging.LoggerFactory; -public class ClickHouseDatabaseMetaData extends Wrapper implements DatabaseMetaData { +public class ClickHouseDatabaseMetaData extends JdbcWrapper implements DatabaseMetaData { private static final Logger log = LoggerFactory.getLogger(ClickHouseDatabaseMetaData.class); private static final String DATABASE_NAME = "ClickHouse"; @@ -657,12 +657,13 @@ public int getMaxUserNameLength() throws SQLException { @Override public int getDefaultTransactionIsolation() throws SQLException { - return connection.isJdbcCompliant() ? Connection.TRANSACTION_READ_COMMITTED : Connection.TRANSACTION_NONE; + return connection.getJdbcConfig().isJdbcCompliant() ? Connection.TRANSACTION_READ_COMMITTED + : Connection.TRANSACTION_NONE; } @Override public boolean supportsTransactions() throws SQLException { - return connection.isJdbcCompliant(); + return connection.getJdbcConfig().isJdbcCompliant(); } @Override @@ -674,7 +675,7 @@ public boolean supportsTransactionIsolationLevel(int level) throws SQLException throw SqlExceptionUtils.clientError("Unknown isolation level: " + level); } - return connection.isJdbcCompliant(); + return connection.getJdbcConfig().isJdbcCompliant(); } @Override @@ -684,7 +685,7 @@ public boolean supportsDataDefinitionAndDataManipulationTransactions() throws SQ @Override public boolean supportsDataManipulationTransactionsOnly() throws SQLException { - return connection.isJdbcCompliant(); + return connection.getJdbcConfig().isJdbcCompliant(); } @Override @@ -753,7 +754,7 @@ public ResultSet getTables(String catalog, String schemaPattern, String tableNam params.put("table", ClickHouseChecker.isNullOrEmpty(tableNamePattern) ? "'%'" : ClickHouseValues.convertToQuotedString(tableNamePattern)); params.put("types", builder.toString()); - String sql = JdbcParameterizedQuery + String sql = ClickHouseParameterizedQuery .apply("select null as TABLE_CAT, t.database as TABLE_SCHEM, t.name as TABLE_NAME, " + "case when t.engine like '%Log' then 'LOG TABLE' " + "when t.engine in ('Buffer', 'Memory', 'Set') then 'MEMORY TABLE' " @@ -807,7 +808,7 @@ public ResultSet getColumns(String catalog, String schemaPattern, String tableNa params.put("defaultNullable", String.valueOf(DatabaseMetaData.typeNullable)); params.put("defaultNonNull", String.valueOf(DatabaseMetaData.typeNoNulls)); params.put("defaultType", String.valueOf(Types.OTHER)); - String sql = JdbcParameterizedQuery + String sql = ClickHouseParameterizedQuery .apply("select null as TABLE_CAT, database as TABLE_SCHEM, table as TABLE_NAME, " + "name as COLUMN_NAME, toInt32(:defaultType) as DATA_TYPE, type as TYPE_NAME, toInt32(0) as COLUMN_SIZE, " + "0 as BUFFER_LENGTH, toInt32(null) as DECIMAL_DIGITS, 10 as NUM_PREC_RADIX, " @@ -924,43 +925,43 @@ private Object[] toTypeRow(String typeName, String aliasTo) { : DatabaseMetaData.typePredBasic; int money = 0; switch (type) { - case Date: - case Date32: - case DateTime: - case DateTime32: - case DateTime64: - case Enum: - case Enum8: - case Enum16: - case String: - case FixedString: - case UUID: - prefix = "'"; - suffix = "'"; - break; - case Array: - case Nested: - case Ring: - case Polygon: - case MultiPolygon: - prefix = "["; - suffix = "]"; - nullable = DatabaseMetaData.typeNoNulls; - break; - case AggregateFunction: - case Tuple: - case Point: - prefix = "("; - suffix = ")"; - nullable = DatabaseMetaData.typeNoNulls; - break; - case Map: - prefix = "{"; - suffix = "}"; - nullable = DatabaseMetaData.typeNoNulls; - break; - default: - break; + case Date: + case Date32: + case DateTime: + case DateTime32: + case DateTime64: + case Enum: + case Enum8: + case Enum16: + case String: + case FixedString: + case UUID: + prefix = "'"; + suffix = "'"; + break; + case Array: + case Nested: + case Ring: + case Polygon: + case MultiPolygon: + prefix = "["; + suffix = "]"; + nullable = DatabaseMetaData.typeNoNulls; + break; + case AggregateFunction: + case Tuple: + case Point: + prefix = "("; + suffix = ")"; + nullable = DatabaseMetaData.typeNoNulls; + break; + case Map: + prefix = "{"; + suffix = "}"; + nullable = DatabaseMetaData.typeNoNulls; + break; + default: + break; } return new Object[] { typeName, JdbcTypeMapping.toJdbcType(ClickHouseColumn.of("", type, false, false, new String[0])), @@ -1005,21 +1006,21 @@ public ResultSet getIndexInfo(String catalog, String schema, String table, boole + "NON_UNIQUE UInt8, INDEX_QUALIFIER Nullable(String), INDEX_NAME Nullable(String), " + "TYPE Int16, ORDINAL_POSITION Int16, COLUMN_NAME Nullable(String), ASC_OR_DESC Nullable(String), " + "CARDINALITY Int64, PAGES Int64, FILTER_CONDITION Nullable(String)"), - query(JdbcParameterizedQuery.apply( + query(ClickHouseParameterizedQuery.apply( "select null as TABLE_CAT, database as TABLE_SCHEM, table as TABLE_NAME, toUInt8(0) as NON_UNIQUE, " + "null as INDEX_QUALIFIER, null as INDEX_NAME, toInt16(:statIndex) as TYPE, " + "toInt16(0) as ORDINAL_POSITION, null as COLUMN_NAME, null as ASC_OR_DESC, " + "sum(rows) as CARDINALITY, uniqExact(name) as PAGES, null as FILTER_CONDITION from system.parts " + "where active = 1 and database like :database and table like :table group by database, table", params), true), - query(JdbcParameterizedQuery.apply( + query(ClickHouseParameterizedQuery.apply( "select null as TABLE_CAT, database as TABLE_SCHEM, table as TABLE_NAME, toUInt8(1) as NON_UNIQUE, " + "type as INDEX_QUALIFIER, name as INDEX_NAME, toInt16(:otherIndex) as TYPE, " + "toInt16(1) as ORDINAL_POSITION, expr as COLUMN_NAME, null as ASC_OR_DESC, " + "0 as CARDINALITY, 0 as PAGES, null as FILTER_CONDITION " + "from system.data_skipping_indices where database like :database and table like :table", params), true), - query(JdbcParameterizedQuery.apply( + query(ClickHouseParameterizedQuery.apply( "select null as TABLE_CAT, database as TABLE_SCHEM, table as TABLE_NAME, toUInt8(1) as NON_UNIQUE, " + "null as INDEX_QUALIFIER, name as INDEX_NAME, toInt16(:otherIndex) as TYPE, " + "column_position as ORDINAL_POSITION, column as COLUMN_NAME, null as ASC_OR_DESC, " @@ -1201,9 +1202,9 @@ public ResultSet getSchemas(String catalog, String schemaPattern) throws SQLExce ClickHouseChecker.isNullOrEmpty(schemaPattern) ? "'%'" : ClickHouseValues.convertToQuotedString(schemaPattern)); return new CombinedResultSet( - query(JdbcParameterizedQuery.apply("select name as TABLE_SCHEM, null as TABLE_CATALOG " + query(ClickHouseParameterizedQuery.apply("select name as TABLE_SCHEM, null as TABLE_CATALOG " + "from system.databases where name like :pattern order by name", params)), - query(JdbcParameterizedQuery.apply( + query(ClickHouseParameterizedQuery.apply( "select concat('jdbc(''', name, ''')') as TABLE_SCHEM, null as TABLE_CATALOG " + "from jdbc('', 'SHOW DATASOURCES') where TABLE_SCHEM like :pattern order by name", params), true)); @@ -1221,14 +1222,25 @@ public boolean autoCommitFailureClosesAllResultSets() throws SQLException { @Override public ResultSet getClientInfoProperties() throws SQLException { - // TODO Auto-generated method stub - return null; + ClickHouseParameterizedQuery q = ClickHouseParameterizedQuery + .of("select :name as NAME, toInt32(0) as MAX_LEN, :default as DEFAULT_VALUE, :desc as DESCRIPTION"); + StringBuilder builder = new StringBuilder(); + builder.append(q.apply(ClickHouseConnection.PROP_APPLICATION_NAME, + connection.getClientInfo(ClickHouseConnection.PROP_APPLICATION_NAME), "Application name")) + .append(" union all "); + builder.append(q.apply(ClickHouseConnection.PROP_CUSTOM_HTTP_HEADERS, + connection.getClientInfo(ClickHouseConnection.PROP_CUSTOM_HTTP_HEADERS), "Custom HTTP headers")) + .append(" union all "); + builder.append(q.apply(ClickHouseConnection.PROP_CUSTOM_HTTP_PARAMS, + connection.getClientInfo(ClickHouseConnection.PROP_CUSTOM_HTTP_PARAMS), + "Customer HTTP query parameters")); + return query(builder.toString()); } @Override public ResultSet getFunctions(String catalog, String schemaPattern, String functionNamePattern) throws SQLException { - String sql = JdbcParameterizedQuery.apply( + String sql = ClickHouseParameterizedQuery.apply( "select null as FUNCTION_CAT, null 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" diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseDriver.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseDriver.java index 4d4fef910..d58440b09 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseDriver.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseDriver.java @@ -66,7 +66,7 @@ public class ClickHouseDriver implements Driver { Map m = new LinkedHashMap<>(); try { for (ClickHouseClient c : ServiceLoader.load(ClickHouseClient.class, - Thread.currentThread().getContextClassLoader())) { + ClickHouseDriver.class.getClassLoader())) { Class clazz = c.getOptionClass(); if (clazz == null || clazz == ClickHouseClientOption.class) { continue; @@ -163,10 +163,7 @@ public DriverPropertyInfo[] getPropertyInfo(String url, Properties info) throws result.add(create(option, info)); } - DriverPropertyInfo custom = new DriverPropertyInfo(ClickHouseJdbcUrlParser.PROP_JDBC_COMPLIANT, "true"); - custom.choices = new String[] { "true", "false" }; - custom.description = "Whether to enable JDBC-compliant features like fake transaction and standard UPDATE and DELETE statements."; - result.add(custom); + result.addAll(JdbcConfig.getDriverProperties()); return result.toArray(new DriverPropertyInfo[0]); } diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHousePreparedStatement.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHousePreparedStatement.java new file mode 100644 index 000000000..b899b64cc --- /dev/null +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHousePreparedStatement.java @@ -0,0 +1,223 @@ +package com.clickhouse.jdbc; + +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.Reader; +import java.net.URL; +import java.nio.charset.StandardCharsets; +import java.sql.Blob; +import java.sql.Clob; +import java.sql.Date; +import java.sql.NClob; +import java.sql.PreparedStatement; +import java.sql.Ref; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.RowId; +import java.sql.SQLException; +import java.sql.SQLXML; +import java.sql.Time; +import java.sql.Timestamp; +import java.sql.Types; + +import com.clickhouse.client.data.BinaryStreamUtils; + +public interface ClickHousePreparedStatement extends PreparedStatement { + @Override + default void setNull(int parameterIndex, int sqlType) throws SQLException { + setNull(parameterIndex, sqlType, null); + } + + @Override + default void setBoolean(int parameterIndex, boolean x) throws SQLException { + setByte(parameterIndex, x ? (byte) 1 : (byte) 0); + } + + @Override + default void setDate(int parameterIndex, Date x) throws SQLException { + setDate(parameterIndex, x, null); + } + + @Override + default void setTime(int parameterIndex, Time x) throws SQLException { + setTime(parameterIndex, x, null); + } + + @Override + default void setTimestamp(int parameterIndex, Timestamp x) throws SQLException { + setTimestamp(parameterIndex, x, null); + } + + @Override + default void setAsciiStream(int parameterIndex, InputStream x, int length) throws SQLException { + setCharacterStream(parameterIndex, new InputStreamReader(x, StandardCharsets.US_ASCII), length); + } + + @Override + default void setUnicodeStream(int parameterIndex, InputStream x, int length) throws SQLException { + setCharacterStream(parameterIndex, new InputStreamReader(x, StandardCharsets.UTF_8), length); + } + + @Override + default void setBinaryStream(int parameterIndex, InputStream x, int length) throws SQLException { + setBinaryStream(parameterIndex, x, (long) length); + } + + @Override + default void setObject(int parameterIndex, Object x, int targetSqlType) throws SQLException { + setObject(parameterIndex, x, targetSqlType, 0); + } + + @Override + default boolean execute() throws SQLException { + return executeQuery() != null; + } + + @Override + default void addBatch(String sql) throws SQLException { + throw SqlExceptionUtils + .unsupportedError("addBatch(String) cannot be called in PreparedStatement or CallableStatement!"); + } + + @Override + default void setCharacterStream(int parameterIndex, Reader reader, int length) throws SQLException { + String s = null; + if (reader != null) { + try { + s = BinaryStreamUtils.readString(reader, length); + } catch (Throwable e) { // IOException and potentially OOM error + throw SqlExceptionUtils.clientError(e); + } + } + + setString(parameterIndex, s); + } + + @Override + default void setRef(int parameterIndex, Ref x) throws SQLException { + throw SqlExceptionUtils.unsupportedError("setRef not implemented"); + } + + @Override + default void setBlob(int parameterIndex, Blob x) throws SQLException { + if (x != null) { + setBinaryStream(parameterIndex, x.getBinaryStream()); + } else { + setNull(parameterIndex, Types.BLOB); + } + } + + @Override + default void setClob(int parameterIndex, Clob x) throws SQLException { + if (x != null) { + setCharacterStream(parameterIndex, x.getCharacterStream()); + } else { + setNull(parameterIndex, Types.CLOB); + } + } + + @Override + default ResultSetMetaData getMetaData() throws SQLException { + ResultSet currentResult = getResultSet(); + return currentResult != null ? currentResult.getMetaData() : null; + } + + @Override + default void setURL(int parameterIndex, URL x) throws SQLException { + if (x != null) { + setString(parameterIndex, String.valueOf(x)); + } else { + setNull(parameterIndex, Types.VARCHAR); + } + } + + @Override + default void setRowId(int parameterIndex, RowId x) throws SQLException { + throw SqlExceptionUtils.unsupportedError("setRowId not implemented"); + } + + @Override + default void setNString(int parameterIndex, String value) throws SQLException { + setString(parameterIndex, value); + } + + @Override + default void setNCharacterStream(int parameterIndex, Reader value, long length) throws SQLException { + setCharacterStream(parameterIndex, value, length); + } + + @Override + default void setNClob(int parameterIndex, NClob value) throws SQLException { + setClob(parameterIndex, value); + } + + @Override + default void setClob(int parameterIndex, Reader reader, long length) throws SQLException { + setCharacterStream(parameterIndex, reader, length); + } + + @Override + default void setBlob(int parameterIndex, InputStream inputStream, long length) throws SQLException { + setBinaryStream(parameterIndex, inputStream, length); + } + + @Override + default void setNClob(int parameterIndex, Reader reader, long length) throws SQLException { + setClob(parameterIndex, reader, length); + } + + @Override + default void setSQLXML(int parameterIndex, SQLXML xmlObject) throws SQLException { + throw SqlExceptionUtils.unsupportedError("setSQLXML not implemented"); + } + + @Override + default void setAsciiStream(int parameterIndex, InputStream x, long length) throws SQLException { + setCharacterStream(parameterIndex, new InputStreamReader(x, StandardCharsets.US_ASCII), length); + } + + @Override + default void setBinaryStream(int parameterIndex, InputStream x, long length) throws SQLException { + throw SqlExceptionUtils.unsupportedError("setBinaryStream not implemented"); + } + + @Override + default void setCharacterStream(int parameterIndex, Reader reader, long length) throws SQLException { + throw SqlExceptionUtils.unsupportedError("setCharacterStream not implemented"); + } + + @Override + default void setAsciiStream(int parameterIndex, InputStream x) throws SQLException { + setCharacterStream(parameterIndex, new InputStreamReader(x, StandardCharsets.US_ASCII)); + } + + @Override + default void setBinaryStream(int parameterIndex, InputStream x) throws SQLException { + setBinaryStream(parameterIndex, x, 0L); + } + + @Override + default void setCharacterStream(int parameterIndex, Reader reader) throws SQLException { + setCharacterStream(parameterIndex, reader, 0L); + } + + @Override + default void setNCharacterStream(int parameterIndex, Reader value) throws SQLException { + setCharacterStream(parameterIndex, value); + } + + @Override + default void setClob(int parameterIndex, Reader reader) throws SQLException { + setCharacterStream(parameterIndex, reader); + } + + @Override + default void setBlob(int parameterIndex, InputStream inputStream) throws SQLException { + setBinaryStream(parameterIndex, inputStream); + } + + @Override + default void setNClob(int parameterIndex, Reader reader) throws SQLException { + setClob(parameterIndex, reader); + } +} 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 d8687817f..f94c6d68a 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseResultSet.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseResultSet.java @@ -49,6 +49,7 @@ public class ClickHouseResultSet extends AbstractResultSet { protected final ClickHouseResponse response; protected final ClickHouseConfig config; + protected final boolean wrapObject; protected final List columns; protected final Calendar defaultCalendar; protected final TimeZone tsTimeZone; @@ -64,6 +65,7 @@ public class ClickHouseResultSet extends AbstractResultSet { this.response = response; this.config = null; + this.wrapObject = false; this.defaultCalendar = new GregorianCalendar(TimeZone.getTimeZone("UTC")); this.tsTimeZone = null; // TimeZone.getDefault(); this.dateTimeZone = this.tsTimeZone; @@ -98,6 +100,7 @@ public ClickHouseResultSet(String database, String table, ClickHouseStatement st ClickHouseConnection conn = statement.getConnection(); this.config = statement.getConfig(); + this.wrapObject = statement.getConnection().getJdbcConfig().useWrapperObject(); this.defaultCalendar = conn.getDefaultCalendar(); this.tsTimeZone = conn.getEffectiveTimeZone().orElse(null); this.dateTimeZone = this.tsTimeZone; @@ -445,11 +448,15 @@ 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.isNested()) { + } else if (c.isTuple() || c.isNested()) { return new ClickHouseStruct(c.getDataType().name(), v.asArray()); } else { return v.asObject(); @@ -715,4 +722,14 @@ public boolean wasNull() throws SQLException { throw SqlExceptionUtils.handle(e); } } + + @Override + public boolean isWrapperFor(Class iface) throws SQLException { + return iface == ClickHouseResponse.class || super.isWrapperFor(iface); + } + + @Override + public T unwrap(Class iface) throws SQLException { + return iface == ClickHouseResponse.class ? iface.cast(response) : super.unwrap(iface); + } } 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 030c72d3e..d7ea52b1d 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseResultSetMetaData.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseResultSetMetaData.java @@ -8,7 +8,7 @@ import com.clickhouse.client.ClickHouseColumn; import com.clickhouse.client.ClickHouseUtils; -public class ClickHouseResultSetMetaData extends Wrapper implements ResultSetMetaData { +public class ClickHouseResultSetMetaData extends JdbcWrapper implements ResultSetMetaData { public static ResultSetMetaData of(String database, String table, List columns) throws SQLException { if (database == null || table == null || columns == null) { diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcConfig.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcConfig.java new file mode 100644 index 000000000..08b5b1c41 --- /dev/null +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcConfig.java @@ -0,0 +1,138 @@ +package com.clickhouse.jdbc; + +import java.sql.DriverPropertyInfo; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; +import java.util.Properties; + +public class JdbcConfig { + 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_WRAPPER_OBJ = "wrapperObject"; + + private static final String BOOLEAN_FALSE = "false"; + private static final String BOOLEAN_TRUE = "true"; + + private static final String DEFAULT_AUTO_COMMIT = BOOLEAN_TRUE; + 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_WRAPPER_OBJ = BOOLEAN_TRUE; + + static boolean extractBooleanValue(Properties props, String key, String defaultValue) { + if (props == null || props.isEmpty() || key == null || key.isEmpty()) { + return Boolean.parseBoolean(defaultValue); + } + + Object value = props.remove(key); + return Boolean.parseBoolean(value != null ? value.toString() : defaultValue); + } + + static int extractIntValue(Properties props, String key, String defaultValue) { + if (props == null || props.isEmpty() || key == null || key.isEmpty()) { + return Integer.parseInt(defaultValue); + } + + Object value = props.remove(key); + return Integer.parseInt(value != null ? value.toString() : defaultValue); + } + + public static List getDriverProperties() { + List list = new LinkedList<>(); + DriverPropertyInfo info = new DriverPropertyInfo(PROP_AUTO_COMMIT, DEFAULT_AUTO_COMMIT); + info.choices = new String[] { BOOLEAN_TRUE, BOOLEAN_FALSE }; + info.description = "Whether to enable auto commit when connection is created."; + + info = new DriverPropertyInfo(PROP_FETCH_SIZE, DEFAULT_FETCH_SIZE); + info.description = "Default fetch size, negative or zero means no preferred option."; + + info = new DriverPropertyInfo(PROP_JDBC_COMPLIANT, DEFAULT_JDBC_COMPLIANT); + info.choices = new String[] { BOOLEAN_TRUE, BOOLEAN_FALSE }; + info.description = "Whether to enable JDBC-compliant features like fake transaction and standard UPDATE and DELETE statements."; + + info = new DriverPropertyInfo(PROP_NAMED_PARAM, DEFAULT_NAMED_PARAM); + info.choices = new String[] { BOOLEAN_TRUE, BOOLEAN_FALSE }; + info.description = "Whether to use named parameter(e.g. :ts(DateTime64(6)) or :value etc.) instead of standard JDBC question mark placeholder."; + + 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."; + + return Collections.unmodifiableList(list); + } + + private final boolean autoCommit; + private final int fetchSize; + private final boolean jdbcCompliant; + private final boolean namedParameter; + private final boolean wrapperObject; + + public JdbcConfig() { + this(null); + } + + public JdbcConfig(Properties props) { + if (props == null) { + props = new Properties(); + } + + this.autoCommit = extractBooleanValue(props, PROP_AUTO_COMMIT, DEFAULT_AUTO_COMMIT); + 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.wrapperObject = extractBooleanValue(props, PROP_WRAPPER_OBJ, DEFAULT_WRAPPER_OBJ); + } + + /** + * Checks whether auto commit should be enabled when creating a connection. + * + * @return true if auto commit should be enabled when creating connection; false + * otherwise + */ + public boolean isAutoCommit() { + return autoCommit; + } + + /** + * Gets default fetch size for query. + * + * @return default fetch size for query + */ + public int getFetchSize() { + return fetchSize; + } + + /** + * Checks whether JDBC-complaint mode is enabled or not. + * + * @return true if JDBC-complaint mode is enabled; false otherwise + */ + public boolean isJdbcCompliant() { + return jdbcCompliant; + } + + /** + * Checks whether named parameter should be used instead of JDBC standard + * question mark placeholder. + * + * @return true if named parameter should be used; false otherwise + */ + public boolean useNamedParameter() { + return namedParameter; + } + + /** + * Checks whether {@link java.sql.Array} and {@link java.sql.Struct} should be + * returned for array and tuple when calling + * {@link java.sql.ResultSet#getObject(int)}. + * + * @return true if wrapper object should be returned instead of array / tuple; + * false otherwise + */ + public boolean useWrapperObject() { + return wrapperObject; + } +} 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 96254d55b..4fe51f147 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcParameterizedQuery.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcParameterizedQuery.java @@ -1,9 +1,9 @@ package com.clickhouse.jdbc; -import java.util.ArrayList; -import java.util.List; +import java.util.Collection; +import java.util.Collections; +import java.util.Map; -import com.clickhouse.client.ClickHouseColumn; import com.clickhouse.client.ClickHouseParameterizedQuery; import com.clickhouse.client.ClickHouseUtils; @@ -38,7 +38,8 @@ protected String parse() { i = ClickHouseUtils.skipQuotedString(originalQuery, i, len, ch) - 1; } else if (ch == '?') { int idx = ClickHouseUtils.skipContentsUntil(originalQuery, i + 2, len, '?', ':'); - if (idx < len && originalQuery.charAt(idx - 1) == ':' && originalQuery.charAt(idx) != ':') { + if (idx < len && originalQuery.charAt(idx - 1) == ':' && originalQuery.charAt(idx) != ':' + && originalQuery.charAt(idx - 2) != ':') { i = idx - 1; } else { addPart(originalQuery.substring(partIndex, i), paramIndex++, null); diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcParseHandler.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcParseHandler.java index a010da0fe..b185da033 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcParseHandler.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcParseHandler.java @@ -2,6 +2,7 @@ import java.util.List; import java.util.Map; +import java.util.Set; import com.clickhouse.client.ClickHouseChecker; import com.clickhouse.jdbc.parser.ClickHouseSqlStatement; @@ -37,7 +38,7 @@ private void addMutationSetting(String sql, StringBuilder builder, Map parameters, - Map positions, Map settings) { + Map positions, Map settings, Set tempTables) { StringBuilder builder = new StringBuilder(); int index = positions.get("DELETE"); if (index > 0) { @@ -56,12 +57,12 @@ private ClickHouseSqlStatement handleDelete(String sql, StatementType stmtType, builder.append("TRUNCATE TABLE").append(sql.substring(index + 4)); } return new ClickHouseSqlStatement(builder.toString(), stmtType, cluster, database, table, input, format, - outfile, parameters, null, settings); + outfile, parameters, null, settings, null); } private ClickHouseSqlStatement handleUpdate(String sql, StatementType stmtType, String cluster, String database, String table, String input, String format, String outfile, List parameters, - Map positions, Map settings) { + Map positions, Map settings, Set tempTables) { StringBuilder builder = new StringBuilder(); int index = positions.get("UPDATE"); if (index > 0) { @@ -75,20 +76,20 @@ private ClickHouseSqlStatement handleUpdate(String sql, StatementType stmtType, builder.append('`').append(table).append('`').append(" UPDATE"); // .append(sql.substring(index + 3)); addMutationSetting(sql, builder, positions, settings, index + 3); return new ClickHouseSqlStatement(builder.toString(), stmtType, cluster, database, table, input, format, - outfile, parameters, null, settings); + outfile, parameters, null, settings, null); } @Override public ClickHouseSqlStatement handleStatement(String sql, StatementType stmtType, String cluster, String database, String table, String input, String format, String outfile, List parameters, - Map positions, Map settings) { + Map positions, Map settings, Set tempTables) { ClickHouseSqlStatement s = null; if (stmtType == StatementType.DELETE) { s = handleDelete(sql, stmtType, cluster, database, table, input, format, outfile, parameters, positions, - settings); + settings, tempTables); } else if (stmtType == StatementType.UPDATE) { s = handleUpdate(sql, stmtType, cluster, database, table, input, format, outfile, parameters, positions, - settings); + settings, tempTables); } return s; } 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 63dacd2ab..20ebf4155 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcTypeMapping.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcTypeMapping.java @@ -16,85 +16,85 @@ public static int toJdbcType(ClickHouseColumn column) { int sqlType = Types.OTHER; switch (column.getDataType()) { - case Enum: - case Enum8: - case Int8: - sqlType = Types.TINYINT; - break; - case UInt8: - case Enum16: - case Int16: - sqlType = Types.SMALLINT; - break; - case UInt16: - case Int32: - sqlType = Types.INTEGER; - break; - case UInt32: - case IntervalYear: - case IntervalQuarter: - case IntervalMonth: - case IntervalWeek: - case IntervalDay: - case IntervalHour: - case IntervalMinute: - case IntervalSecond: - case Int64: - sqlType = Types.BIGINT; - break; - case UInt64: - case Int128: - case UInt128: - case Int256: - case UInt256: - sqlType = Types.NUMERIC; - break; - case Float32: - sqlType = Types.FLOAT; - break; - case Float64: - sqlType = Types.DOUBLE; - break; - case Decimal: - case Decimal32: - case Decimal64: - case Decimal128: - case Decimal256: - sqlType = Types.DECIMAL; - break; - case Date: - case Date32: - sqlType = Types.DATE; - break; - case DateTime: - case DateTime32: - case DateTime64: - sqlType = column.getTimeZone() != null ? Types.TIMESTAMP_WITH_TIMEZONE : Types.TIMESTAMP; - break; - case IPv4: - case IPv6: - case FixedString: - case String: - case UUID: - sqlType = Types.VARCHAR; - break; - case Point: - case Ring: - case Polygon: - case MultiPolygon: - case Array: - sqlType = Types.ARRAY; - break; - case Nested: - sqlType = Types.STRUCT; - break; - case Nothing: - sqlType = Types.NULL; - break; - case Map: - case Tuple: - default: - break; + case Enum: + case Enum8: + case Int8: + sqlType = Types.TINYINT; + break; + case UInt8: + case Enum16: + case Int16: + sqlType = Types.SMALLINT; + break; + case UInt16: + case Int32: + sqlType = Types.INTEGER; + break; + case UInt32: + case IntervalYear: + case IntervalQuarter: + case IntervalMonth: + case IntervalWeek: + case IntervalDay: + case IntervalHour: + case IntervalMinute: + case IntervalSecond: + case Int64: + sqlType = Types.BIGINT; + break; + case UInt64: + case Int128: + case UInt128: + case Int256: + case UInt256: + sqlType = Types.NUMERIC; + break; + case Float32: + sqlType = Types.FLOAT; + break; + case Float64: + sqlType = Types.DOUBLE; + break; + case Decimal: + case Decimal32: + case Decimal64: + case Decimal128: + case Decimal256: + sqlType = Types.DECIMAL; + break; + case Date: + case Date32: + sqlType = Types.DATE; + break; + case DateTime: + case DateTime32: + case DateTime64: + sqlType = column.getTimeZone() != null ? Types.TIMESTAMP_WITH_TIMEZONE : Types.TIMESTAMP; + break; + case IPv4: + case IPv6: + case FixedString: + case String: + case UUID: + sqlType = Types.VARCHAR; + break; + case Point: + case Ring: + case Polygon: + case MultiPolygon: + case Array: + sqlType = Types.ARRAY; + break; + case Tuple: + case Nested: + sqlType = Types.STRUCT; + break; + case Nothing: + sqlType = Types.NULL; + break; + case Map: + default: + break; } return sqlType; @@ -130,76 +130,76 @@ public static ClickHouseDataType fromJdbcType(int jdbcType) { ClickHouseDataType dataType; switch (jdbcType) { - case Types.BIT: - case Types.BOOLEAN: - dataType = ClickHouseDataType.UInt8; - break; - case Types.TINYINT: - dataType = ClickHouseDataType.Int8; - break; - case Types.SMALLINT: - dataType = ClickHouseDataType.Int16; - break; - case Types.INTEGER: - dataType = ClickHouseDataType.Int32; - break; - case Types.BIGINT: - dataType = ClickHouseDataType.Int64; - break; - case Types.NUMERIC: - dataType = ClickHouseDataType.Int256; - break; - case Types.FLOAT: - case Types.REAL: - dataType = ClickHouseDataType.Float32; - break; - case Types.DOUBLE: - dataType = ClickHouseDataType.Float64; - break; - case Types.DECIMAL: - dataType = ClickHouseDataType.Decimal; - break; - case Types.BLOB: - case Types.BINARY: - case Types.CHAR: - case Types.CLOB: - case Types.JAVA_OBJECT: - case Types.LONGNVARCHAR: - case Types.LONGVARBINARY: - case Types.LONGVARCHAR: - case Types.NCHAR: - case Types.NCLOB: - case Types.NVARCHAR: - case Types.OTHER: - case Types.SQLXML: - case Types.VARBINARY: - case Types.VARCHAR: - dataType = ClickHouseDataType.String; - break; - case Types.DATE: - dataType = ClickHouseDataType.Date; - break; - case Types.TIME: - case Types.TIME_WITH_TIMEZONE: - case Types.TIMESTAMP: - case Types.TIMESTAMP_WITH_TIMEZONE: - dataType = ClickHouseDataType.DateTime; - break; - case Types.ARRAY: - dataType = ClickHouseDataType.Array; - break; - case Types.STRUCT: - dataType = ClickHouseDataType.Nested; - break; - case Types.DATALINK: - case Types.DISTINCT: - case Types.REF: - case Types.REF_CURSOR: - case Types.ROWID: - case Types.NULL: - default: - dataType = ClickHouseDataType.Nothing; - break; + case Types.BIT: + case Types.BOOLEAN: + dataType = ClickHouseDataType.UInt8; + break; + case Types.TINYINT: + dataType = ClickHouseDataType.Int8; + break; + case Types.SMALLINT: + dataType = ClickHouseDataType.Int16; + break; + case Types.INTEGER: + dataType = ClickHouseDataType.Int32; + break; + case Types.BIGINT: + dataType = ClickHouseDataType.Int64; + break; + case Types.NUMERIC: + dataType = ClickHouseDataType.Int256; + break; + case Types.FLOAT: + case Types.REAL: + dataType = ClickHouseDataType.Float32; + break; + case Types.DOUBLE: + dataType = ClickHouseDataType.Float64; + break; + case Types.DECIMAL: + dataType = ClickHouseDataType.Decimal; + break; + case Types.BLOB: + case Types.BINARY: + case Types.CHAR: + case Types.CLOB: + case Types.JAVA_OBJECT: + case Types.LONGNVARCHAR: + case Types.LONGVARBINARY: + case Types.LONGVARCHAR: + case Types.NCHAR: + case Types.NCLOB: + case Types.NVARCHAR: + case Types.OTHER: + case Types.SQLXML: + case Types.VARBINARY: + case Types.VARCHAR: + dataType = ClickHouseDataType.String; + break; + case Types.DATE: + dataType = ClickHouseDataType.Date; + break; + case Types.TIME: + case Types.TIME_WITH_TIMEZONE: + case Types.TIMESTAMP: + case Types.TIMESTAMP_WITH_TIMEZONE: + dataType = ClickHouseDataType.DateTime; + break; + case Types.ARRAY: + dataType = ClickHouseDataType.Array; + break; + case Types.STRUCT: + dataType = ClickHouseDataType.Nested; + break; + case Types.DATALINK: + case Types.DISTINCT: + case Types.REF: + case Types.REF_CURSOR: + case Types.ROWID: + case Types.NULL: + default: + dataType = ClickHouseDataType.Nothing; + break; } return dataType; } diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/Wrapper.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcWrapper.java similarity index 92% rename from clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/Wrapper.java rename to clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcWrapper.java index e37dbe76e..e9169d03b 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/Wrapper.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcWrapper.java @@ -2,7 +2,7 @@ import java.sql.SQLException; -public abstract class Wrapper { +public abstract class JdbcWrapper { public T unwrap(Class iface) throws SQLException { if (iface.isAssignableFrom(getClass())) { return iface.cast(this); 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 1b273fd22..957280ba4 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 @@ -1,11 +1,7 @@ package com.clickhouse.jdbc.internal; import java.net.URI; -import java.net.URISyntaxException; -import java.sql.Array; -import java.sql.Blob; import java.sql.ClientInfoStatus; -import java.sql.Clob; import java.sql.Connection; import java.sql.DatabaseMetaData; import java.sql.NClob; @@ -14,15 +10,10 @@ import java.sql.SQLClientInfoException; import java.sql.SQLException; import java.sql.SQLWarning; -import java.sql.SQLXML; import java.sql.Savepoint; -import java.sql.Struct; -import java.util.ArrayList; import java.util.Calendar; -import java.util.Collections; import java.util.GregorianCalendar; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Properties; @@ -40,45 +31,35 @@ import com.clickhouse.client.ClickHouseFormat; import com.clickhouse.client.ClickHouseNode; import com.clickhouse.client.ClickHouseNodeSelector; +import com.clickhouse.client.ClickHouseParameterizedQuery; import com.clickhouse.client.ClickHouseRecord; import com.clickhouse.client.ClickHouseRequest; import com.clickhouse.client.ClickHouseResponse; -import com.clickhouse.client.ClickHouseUtils; import com.clickhouse.client.ClickHouseValues; import com.clickhouse.client.ClickHouseVersion; import com.clickhouse.client.config.ClickHouseClientOption; import com.clickhouse.client.http.config.ClickHouseHttpOption; import com.clickhouse.client.logging.Logger; import com.clickhouse.client.logging.LoggerFactory; -import com.clickhouse.jdbc.ClickHouseBlob; -import com.clickhouse.jdbc.ClickHouseClob; import com.clickhouse.jdbc.ClickHouseConnection; import com.clickhouse.jdbc.ClickHouseDatabaseMetaData; import com.clickhouse.jdbc.ClickHouseDriver; import com.clickhouse.jdbc.ClickHouseStatement; -import com.clickhouse.jdbc.ClickHouseXml; +import com.clickhouse.jdbc.JdbcConfig; import com.clickhouse.jdbc.JdbcParameterizedQuery; +import com.clickhouse.jdbc.JdbcParseHandler; import com.clickhouse.jdbc.SqlExceptionUtils; -import com.clickhouse.jdbc.Wrapper; +import com.clickhouse.jdbc.JdbcWrapper; import com.clickhouse.jdbc.internal.ClickHouseJdbcUrlParser.ConnectionInfo; import com.clickhouse.jdbc.internal.FakeTransaction.FakeSavepoint; +import com.clickhouse.jdbc.parser.ClickHouseSqlParser; +import com.clickhouse.jdbc.parser.ClickHouseSqlStatement; +import com.clickhouse.jdbc.parser.StatementType; -public class ClickHouseConnectionImpl extends Wrapper implements ClickHouseConnection { +public class ClickHouseConnectionImpl extends JdbcWrapper implements ClickHouseConnection { private static final Logger log = LoggerFactory.getLogger(ClickHouseConnectionImpl.class); - // The name of the application currently utilizing the connection - private static final String PROP_APPLICATION_NAME = "ApplicationName"; - private static final String PROP_CUSTOM_HTTP_HEADERS = "CustomHttpHeaders"; - private static final String PROP_CUSTOM_HTTP_PARAMS = "CustomHttpParameters"; - // The name of the user that the application using the connection is performing - // work for. This may not be the same as the user name that was used in - // establishing the connection. - // private static final String PROP_CLIENT_USER = "ClientUser"; - // The hostname of the computer the application using the connection is running - // on. - // private static final String PROP_CLIENT_HOST = "ClientHostname"; - - private final boolean jdbcCompliant; + private final JdbcConfig jdbcConf; private final ClickHouseClient client; private final ClickHouseRequest clientRequest; @@ -126,13 +107,13 @@ protected void ensureOpen() throws SQLException { protected void ensureSupport(String feature, boolean silent) throws SQLException { String msg = feature + " is not supported"; - if (isJdbcCompliant()) { + if (jdbcConf.isJdbcCompliant()) { if (silent) { log.debug("[JDBC Compliant Mode] %s. Change %s to false to throw SQLException instead.", msg, - ClickHouseJdbcUrlParser.PROP_JDBC_COMPLIANT); + JdbcConfig.PROP_JDBC_COMPLIANT); } else { log.warn("[JDBC Compliant Mode] %s. Change %s to false to throw SQLException instead.", msg, - ClickHouseJdbcUrlParser.PROP_JDBC_COMPLIANT); + JdbcConfig.PROP_JDBC_COMPLIANT); } } else if (!silent) { throw SqlExceptionUtils.unsupportedError(msg); @@ -153,19 +134,19 @@ public ClickHouseConnectionImpl(String url) throws SQLException { } public ClickHouseConnectionImpl(String url, Properties properties) throws SQLException { - ConnectionInfo connInfo; - try { - connInfo = ClickHouseJdbcUrlParser.parse(url, properties); - } catch (URISyntaxException | IllegalArgumentException e) { - throw SqlExceptionUtils.clientError(e); - } + this(ClickHouseJdbcUrlParser.parse(url, properties)); + } + + public ClickHouseConnectionImpl(ConnectionInfo connInfo) throws SQLException { + Properties properties = connInfo.getProperties(); + + jdbcConf = connInfo.getJdbcConfig(); + + autoCommit = !jdbcConf.isJdbcCompliant() || jdbcConf.isAutoCommit(); - this.jdbcCompliant = Boolean.parseBoolean( - connInfo.getProperties().getProperty(ClickHouseJdbcUrlParser.PROP_JDBC_COMPLIANT, "true")); - connInfo.getProperties().remove(ClickHouseJdbcUrlParser.PROP_JDBC_COMPLIANT); this.uri = connInfo.getUri(); - log.debug("Creating a new connection to %s", url); + log.debug("Creating a new connection to %s", connInfo.getUri()); ClickHouseNode node = connInfo.getServer(); log.debug("Target node: %s", node); @@ -218,7 +199,8 @@ public ClickHouseConnectionImpl(String url, Properties properties) throws SQLExc this.readOnly = false; this.networkTimeout = 0; this.rsHoldability = ResultSet.HOLD_CURSORS_OVER_COMMIT; - this.txIsolation = jdbcCompliant ? Connection.TRANSACTION_READ_COMMITTED : Connection.TRANSACTION_NONE; + this.txIsolation = jdbcConf.isJdbcCompliant() ? Connection.TRANSACTION_READ_COMMITTED + : Connection.TRANSACTION_NONE; this.user = currentUser != null ? currentUser : node.getCredentials(config).getUserName(); this.serverTimeZone = timeZone; @@ -448,7 +430,7 @@ public Savepoint setSavepoint(String name) throws SQLException { throw SqlExceptionUtils.clientError("Cannot set savepoint in auto-commit mode"); } - if (!isJdbcCompliant()) { + if (!jdbcConf.isJdbcCompliant()) { throw SqlExceptionUtils.unsupportedError("setSavepoint not implemented"); } @@ -464,7 +446,7 @@ public void rollback(Savepoint savepoint) throws SQLException { throw SqlExceptionUtils.clientError("Cannot rollback to savepoint in auto-commit mode"); } - if (!isJdbcCompliant()) { + if (!jdbcConf.isJdbcCompliant()) { throw SqlExceptionUtils.unsupportedError("rollback not implemented"); } @@ -491,7 +473,7 @@ public void releaseSavepoint(Savepoint savepoint) throws SQLException { throw SqlExceptionUtils.clientError("Cannot release savepoint in auto-commit mode"); } - if (!isJdbcCompliant()) { + if (!jdbcConf.isJdbcCompliant()) { throw SqlExceptionUtils.unsupportedError("rollback not implemented"); } @@ -524,62 +506,59 @@ public PreparedStatement prepareStatement(String sql, int resultSetType, int res int resultSetHoldability) throws SQLException { ensureOpen(); - JdbcParameterizedQuery preparedQuery; + // TODO remove the extra parsing + ClickHouseSqlStatement[] stmts = parse(sql, clientRequest.getConfig()); + if (stmts.length != 1) { + throw SqlExceptionUtils + .clientError("Prepared statement only supports one query but we got: " + stmts.length); + } + ClickHouseSqlStatement parsedStmt = stmts[0]; + + ClickHouseParameterizedQuery preparedQuery; try { - preparedQuery = JdbcParameterizedQuery.of(sql); + preparedQuery = jdbcConf.useNamedParameter() ? ClickHouseParameterizedQuery.of(parsedStmt.getSQL()) + : JdbcParameterizedQuery.of(parsedStmt.getSQL()); } catch (RuntimeException e) { throw SqlExceptionUtils.clientError(e); } PreparedStatement ps = null; - if (!preparedQuery.hasParameter()) { - // is it insert statement using input function? - int index = 0; - int len = sql.length(); - List columns = Collections.emptyList(); - index = ClickHouseUtils.skipContentsUntil(sql, index, len, new String[] { "insert", "into" }, false); - if (index < len) { // insert statement - index = ClickHouseUtils.skipContentsUntil(sql, index, len, new String[] { "from", "input" }, false); - List params = new ArrayList<>(); - index = ClickHouseUtils.readParameters(sql, index, len, params); - - if (params.size() == 1) { - ps = new StreamBasedPreparedStatement(this, clientRequest.write().query(sql, newQueryId()), - ClickHouseColumn.parse(ClickHouseUtils.unescape(params.get(0))), resultSetType, - resultSetConcurrency, resultSetHoldability); - } + if (preparedQuery.hasParameter()) { + if (parsedStmt.hasTempTable() || parsedStmt.hasInput()) { + throw SqlExceptionUtils + .clientError( + "External table, input function, and query parameter cannot be used together in PreparedStatement."); + } + if (parsedStmt.hasValues()) { // consolidate multiple inserts into one + + } + } else { + if (parsedStmt.hasTempTable()) { + // non-insert queries using temp table + ps = new TableBasedPreparedStatement(this, + clientRequest.write().query(parsedStmt.getSQL(), newQueryId()), + parsedStmt.getTempTables(), resultSetType, + resultSetConcurrency, resultSetHoldability); + } else if (parsedStmt.getStatementType() == StatementType.INSERT + && !ClickHouseChecker.isNullOrBlank(parsedStmt.getInput())) { + // insert query using input function + ps = new InputBasedPreparedStatement(this, + clientRequest.write().query(parsedStmt.getSQL(), newQueryId()), + ClickHouseColumn.parse(parsedStmt.getInput()), resultSetType, + resultSetConcurrency, resultSetHoldability); } } return ps != null ? ps : new SqlBasedPreparedStatement(this, clientRequest.copy().query(preparedQuery, newQueryId()), - preparedQuery, resultSetType, resultSetConcurrency, resultSetHoldability); - } - - @Override - public Clob createClob() throws SQLException { - return createNClob(); - } - - @Override - public Blob createBlob() throws SQLException { - ensureOpen(); - - return new ClickHouseBlob(); + stmts[0], resultSetType, resultSetConcurrency, resultSetHoldability); } @Override public NClob createNClob() throws SQLException { ensureOpen(); - return new ClickHouseClob(); - } - - @Override - public SQLXML createSQLXML() throws SQLException { - ensureOpen(); - - return new ClickHouseXml(); + return createClob(); } @Override @@ -692,21 +671,6 @@ public Properties getClientInfo() throws SQLException { return props; } - @Override - public Array createArrayOf(String typeName, Object[] elements) throws SQLException { - // TODO Auto-generated method stub - // return new - // ClickHouseArray(ClickHouseDataType.resolveDefaultArrayDataType(typeName), - // elements); - return null; - } - - @Override - public Struct createStruct(String typeName, Object[] attributes) throws SQLException { - // TODO Auto-generated method stub - return null; - } - @Override public void setSchema(String schema) throws SQLException { ensureOpen(); @@ -809,8 +773,8 @@ public URI getUri() { } @Override - public boolean isJdbcCompliant() { - return jdbcCompliant; + public JdbcConfig getJdbcConfig() { + return jdbcConf; } @Override @@ -818,4 +782,27 @@ public String newQueryId() { FakeTransaction tx = fakeTransaction.get(); return tx != null ? tx.newQuery(null) : UUID.randomUUID().toString(); } + + @Override + public ClickHouseSqlStatement[] parse(String sql, ClickHouseConfig config) { + return ClickHouseSqlParser.parse(sql, config != null ? config : clientRequest.getConfig(), + jdbcConf.isJdbcCompliant() ? JdbcParseHandler.INSTANCE : null); + } + + @Override + public boolean isWrapperFor(Class iface) throws SQLException { + return iface == ClickHouseClient.class || iface == ClickHouseRequest.class + || super.isWrapperFor(iface); + } + + @Override + public T unwrap(Class iface) throws SQLException { + if (iface == ClickHouseClient.class) { + return iface.cast(client); + } else if (iface == ClickHouseRequest.class) { + return iface.cast(clientRequest); + } + + return super.unwrap(iface); + } } 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 adb436540..435799260 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 @@ -5,6 +5,7 @@ import java.net.URISyntaxException; import java.net.URLDecoder; import java.nio.charset.StandardCharsets; +import java.sql.SQLException; import java.util.Properties; import com.clickhouse.client.ClickHouseChecker; @@ -17,15 +18,16 @@ import com.clickhouse.client.config.ClickHouseDefaults; import com.clickhouse.client.logging.Logger; import com.clickhouse.client.logging.LoggerFactory; +import com.clickhouse.jdbc.JdbcConfig; +import com.clickhouse.jdbc.SqlExceptionUtils; public class ClickHouseJdbcUrlParser { private static final Logger log = LoggerFactory.getLogger(ClickHouseJdbcUrlParser.class); - public static final String PROP_JDBC_COMPLIANT = "jdbc_compliant"; - public static class ConnectionInfo { private final URI uri; private final ClickHouseNode server; + private final JdbcConfig jdbcConf; private final Properties props; protected ConnectionInfo(URI uri, ClickHouseNode server, Properties props) throws URISyntaxException { @@ -33,6 +35,7 @@ protected ConnectionInfo(URI uri, ClickHouseNode server, Properties props) throw server.getPort(), "/" + server.getDatabase().orElse(""), removeCredentialsFromQuery(uri.getRawQuery()), null); this.server = server; + this.jdbcConf = new JdbcConfig(props); this.props = props; } @@ -44,6 +47,10 @@ public ClickHouseNode getServer() { return server; } + public JdbcConfig getJdbcConfig() { + return jdbcConf; + } + public Properties getProperties() { return props; } @@ -178,32 +185,42 @@ static Properties newProperties() { return props; } - public static ConnectionInfo parse(String jdbcUrl, Properties defaults) throws URISyntaxException { + public static ConnectionInfo parse(String jdbcUrl, Properties defaults) throws SQLException { if (defaults == null) { defaults = new Properties(); } - if (ClickHouseChecker.nonBlank(jdbcUrl, "JDBC URL").startsWith(JDBC_CLICKHOUSE_PREFIX)) { + if (ClickHouseChecker.isNullOrBlank(jdbcUrl)) { + throw SqlExceptionUtils.clientError("Non-blank JDBC URL is required"); + } + + if (jdbcUrl.startsWith(JDBC_CLICKHOUSE_PREFIX)) { jdbcUrl = jdbcUrl.substring(JDBC_CLICKHOUSE_PREFIX.length()); } else if (jdbcUrl.startsWith(JDBC_ABBREVIATION_PREFIX)) { jdbcUrl = jdbcUrl.substring(JDBC_ABBREVIATION_PREFIX.length()); } else { - throw new URISyntaxException(jdbcUrl, ClickHouseUtils.format("'%s' or '%s' prefix is mandatory", - JDBC_CLICKHOUSE_PREFIX, JDBC_ABBREVIATION_PREFIX)); + throw SqlExceptionUtils.clientError( + new URISyntaxException(jdbcUrl, ClickHouseUtils.format("'%s' or '%s' prefix is mandatory", + JDBC_CLICKHOUSE_PREFIX, JDBC_ABBREVIATION_PREFIX))); } int index = jdbcUrl.indexOf("//"); if (index == -1) { - throw new URISyntaxException(jdbcUrl, "Missing '//' from the given JDBC URL"); + throw SqlExceptionUtils + .clientError(new URISyntaxException(jdbcUrl, "Missing '//' from the given JDBC URL")); } else if (index == 0) { jdbcUrl = "http:" + jdbcUrl; } - URI uri = new URI(jdbcUrl); - Properties props = newProperties(); - props.putAll(defaults); - parseParams(uri.getQuery(), props); - return new ConnectionInfo(uri, parseNode(uri, props), props); + try { + URI uri = new URI(jdbcUrl); + Properties props = newProperties(); + props.putAll(defaults); + parseParams(uri.getQuery(), props); + return new ConnectionInfo(uri, parseNode(uri, props), props); + } catch (URISyntaxException | IllegalArgumentException e) { + throw SqlExceptionUtils.clientError(e); + } } private ClickHouseJdbcUrlParser() { 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 2d368c76d..f31ef9fe9 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 @@ -10,9 +10,9 @@ import com.clickhouse.client.ClickHouseUtils; import com.clickhouse.jdbc.JdbcTypeMapping; import com.clickhouse.jdbc.SqlExceptionUtils; -import com.clickhouse.jdbc.Wrapper; +import com.clickhouse.jdbc.JdbcWrapper; -public class ClickHouseParameterMetaData extends Wrapper implements ParameterMetaData { +public class ClickHouseParameterMetaData extends JdbcWrapper implements ParameterMetaData { protected final List params; protected ClickHouseParameterMetaData(List params) { 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 aa35fdb11..d71911d27 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseStatementImpl.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/ClickHouseStatementImpl.java @@ -10,6 +10,7 @@ import java.util.List; import java.util.Map; import java.util.UUID; +import java.util.Map.Entry; import com.clickhouse.client.ClickHouseChecker; import com.clickhouse.client.ClickHouseConfig; @@ -26,14 +27,11 @@ import com.clickhouse.jdbc.ClickHouseConnection; import com.clickhouse.jdbc.ClickHouseResultSet; import com.clickhouse.jdbc.ClickHouseStatement; -import com.clickhouse.jdbc.JdbcParseHandler; import com.clickhouse.jdbc.SqlExceptionUtils; -import com.clickhouse.jdbc.Wrapper; -import com.clickhouse.jdbc.parser.ClickHouseSqlParser; +import com.clickhouse.jdbc.JdbcWrapper; import com.clickhouse.jdbc.parser.ClickHouseSqlStatement; -import com.clickhouse.jdbc.parser.ParseHandler; -public class ClickHouseStatementImpl extends Wrapper implements ClickHouseStatement { +public class ClickHouseStatementImpl extends JdbcWrapper implements ClickHouseStatement { private static final Logger log = LoggerFactory.getLogger(ClickHouseStatementImpl.class); private final ClickHouseConnection connection; @@ -113,23 +111,57 @@ protected void ensureOpen() throws SQLException { } } - protected int executeStatement(ClickHouseSqlStatement stmt, Map options, - List tables, Map settings) throws SQLException { - stmt = applyFormat(stmt, request.getFormat()); + protected ClickHouseResponse executeStatement(String stmt, + Map options, List tables, + Map settings) throws SQLException { + try { + if (options != null) { + request.options(options); + } + if (settings != null && !settings.isEmpty()) { + if (request.getSessionId().isEmpty()) { + request.session(UUID.randomUUID().toString()); + } + for (Entry e : settings.entrySet()) { + request.set(e.getKey(), e.getValue()); + } + } + if (tables != null && !tables.isEmpty()) { + List list = new ArrayList<>(tables.size()); + for (ClickHouseExternalTable t : tables) { + if (t.isTempTable()) { + if (request.getSessionId().isEmpty()) { + request.session(UUID.randomUUID().toString()); + } + request.query("drop temporary table if exists `" + t.getName() + "`").execute().get(); + request.query("create temporary table `" + t.getName() + "`(" + t.getStructure() + ")") + .execute().get(); + request.write() + .table(t.getName()) + .format(t.getFormat() != null ? t.getFormat() : ClickHouseFormat.RowBinary) + .data(t.getContent()).send().get(); + } else { + list.add(t); + } + } + request.external(list); + } - ClickHouseResponseSummary summary = null; - try (ClickHouseResponse resp = request.query(stmt.getSQL(), queryId = connection.newQueryId()).execute() - .get()) { - summary = resp.getSummary(); + return request.query(stmt, queryId = connection.newQueryId()).execute().get(); } catch (InterruptedException e) { log.error("can not close stream: %s", e.getMessage()); Thread.currentThread().interrupt(); - SqlExceptionUtils.forCancellation(e); + throw SqlExceptionUtils.forCancellation(e); } catch (Exception e) { throw SqlExceptionUtils.handle(e); } + } - return summary != null ? (int) summary.getWrittenRows() : 1; + 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); } protected int executeInsert(String sql, InputStream input) throws SQLException { @@ -141,7 +173,7 @@ protected int executeInsert(String sql, InputStream input) throws SQLException { } catch (InterruptedException e) { log.error("can not close stream: %s", e.getMessage()); Thread.currentThread().interrupt(); - SqlExceptionUtils.forCancellation(e); + throw SqlExceptionUtils.forCancellation(e); } catch (Exception e) { throw SqlExceptionUtils.handle(e); } @@ -166,8 +198,7 @@ protected void setLastStatement(ClickHouseSqlStatement stmt) { } protected ClickHouseSqlStatement parseSqlStatements(String sql) { - parsedStmts = ClickHouseSqlParser.parse(sql, getConfig(), - connection.isJdbcCompliant() ? JdbcParseHandler.INSTANCE : null); + parsedStmts = connection.parse(sql, getConfig()); if (parsedStmts == null || parsedStmts.length == 0) { // should never happen @@ -185,21 +216,21 @@ protected ClickHouseSqlStatement parseSqlStatements(String sql) { protected ResultSet updateResult(ClickHouseSqlStatement stmt, ClickHouseResponse response) throws SQLException { ResultSet rs = null; - if (stmt.isQuery()) { + if (stmt.isQuery() || !response.getColumns().isEmpty()) { currentUpdateCount = -1; currentResult = new ClickHouseResultSet(stmt.getDatabaseOrDefault(getConnection().getCurrentDatabase()), stmt.getTable(), this, response); rs = currentResult; } else { - currentUpdateCount = 0; + currentUpdateCount = response.getSummary().getUpdateCount(); response.close(); } return rs; } - protected ClickHouseStatementImpl(ClickHouseConnection connection, ClickHouseRequest request, int resultSetType, - int resultSetConcurrency, int resultSetHoldability) throws SQLException { + protected ClickHouseStatementImpl(ClickHouseConnectionImpl connection, ClickHouseRequest request, + int resultSetType, int resultSetConcurrency, int resultSetHoldability) throws SQLException { if (connection == null || request == null) { throw SqlExceptionUtils.clientError("Non-null connection and request are required"); } @@ -215,7 +246,7 @@ protected ClickHouseStatementImpl(ClickHouseConnection connection, ClickHouseReq this.closed = false; this.closeOnCompletion = true; - this.fetchSize = 0; + this.fetchSize = connection.getJdbcConfig().getFetchSize(); this.maxFieldSize = 0; this.maxRows = 0; this.poolable = false; @@ -476,11 +507,7 @@ public int getResultSetType() throws SQLException { public void addBatch(String sql) throws SQLException { ensureOpen(); - ParseHandler handler = null; - if (connection.isJdbcCompliant()) { - handler = JdbcParseHandler.INSTANCE; - } - for (ClickHouseSqlStatement s : ClickHouseSqlParser.parse(sql, getConfig(), handler)) { + for (ClickHouseSqlStatement s : connection.parse(sql, getConfig())) { this.batchStmts.add(s); } } @@ -499,8 +526,8 @@ public int[] executeBatch() throws SQLException { int len = batchStmts.size(); int[] results = new int[len]; for (int i = 0; i < len; i++) { - try { - results[i] = executeStatement(batchStmts.get(i), null, null, null); + try (ClickHouseResponse r = executeStatement(batchStmts.get(i), null, null, null)) { + results[i] = (int) r.getSummary().getWrittenRows(); } catch (Exception e) { results[i] = EXECUTE_FAILED; log.error("Faled to execute task %d of %d", i + 1, len, e); @@ -625,4 +652,14 @@ public ClickHouseConfig getConfig() { public ClickHouseRequest getRequest() { return request; } + + @Override + public boolean isWrapperFor(Class iface) throws SQLException { + return iface == ClickHouseRequest.class || super.isWrapperFor(iface); + } + + @Override + public T unwrap(Class iface) throws SQLException { + return iface == ClickHouseRequest.class ? iface.cast(request) : super.unwrap(iface); + } } diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/StreamBasedPreparedStatement.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/InputBasedPreparedStatement.java similarity index 55% rename from clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/StreamBasedPreparedStatement.java rename to clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/InputBasedPreparedStatement.java index 3009088c7..6e3d12e71 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/StreamBasedPreparedStatement.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/InputBasedPreparedStatement.java @@ -2,26 +2,16 @@ import java.io.IOException; import java.io.InputStream; -import java.io.Reader; import java.math.BigDecimal; -import java.net.URL; import java.nio.charset.StandardCharsets; import java.sql.Array; -import java.sql.Blob; -import java.sql.Clob; import java.sql.Date; -import java.sql.NClob; import java.sql.ParameterMetaData; -import java.sql.PreparedStatement; -import java.sql.Ref; import java.sql.ResultSet; -import java.sql.ResultSetMetaData; -import java.sql.RowId; import java.sql.SQLException; -import java.sql.SQLXML; import java.sql.Time; import java.sql.Timestamp; -import java.sql.Types; +import java.time.LocalDate; import java.time.LocalDateTime; import java.time.ZoneId; import java.util.ArrayList; @@ -31,22 +21,20 @@ import com.clickhouse.client.ClickHouseColumn; import com.clickhouse.client.ClickHouseConfig; -import com.clickhouse.client.ClickHouseInputStream; import com.clickhouse.client.ClickHouseRequest; import com.clickhouse.client.ClickHouseUtils; import com.clickhouse.client.ClickHouseValue; import com.clickhouse.client.ClickHouseValues; -import com.clickhouse.client.data.BinaryStreamUtils; import com.clickhouse.client.data.ClickHousePipedStream; import com.clickhouse.client.data.ClickHouseRowBinaryProcessor; import com.clickhouse.client.data.ClickHouseRowBinaryProcessor.MappedFunctions; import com.clickhouse.client.logging.Logger; import com.clickhouse.client.logging.LoggerFactory; -import com.clickhouse.jdbc.ClickHouseConnection; +import com.clickhouse.jdbc.ClickHousePreparedStatement; import com.clickhouse.jdbc.SqlExceptionUtils; -public class StreamBasedPreparedStatement extends ClickHouseStatementImpl implements PreparedStatement { - private static final Logger log = LoggerFactory.getLogger(StreamBasedPreparedStatement.class); +public class InputBasedPreparedStatement extends ClickHouseStatementImpl implements ClickHousePreparedStatement { + private static final Logger log = LoggerFactory.getLogger(InputBasedPreparedStatement.class); private final Calendar defaultCalendar; private final ZoneId jvmZoneId; @@ -58,11 +46,15 @@ public class StreamBasedPreparedStatement extends ClickHouseStatementImpl implem private ClickHousePipedStream stream; private final List batch; - protected StreamBasedPreparedStatement(ClickHouseConnection connection, ClickHouseRequest request, + protected InputBasedPreparedStatement(ClickHouseConnectionImpl connection, ClickHouseRequest request, List columns, int resultSetType, int resultSetConcurrency, int resultSetHoldability) throws SQLException { super(connection, request, resultSetType, resultSetConcurrency, resultSetHoldability); + if (columns == null) { + throw SqlExceptionUtils.clientError("Non-null column list is required"); + } + defaultCalendar = connection.getDefaultCalendar(); jvmZoneId = connection.getJvmTimeZone().toZoneId(); @@ -104,9 +96,7 @@ protected int toArrayIndex(int parameterIndex) throws SQLException { @Override public ResultSet executeQuery() throws SQLException { - ensureParams(); - - return null; // return executeQuery(preparedQuery.apply(values)); + throw SqlExceptionUtils.clientError("Input function can be only used for insertion not query"); } @Override @@ -117,20 +107,6 @@ public int executeUpdate() throws SQLException { return executeBatch()[0]; } - @Override - public void setNull(int parameterIndex, int sqlType) throws SQLException { - setNull(parameterIndex, sqlType, null); - } - - @Override - public void setBoolean(int parameterIndex, boolean x) throws SQLException { - ensureOpen(); - - int idx = toArrayIndex(parameterIndex); - values[idx].update(x); - flags[idx] = true; - } - @Override public void setByte(int parameterIndex, byte x) throws SQLException { ensureOpen(); @@ -212,54 +188,6 @@ public void setBytes(int parameterIndex, byte[] x) throws SQLException { flags[idx] = true; } - @Override - public void setDate(int parameterIndex, Date x) throws SQLException { - setDate(parameterIndex, x, null); - } - - @Override - public void setTime(int parameterIndex, Time x) throws SQLException { - setTime(parameterIndex, x, null); - } - - @Override - public void setTimestamp(int parameterIndex, Timestamp x) throws SQLException { - setTimestamp(parameterIndex, x, null); - } - - @Override - public void setAsciiStream(int parameterIndex, InputStream x, int length) throws SQLException { - String s = null; - if (x != null) { - try { - s = BinaryStreamUtils.readFixedString(ClickHouseInputStream.of(x), length, StandardCharsets.US_ASCII); - } catch (Throwable e) { // IOException and potentially OOM error - throw SqlExceptionUtils.clientError(e); - } - } - - setString(parameterIndex, s); - } - - @Override - public void setUnicodeStream(int parameterIndex, InputStream x, int length) throws SQLException { - String s = null; - if (x != null) { - try { - s = BinaryStreamUtils.readFixedString(ClickHouseInputStream.of(x), length, StandardCharsets.UTF_8); - } catch (Throwable e) { // IOException and potentially OOM error - throw SqlExceptionUtils.clientError(e); - } - } - - setString(parameterIndex, s); - } - - @Override - public void setBinaryStream(int parameterIndex, InputStream x, int length) throws SQLException { - setUnicodeStream(parameterIndex, x, length); - } - @Override public void clearParameters() throws SQLException { ensureOpen(); @@ -281,11 +209,6 @@ public void clearParameters() throws SQLException { } } - @Override - public void setObject(int parameterIndex, Object x, int targetSqlType) throws SQLException { - setObject(parameterIndex, x, targetSqlType, 0); - } - @Override public void setObject(int parameterIndex, Object x) throws SQLException { ensureOpen(); @@ -299,7 +222,9 @@ public void setObject(int parameterIndex, Object x) throws SQLException { public boolean execute() throws SQLException { ensureParams(); - return false; // execute(preparedQuery.apply(values)); + addBatch(); + executeBatch(); + return false; } @Override @@ -328,6 +253,7 @@ public void addBatch() throws SQLException { } batch.add(stream.getInput()); + // stream.close(); clearParameters(); } @@ -361,63 +287,38 @@ public void clearBatch() throws SQLException { } @Override - public void setCharacterStream(int parameterIndex, Reader reader, int length) throws SQLException { - String s = null; - if (reader != null) { - try { - s = BinaryStreamUtils.readString(reader, length); - } catch (Throwable e) { // IOException and potentially OOM error - throw SqlExceptionUtils.clientError(e); - } - } + public void setArray(int parameterIndex, Array x) throws SQLException { + ensureOpen(); - setString(parameterIndex, s); + int idx = toArrayIndex(parameterIndex); + Object array = x != null ? x.getArray() : x; + values[idx].update(array); + flags[idx] = true; } @Override - public void setRef(int parameterIndex, Ref x) throws SQLException { - // TODO Auto-generated method stub - - } + public void setDate(int parameterIndex, Date x, Calendar cal) throws SQLException { + ensureOpen(); - @Override - public void setBlob(int parameterIndex, Blob x) throws SQLException { + int idx = toArrayIndex(parameterIndex); if (x != null) { - setBinaryStream(parameterIndex, x.getBinaryStream()); + LocalDate d = null; + if (cal != null) { + d = x.toLocalDate().atStartOfDay(jvmZoneId) + .withZoneSameInstant(cal.getTimeZone().toZoneId()).toLocalDate(); + } else { + d = x.toLocalDate(); + } + values[idx].update(d); } else { - setNull(parameterIndex, Types.BINARY); + values[idx].resetToNullOrEmpty(); } - } - - @Override - public void setClob(int parameterIndex, Clob x) throws SQLException { - // TODO Auto-generated method stub - - } - - @Override - public void setArray(int parameterIndex, Array x) throws SQLException { - // TODO Auto-generated method stub - - } - - @Override - public ResultSetMetaData getMetaData() throws SQLException { - ResultSet currentResult = getResultSet(); - - return currentResult != null ? currentResult.getMetaData() : null; - } - - @Override - public void setDate(int parameterIndex, Date x, Calendar cal) throws SQLException { - // TODO Auto-generated method stub - + flags[idx] = true; } @Override public void setTime(int parameterIndex, Time x, Calendar cal) throws SQLException { - // TODO Auto-generated method stub - + throw SqlExceptionUtils.clientError("setTime not implemented"); } @Override @@ -452,77 +353,12 @@ public void setNull(int parameterIndex, int sqlType, String typeName) throws SQL flags[idx] = true; } - @Override - public void setURL(int parameterIndex, URL x) throws SQLException { - setString(parameterIndex, String.valueOf(x)); - } - @Override public ParameterMetaData getParameterMetaData() throws SQLException { // TODO Auto-generated method stub return null; } - @Override - public void setRowId(int parameterIndex, RowId x) throws SQLException { - ensureOpen(); - - toArrayIndex(parameterIndex); - - throw SqlExceptionUtils.unsupportedError("setRowId not implemented"); - } - - @Override - public void setNString(int parameterIndex, String value) throws SQLException { - setString(parameterIndex, value); - } - - @Override - public void setNCharacterStream(int parameterIndex, Reader value, long length) throws SQLException { - setCharacterStream(parameterIndex, value, length); - } - - @Override - public void setNClob(int parameterIndex, NClob value) throws SQLException { - ensureOpen(); - - toArrayIndex(parameterIndex); - - throw SqlExceptionUtils.unsupportedError("setNClob not implemented"); - } - - @Override - public void setClob(int parameterIndex, Reader reader, long length) throws SQLException { - ensureOpen(); - - toArrayIndex(parameterIndex); - - throw SqlExceptionUtils.unsupportedError("setClob not implemented"); - } - - @Override - public void setBlob(int parameterIndex, InputStream inputStream, long length) throws SQLException { - ensureOpen(); - - toArrayIndex(parameterIndex); - - throw SqlExceptionUtils.unsupportedError("setBlob not implemented"); - } - - @Override - public void setNClob(int parameterIndex, Reader reader, long length) throws SQLException { - setClob(parameterIndex, reader, length); - } - - @Override - public void setSQLXML(int parameterIndex, SQLXML xmlObject) throws SQLException { - ensureOpen(); - - toArrayIndex(parameterIndex); - - throw SqlExceptionUtils.unsupportedError("setSQLXML not implemented"); - } - @Override public void setObject(int parameterIndex, Object x, int targetSqlType, int scaleOrLength) throws SQLException { ensureOpen(); @@ -531,71 +367,4 @@ public void setObject(int parameterIndex, Object x, int targetSqlType, int scale values[idx].update(x); flags[idx] = true; } - - @Override - public void setAsciiStream(int parameterIndex, InputStream x, long length) throws SQLException { - ensureOpen(); - - toArrayIndex(parameterIndex); - - throw SqlExceptionUtils.unsupportedError("setAsciiStream not implemented"); - } - - @Override - public void setBinaryStream(int parameterIndex, InputStream x, long length) throws SQLException { - ensureOpen(); - - toArrayIndex(parameterIndex); - - throw SqlExceptionUtils.unsupportedError("setBinaryStream not implemented"); - } - - @Override - public void setCharacterStream(int parameterIndex, Reader reader, long length) throws SQLException { - ensureOpen(); - - toArrayIndex(parameterIndex); - - throw SqlExceptionUtils.unsupportedError("setCharacterStream not implemented"); - } - - @Override - public void setAsciiStream(int parameterIndex, InputStream x) throws SQLException { - // TODO Auto-generated method stub - - } - - @Override - public void setBinaryStream(int parameterIndex, InputStream x) throws SQLException { - // TODO Auto-generated method stub - - } - - @Override - public void setCharacterStream(int parameterIndex, Reader reader) throws SQLException { - // TODO Auto-generated method stub - - } - - @Override - public void setNCharacterStream(int parameterIndex, Reader value) throws SQLException { - setCharacterStream(parameterIndex, value); - } - - @Override - public void setClob(int parameterIndex, Reader reader) throws SQLException { - // TODO Auto-generated method stub - - } - - @Override - public void setBlob(int parameterIndex, InputStream inputStream) throws SQLException { - // TODO Auto-generated method stub - - } - - @Override - public void setNClob(int parameterIndex, Reader reader) throws SQLException { - setClob(parameterIndex, reader); - } } diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/SqlBasedPreparedStatement.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/SqlBasedPreparedStatement.java index 8e68e7b94..a715b664d 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/SqlBasedPreparedStatement.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/SqlBasedPreparedStatement.java @@ -1,72 +1,63 @@ package com.clickhouse.jdbc.internal; -import java.io.InputStream; -import java.io.Reader; import java.math.BigDecimal; -import java.net.URL; import java.nio.charset.StandardCharsets; import java.sql.Array; -import java.sql.Blob; -import java.sql.Clob; import java.sql.Date; -import java.sql.NClob; import java.sql.ParameterMetaData; -import java.sql.PreparedStatement; -import java.sql.Ref; import java.sql.ResultSet; -import java.sql.ResultSetMetaData; -import java.sql.RowId; import java.sql.SQLException; -import java.sql.SQLXML; import java.sql.Time; import java.sql.Timestamp; -import java.sql.Types; +import java.time.LocalDate; import java.time.LocalDateTime; import java.time.ZoneId; import java.util.ArrayList; import java.util.Calendar; -import java.util.GregorianCalendar; import java.util.LinkedList; import java.util.List; -import java.util.TimeZone; -import com.clickhouse.client.ClickHouseInputStream; +import com.clickhouse.client.ClickHouseParameterizedQuery; import com.clickhouse.client.ClickHouseRequest; +import com.clickhouse.client.ClickHouseResponse; import com.clickhouse.client.ClickHouseUtils; import com.clickhouse.client.ClickHouseValue; import com.clickhouse.client.ClickHouseValues; -import com.clickhouse.client.data.BinaryStreamUtils; import com.clickhouse.client.data.ClickHouseDateTimeValue; +import com.clickhouse.client.data.ClickHouseDateValue; import com.clickhouse.client.logging.Logger; import com.clickhouse.client.logging.LoggerFactory; -import com.clickhouse.jdbc.ClickHouseConnection; -import com.clickhouse.jdbc.JdbcParameterizedQuery; +import com.clickhouse.jdbc.ClickHousePreparedStatement; import com.clickhouse.jdbc.JdbcTypeMapping; import com.clickhouse.jdbc.SqlExceptionUtils; import com.clickhouse.jdbc.parser.ClickHouseSqlStatement; -public class SqlBasedPreparedStatement extends ClickHouseStatementImpl implements PreparedStatement { +public class SqlBasedPreparedStatement extends ClickHouseStatementImpl implements ClickHousePreparedStatement { private static final Logger log = LoggerFactory.getLogger(SqlBasedPreparedStatement.class); private final Calendar defaultCalendar; private final ZoneId jvmZoneId; - private final JdbcParameterizedQuery preparedQuery; + private final ClickHouseSqlStatement parsedStmt; + private final ClickHouseParameterizedQuery preparedQuery; private final ClickHouseValue[] templates; private final String[] values; private final List batch; - protected SqlBasedPreparedStatement(ClickHouseConnection connection, ClickHouseRequest request, - JdbcParameterizedQuery preparedQuery, int resultSetType, int resultSetConcurrency, int resultSetHoldability) + protected SqlBasedPreparedStatement(ClickHouseConnectionImpl connection, ClickHouseRequest request, + ClickHouseSqlStatement parsedStmt, int resultSetType, int resultSetConcurrency, int resultSetHoldability) throws SQLException { super(connection, request, resultSetType, resultSetConcurrency, resultSetHoldability); defaultCalendar = connection.getDefaultCalendar(); jvmZoneId = connection.getJvmTimeZone().toZoneId(); - this.preparedQuery = preparedQuery; + this.parsedStmt = parsedStmt; - templates = new ClickHouseValue[preparedQuery.getNamedParameters().size()]; + preparedQuery = request.getPreparedQuery(); + + templates = preparedQuery.getParameterTemplates(); + values = new String[templates.length]; batch = new LinkedList<>(); } @@ -97,6 +88,7 @@ protected int toArrayIndex(int parameterIndex) throws SQLException { public ResultSet executeQuery() throws SQLException { ensureParams(); + // FIXME ResultSet should never be null return executeQuery(preparedQuery.apply(values)); } @@ -107,25 +99,6 @@ public int executeUpdate() throws SQLException { return executeUpdate(preparedQuery.apply(values)); } - @Override - public void setNull(int parameterIndex, int sqlType) throws SQLException { - setNull(parameterIndex, sqlType, null); - } - - @Override - public void setBoolean(int parameterIndex, boolean x) throws SQLException { - ensureOpen(); - - int idx = toArrayIndex(parameterIndex); - ClickHouseValue value = templates[idx]; - if (value != null) { - value.update(x); - values[idx] = value.toSqlExpression(); - } else { - values[idx] = x ? "1" : "0"; - } - } - @Override public void setByte(int parameterIndex, byte x) throws SQLException { ensureOpen(); @@ -252,54 +225,6 @@ public void setBytes(int parameterIndex, byte[] x) throws SQLException { } } - @Override - public void setDate(int parameterIndex, Date x) throws SQLException { - setDate(parameterIndex, x, null); - } - - @Override - public void setTime(int parameterIndex, Time x) throws SQLException { - setTime(parameterIndex, x, null); - } - - @Override - public void setTimestamp(int parameterIndex, Timestamp x) throws SQLException { - setTimestamp(parameterIndex, x, null); - } - - @Override - public void setAsciiStream(int parameterIndex, InputStream x, int length) throws SQLException { - String s = null; - if (x != null) { - try { - s = BinaryStreamUtils.readFixedString(ClickHouseInputStream.of(x), length, StandardCharsets.US_ASCII); - } catch (Throwable e) { // IOException and potentially OOM error - throw SqlExceptionUtils.clientError(e); - } - } - - setString(parameterIndex, s); - } - - @Override - public void setUnicodeStream(int parameterIndex, InputStream x, int length) throws SQLException { - String s = null; - if (x != null) { - try { - s = BinaryStreamUtils.readFixedString(ClickHouseInputStream.of(x), length, StandardCharsets.UTF_8); - } catch (Throwable e) { // IOException and potentially OOM error - throw SqlExceptionUtils.clientError(e); - } - } - - setString(parameterIndex, s); - } - - @Override - public void setBinaryStream(int parameterIndex, InputStream x, int length) throws SQLException { - setUnicodeStream(parameterIndex, x, length); - } - @Override public void clearParameters() throws SQLException { ensureOpen(); @@ -309,11 +234,6 @@ public void clearParameters() throws SQLException { } } - @Override - public void setObject(int parameterIndex, Object x, int targetSqlType) throws SQLException { - setObject(parameterIndex, x, targetSqlType, 0); - } - @Override public void setObject(int parameterIndex, Object x) throws SQLException { ensureOpen(); @@ -341,14 +261,6 @@ public boolean execute() throws SQLException { return execute(preparedQuery.apply(values)); } - @Override - public void addBatch(String sql) throws SQLException { - ensureOpen(); - - throw SqlExceptionUtils - .unsupportedError("addBatch(String) cannot be called in PreparedStatement or CallableStatement!"); - } - @Override public void addBatch() throws SQLException { ensureOpen(); @@ -375,9 +287,8 @@ public int[] executeBatch() throws SQLException { int[] results = new int[len]; int counter = 0; for (String[] params : batch) { - try { - results[counter] = executeStatement(new ClickHouseSqlStatement(preparedQuery.apply(params)), null, null, - null); + try (ClickHouseResponse r = executeStatement(preparedQuery.apply(params), null, null, null)) { + results[counter] = (int) r.getSummary().getWrittenRows(); } catch (Exception e) { results[counter] = EXECUTE_FAILED; log.error("Failed to execute task %d of %d", counter + 1, len, e); @@ -398,63 +309,43 @@ public void clearBatch() throws SQLException { } @Override - public void setCharacterStream(int parameterIndex, Reader reader, int length) throws SQLException { - String s = null; - if (reader != null) { - try { - s = BinaryStreamUtils.readString(reader, length); - } catch (Throwable e) { // IOException and potentially OOM error - throw SqlExceptionUtils.clientError(e); - } - } + public void setArray(int parameterIndex, Array x) throws SQLException { + ensureOpen(); - setString(parameterIndex, s); + int idx = toArrayIndex(parameterIndex); + Object array = x != null ? x.getArray() : x; + values[idx] = array != null ? ClickHouseValues.convertToSqlExpression(array) + : ClickHouseValues.EMPTY_ARRAY_EXPR; } @Override - public void setRef(int parameterIndex, Ref x) throws SQLException { - // TODO Auto-generated method stub + public void setDate(int parameterIndex, Date x, Calendar cal) throws SQLException { + ensureOpen(); - } + int idx = toArrayIndex(parameterIndex); + if (x == null) { + values[idx] = ClickHouseValues.NULL_EXPR; + return; + } - @Override - public void setBlob(int parameterIndex, Blob x) throws SQLException { - if (x != null) { - setBinaryStream(parameterIndex, x.getBinaryStream()); + LocalDate d = null; + if (cal != null) { + d = x.toLocalDate().atStartOfDay(jvmZoneId) + .withZoneSameInstant(cal.getTimeZone().toZoneId()).toLocalDate(); } else { - setNull(parameterIndex, Types.BINARY); + d = x.toLocalDate(); } - } - - @Override - public void setClob(int parameterIndex, Clob x) throws SQLException { - // TODO Auto-generated method stub - - } - - @Override - public void setArray(int parameterIndex, Array x) throws SQLException { - // TODO Auto-generated method stub - - } - - @Override - public ResultSetMetaData getMetaData() throws SQLException { - ResultSet currentResult = getResultSet(); - - return currentResult != null ? currentResult.getMetaData() : null; - } - - @Override - public void setDate(int parameterIndex, Date x, Calendar cal) throws SQLException { - // TODO Auto-generated method stub + ClickHouseValue value = templates[idx]; + if (value == null) { + value = ClickHouseDateValue.ofNull(); + } + values[idx] = value.update(d).toSqlExpression(); } @Override public void setTime(int parameterIndex, Time x, Calendar cal) throws SQLException { - // TODO Auto-generated method stub - + throw SqlExceptionUtils.clientError("setTime not implemented"); } @Override @@ -496,77 +387,12 @@ public void setNull(int parameterIndex, int sqlType, String typeName) throws SQL } } - @Override - public void setURL(int parameterIndex, URL x) throws SQLException { - setString(parameterIndex, String.valueOf(x)); - } - @Override public ParameterMetaData getParameterMetaData() throws SQLException { // TODO Auto-generated method stub return null; } - @Override - public void setRowId(int parameterIndex, RowId x) throws SQLException { - ensureOpen(); - - toArrayIndex(parameterIndex); - - throw SqlExceptionUtils.unsupportedError("setRowId not implemented"); - } - - @Override - public void setNString(int parameterIndex, String value) throws SQLException { - setString(parameterIndex, value); - } - - @Override - public void setNCharacterStream(int parameterIndex, Reader value, long length) throws SQLException { - setCharacterStream(parameterIndex, value, length); - } - - @Override - public void setNClob(int parameterIndex, NClob value) throws SQLException { - ensureOpen(); - - toArrayIndex(parameterIndex); - - throw SqlExceptionUtils.unsupportedError("setNClob not implemented"); - } - - @Override - public void setClob(int parameterIndex, Reader reader, long length) throws SQLException { - ensureOpen(); - - toArrayIndex(parameterIndex); - - throw SqlExceptionUtils.unsupportedError("setClob not implemented"); - } - - @Override - public void setBlob(int parameterIndex, InputStream inputStream, long length) throws SQLException { - ensureOpen(); - - toArrayIndex(parameterIndex); - - throw SqlExceptionUtils.unsupportedError("setBlob not implemented"); - } - - @Override - public void setNClob(int parameterIndex, Reader reader, long length) throws SQLException { - setClob(parameterIndex, reader, length); - } - - @Override - public void setSQLXML(int parameterIndex, SQLXML xmlObject) throws SQLException { - ensureOpen(); - - toArrayIndex(parameterIndex); - - throw SqlExceptionUtils.unsupportedError("setSQLXML not implemented"); - } - @Override public void setObject(int parameterIndex, Object x, int targetSqlType, int scaleOrLength) throws SQLException { ensureOpen(); @@ -581,71 +407,4 @@ public void setObject(int parameterIndex, Object x, int targetSqlType, int scale value.update(x); values[idx] = value.toSqlExpression(); } - - @Override - public void setAsciiStream(int parameterIndex, InputStream x, long length) throws SQLException { - ensureOpen(); - - toArrayIndex(parameterIndex); - - throw SqlExceptionUtils.unsupportedError("setAsciiStream not implemented"); - } - - @Override - public void setBinaryStream(int parameterIndex, InputStream x, long length) throws SQLException { - ensureOpen(); - - toArrayIndex(parameterIndex); - - throw SqlExceptionUtils.unsupportedError("setBinaryStream not implemented"); - } - - @Override - public void setCharacterStream(int parameterIndex, Reader reader, long length) throws SQLException { - ensureOpen(); - - toArrayIndex(parameterIndex); - - throw SqlExceptionUtils.unsupportedError("setCharacterStream not implemented"); - } - - @Override - public void setAsciiStream(int parameterIndex, InputStream x) throws SQLException { - // TODO Auto-generated method stub - - } - - @Override - public void setBinaryStream(int parameterIndex, InputStream x) throws SQLException { - // TODO Auto-generated method stub - - } - - @Override - public void setCharacterStream(int parameterIndex, Reader reader) throws SQLException { - // TODO Auto-generated method stub - - } - - @Override - public void setNCharacterStream(int parameterIndex, Reader value) throws SQLException { - setCharacterStream(parameterIndex, value); - } - - @Override - public void setClob(int parameterIndex, Reader reader) throws SQLException { - // TODO Auto-generated method stub - - } - - @Override - public void setBlob(int parameterIndex, InputStream inputStream) throws SQLException { - // TODO Auto-generated method stub - - } - - @Override - public void setNClob(int parameterIndex, Reader reader) throws SQLException { - setClob(parameterIndex, reader); - } } diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/TableBasedPreparedStatement.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/TableBasedPreparedStatement.java new file mode 100644 index 000000000..91d92fee9 --- /dev/null +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/TableBasedPreparedStatement.java @@ -0,0 +1,262 @@ +package com.clickhouse.jdbc.internal; + +import java.math.BigDecimal; +import java.sql.Array; +import java.sql.Date; +import java.sql.ParameterMetaData; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Time; +import java.sql.Timestamp; +import java.time.ZoneId; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Calendar; +import java.util.Collection; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; + +import com.clickhouse.client.ClickHouseRequest; +import com.clickhouse.client.ClickHouseResponse; +import com.clickhouse.client.ClickHouseUtils; +import com.clickhouse.client.data.ClickHouseExternalTable; +import com.clickhouse.client.logging.Logger; +import com.clickhouse.client.logging.LoggerFactory; +import com.clickhouse.jdbc.ClickHousePreparedStatement; +import com.clickhouse.jdbc.SqlExceptionUtils; +import com.clickhouse.jdbc.parser.ClickHouseSqlStatement; + +public class TableBasedPreparedStatement extends ClickHouseStatementImpl implements ClickHousePreparedStatement { + private static final Logger log = LoggerFactory.getLogger(TableBasedPreparedStatement.class); + + private static final String ERROR_SET_TABLE = "Please use setObject(ClickHouseExternalTable) method instead"; + + private final Calendar defaultCalendar; + private final ZoneId jvmZoneId; + + private final List tables; + private final ClickHouseExternalTable[] values; + + private final List> batch; + + protected TableBasedPreparedStatement(ClickHouseConnectionImpl connection, ClickHouseRequest request, + Collection tables, int resultSetType, int resultSetConcurrency, int resultSetHoldability) + throws SQLException { + super(connection, request, resultSetType, resultSetConcurrency, resultSetHoldability); + + if (tables == null) { + throw SqlExceptionUtils.clientError("Non-null table list is required"); + } + + defaultCalendar = connection.getDefaultCalendar(); + jvmZoneId = connection.getJvmTimeZone().toZoneId(); + + int size = tables.size(); + this.tables = new ArrayList<>(size); + this.tables.addAll(tables); + values = new ClickHouseExternalTable[size]; + batch = new LinkedList<>(); + } + + protected void ensureParams() throws SQLException { + List list = new ArrayList<>(); + for (int i = 0, len = values.length; i < len; i++) { + if (values[i] == null) { + list.add(tables.get(i)); + } + } + + if (!list.isEmpty()) { + throw SqlExceptionUtils.clientError(ClickHouseUtils.format("Missing table(s): %s", list)); + } + } + + protected int toArrayIndex(int parameterIndex) throws SQLException { + if (parameterIndex < 1 || parameterIndex > values.length) { + throw SqlExceptionUtils.clientError(ClickHouseUtils + .format("Parameter index must between 1 and %d but we got %d", values.length, parameterIndex)); + } + + return parameterIndex - 1; + } + + @Override + public ResultSet executeQuery() throws SQLException { + ensureParams(); + + ClickHouseSqlStatement stmt = new ClickHouseSqlStatement(getRequest().getStatements(false).get(0)); + return updateResult(stmt, executeStatement(stmt, null, Arrays.asList(values), null)); + } + + @Override + public int executeUpdate() throws SQLException { + ensureParams(); + + try (ClickHouseResponse r = executeStatement(getRequest().getStatements(false).get(0), null, + Arrays.asList(values), null)) { + return (int) r.getSummary().getWrittenRows(); + } + } + + @Override + public void setByte(int parameterIndex, byte x) throws SQLException { + throw SqlExceptionUtils.clientError(ERROR_SET_TABLE); + } + + @Override + public void setShort(int parameterIndex, short x) throws SQLException { + throw SqlExceptionUtils.clientError(ERROR_SET_TABLE); + } + + @Override + public void setInt(int parameterIndex, int x) throws SQLException { + throw SqlExceptionUtils.clientError(ERROR_SET_TABLE); + } + + @Override + public void setLong(int parameterIndex, long x) throws SQLException { + throw SqlExceptionUtils.clientError(ERROR_SET_TABLE); + } + + @Override + public void setFloat(int parameterIndex, float x) throws SQLException { + throw SqlExceptionUtils.clientError(ERROR_SET_TABLE); + } + + @Override + public void setDouble(int parameterIndex, double x) throws SQLException { + throw SqlExceptionUtils.clientError(ERROR_SET_TABLE); + } + + @Override + public void setBigDecimal(int parameterIndex, BigDecimal x) throws SQLException { + throw SqlExceptionUtils.clientError(ERROR_SET_TABLE); + } + + @Override + public void setString(int parameterIndex, String x) throws SQLException { + throw SqlExceptionUtils.clientError(ERROR_SET_TABLE); + } + + @Override + public void setBytes(int parameterIndex, byte[] x) throws SQLException { + throw SqlExceptionUtils.clientError(ERROR_SET_TABLE); + } + + @Override + public void clearParameters() throws SQLException { + ensureOpen(); + + for (int i = 0, len = values.length; i < len; i++) { + values[i] = null; + } + } + + @Override + public void setObject(int parameterIndex, Object x) throws SQLException { + ensureOpen(); + + if (x instanceof ClickHouseExternalTable) { + int idx = toArrayIndex(parameterIndex); + values[idx] = (ClickHouseExternalTable) x; + } else { + throw SqlExceptionUtils.clientError("Only ClickHouseExternalTable is allowed"); + } + } + + @Override + public boolean execute() throws SQLException { + ensureParams(); + + ClickHouseSqlStatement stmt = new ClickHouseSqlStatement(getRequest().getStatements(false).get(0)); + ClickHouseResponse r = executeStatement(stmt, null, Arrays.asList(values), null); + return updateResult(stmt, r) != null; + } + + @Override + public void addBatch(String sql) throws SQLException { + ensureOpen(); + + throw SqlExceptionUtils + .unsupportedError("addBatch(String) cannot be called in PreparedStatement or CallableStatement!"); + } + + @Override + public void addBatch() throws SQLException { + ensureOpen(); + + ensureParams(); + List list = new ArrayList<>(values.length); + for (ClickHouseExternalTable v : values) { + list.add(v); + } + batch.add(Collections.unmodifiableList(list)); + clearParameters(); + } + + @Override + public int[] executeBatch() throws SQLException { + ensureOpen(); + + int len = batch.size(); + int[] results = new int[len]; + int counter = 0; + for (List list : batch) { + try (ClickHouseResponse r = executeStatement(getRequest().getStatements(false).get(0), null, list, null)) { + results[counter] = (int) r.getSummary().getWrittenRows(); + } catch (Exception e) { + results[counter] = EXECUTE_FAILED; + log.error("Failed to execute task %d of %d", counter + 1, len, e); + } + counter++; + } + + clearBatch(); + + return results; + } + + @Override + public void clearBatch() throws SQLException { + ensureOpen(); + + this.batch.clear(); + } + + @Override + public void setArray(int parameterIndex, Array x) throws SQLException { + throw SqlExceptionUtils.clientError(ERROR_SET_TABLE); + } + + @Override + public void setDate(int parameterIndex, Date x, Calendar cal) throws SQLException { + throw SqlExceptionUtils.clientError(ERROR_SET_TABLE); + } + + @Override + public void setTime(int parameterIndex, Time x, Calendar cal) throws SQLException { + throw SqlExceptionUtils.clientError(ERROR_SET_TABLE); + } + + @Override + public void setTimestamp(int parameterIndex, Timestamp x, Calendar cal) throws SQLException { + throw SqlExceptionUtils.clientError(ERROR_SET_TABLE); + } + + @Override + public void setNull(int parameterIndex, int sqlType, String typeName) throws SQLException { + throw SqlExceptionUtils.clientError(ERROR_SET_TABLE); + } + + @Override + public ParameterMetaData getParameterMetaData() throws SQLException { + // TODO Auto-generated method stub + return null; + } + + @Override + public void setObject(int parameterIndex, Object x, int targetSqlType, int scaleOrLength) throws SQLException { + setObject(parameterIndex, x); + } +} diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/parser/ClickHouseSqlStatement.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/parser/ClickHouseSqlStatement.java index ea4292c5b..4a297947b 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/parser/ClickHouseSqlStatement.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/parser/ClickHouseSqlStatement.java @@ -3,18 +3,17 @@ import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashMap; +import java.util.LinkedHashSet; import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Objects; +import java.util.Set; import java.util.Map.Entry; public class ClickHouseSqlStatement { public static final String DEFAULT_DATABASE = "system"; public static final String DEFAULT_TABLE = "unknown"; - public static final List DEFAULT_PARAMETERS = Collections.emptyList(); - public static final Map DEFAULT_POSITIONS = Collections.emptyMap(); - public static final Map DEFAULT_SETTINGS = Collections.emptyMap(); public static final String KEYWORD_DATABASE = "DATABASE"; public static final String KEYWORD_EXISTS = "EXISTS"; @@ -34,18 +33,19 @@ public class ClickHouseSqlStatement { private final List parameters; private final Map positions; private final Map settings; + private final Set tempTables; public ClickHouseSqlStatement(String sql) { - this(sql, StatementType.UNKNOWN, null, null, null, null, null, null, null, null, null); + this(sql, StatementType.UNKNOWN, null, null, null, null, null, null, null, null, null, null); } public ClickHouseSqlStatement(String sql, StatementType stmtType) { - this(sql, stmtType, null, null, null, null, null, null, null, null, null); + this(sql, stmtType, null, null, null, null, null, null, null, null, null, null); } public ClickHouseSqlStatement(String sql, StatementType stmtType, String cluster, String database, String table, String input, String format, String outfile, List parameters, Map positions, - Map settings) { + Map settings, Set tempTables) { this.sql = sql; this.stmtType = stmtType; @@ -59,7 +59,7 @@ public ClickHouseSqlStatement(String sql, StatementType stmtType, String cluster if (parameters != null && !parameters.isEmpty()) { this.parameters = Collections.unmodifiableList(parameters); } else { - this.parameters = DEFAULT_PARAMETERS; + this.parameters = Collections.emptyList(); } if (positions != null && !positions.isEmpty()) { @@ -74,7 +74,7 @@ public ClickHouseSqlStatement(String sql, StatementType stmtType, String cluster } this.positions = Collections.unmodifiableMap(p); } else { - this.positions = DEFAULT_POSITIONS; + this.positions = Collections.emptyMap(); } if (settings != null && !settings.isEmpty()) { @@ -89,7 +89,15 @@ public ClickHouseSqlStatement(String sql, StatementType stmtType, String cluster } this.settings = Collections.unmodifiableMap(s); } else { - this.settings = DEFAULT_SETTINGS; + this.settings = Collections.emptyMap(); + } + + if (tempTables != null && !tempTables.isEmpty()) { + Set s = new LinkedHashSet<>(); + s.addAll(tempTables); + this.tempTables = Collections.unmodifiableSet(s); + } else { + this.tempTables = Collections.emptySet(); } } @@ -122,15 +130,15 @@ public boolean isIdemponent() { if (!result) { // try harder switch (this.stmtType) { - case ATTACH: - case CREATE: - case DETACH: - case DROP: - result = positions.containsKey(KEYWORD_EXISTS) || positions.containsKey(KEYWORD_REPLACE); - break; - - default: - break; + case ATTACH: + case CREATE: + case DETACH: + case DROP: + result = positions.containsKey(KEYWORD_EXISTS) || positions.containsKey(KEYWORD_REPLACE); + break; + + default: + break; } } @@ -189,6 +197,10 @@ public boolean hasFormat() { return this.format != null && !this.format.isEmpty(); } + public boolean hasInput() { + return this.input != null && !this.input.isEmpty(); + } + public boolean hasOutfile() { return this.outfile != null && !this.outfile.isEmpty(); } @@ -205,6 +217,10 @@ public boolean hasValues() { return this.positions.containsKey(KEYWORD_VALUES); } + public boolean hasTempTable() { + return !this.tempTables.isEmpty(); + } + public List getParameters() { return this.parameters; } @@ -236,6 +252,10 @@ public Map getSettings() { return this.settings; } + public Set getTempTables() { + return this.tempTables; + } + @Override public String toString() { StringBuilder sb = new StringBuilder(); @@ -243,7 +263,8 @@ public String toString() { sb.append('[').append(stmtType.name()).append(']').append(" cluster=").append(cluster).append(", database=") .append(database).append(", table=").append(table).append(", input=").append(input).append(", format=") .append(format).append(", outfile=").append(outfile).append(", parameters=").append(parameters) - .append(", positions=").append(positions).append(", settings=").append(settings).append("\nSQL:\n") + .append(", positions=").append(positions).append(", settings=").append(settings).append(", tempTables=") + .append(settings).append("\nSQL:\n") .append(sql); return sb.toString(); @@ -265,6 +286,7 @@ public int hashCode() { result = prime * result + parameters.hashCode(); result = prime * result + positions.hashCode(); result = prime * result + settings.hashCode(); + result = prime * result + tempTables.hashCode(); return result; } @@ -283,6 +305,7 @@ public boolean equals(Object obj) { && Objects.equals(database, other.database) && Objects.equals(table, other.table) && Objects.equals(input, other.input) && Objects.equals(format, other.format) && Objects.equals(outfile, other.outfile) && parameters.equals(other.parameters) - && positions.equals(other.positions) && settings.equals(other.settings); + && positions.equals(other.positions) && settings.equals(other.settings) + && tempTables.equals(other.tempTables); } } diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/parser/ParseHandler.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/parser/ParseHandler.java index 30d8b2821..4330157f7 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/parser/ParseHandler.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/parser/ParseHandler.java @@ -2,6 +2,7 @@ import java.util.List; import java.util.Map; +import java.util.Set; public abstract class ParseHandler { /** @@ -42,11 +43,12 @@ public String handleParameter(String cluster, String database, String table, int * @param parameters positions of parameters * @param positions keyword positions * @param settings settings + * @param tempTables temporary tables * @return sql statement, or null means no change */ public ClickHouseSqlStatement handleStatement(String sql, StatementType stmtType, String cluster, String database, String table, String input, String format, String outfile, List parameters, - Map positions, Map settings) { + Map positions, Map settings, Set tempTables) { return null; } } diff --git a/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/ClickHouseStatementImpl.java b/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/ClickHouseStatementImpl.java index d9498a5a5..6a22d2c81 100644 --- a/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/ClickHouseStatementImpl.java +++ b/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/ClickHouseStatementImpl.java @@ -209,7 +209,7 @@ protected ClickHouseSqlStatement applyFormat(ClickHouseSqlStatement stmt, ClickH sql = new StringBuilder(sql).append("\nFORMAT ").append(format).toString(); stmt = new ClickHouseSqlStatement(sql, stmt.getStatementType(), stmt.getCluster(), stmt.getDatabase(), - stmt.getTable(), stmt.getInput(), format, stmt.getOutfile(), stmt.getParameters(), positions, null); + stmt.getTable(), stmt.getInput(), format, stmt.getOutfile(), stmt.getParameters(), positions, null, null); } return stmt; diff --git a/clickhouse-jdbc/src/main/java9/module-info.java b/clickhouse-jdbc/src/main/java9/module-info.java index 12ffff856..22bccd70d 100644 --- a/clickhouse-jdbc/src/main/java9/module-info.java +++ b/clickhouse-jdbc/src/main/java9/module-info.java @@ -1,7 +1,9 @@ /** * Declares ru.yandex.clickhouse module. */ -module ru.yandex.clickhouse { +module com.clickhouse.jdbc { + exports com.clickhouse.jdbc; + exports ru.yandex.clickhouse; exports ru.yandex.clickhouse.domain; exports ru.yandex.clickhouse.except; diff --git a/clickhouse-jdbc/src/main/javacc/ClickHouseSqlParser.jj b/clickhouse-jdbc/src/main/javacc/ClickHouseSqlParser.jj index af3e59663..5d1da4ece 100644 --- a/clickhouse-jdbc/src/main/javacc/ClickHouseSqlParser.jj +++ b/clickhouse-jdbc/src/main/javacc/ClickHouseSqlParser.jj @@ -32,9 +32,11 @@ import java.io.StringReader; import java.util.ArrayList; import java.util.HashMap; import java.util.LinkedHashMap; +import java.util.LinkedHashSet; import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Set; import com.clickhouse.client.ClickHouseConfig; import com.clickhouse.client.logging.Logger; @@ -145,6 +147,7 @@ TOKEN_MGR_DECLS: { final List parameters = new ArrayList<>(); final Map positions = new HashMap<>(); final Map settings = new LinkedHashMap<>(); + final Set tempTables = new LinkedHashSet<>(); public void CommonTokenAction(Token t) { if (t.kind != ClickHouseSqlParserConstants.SEMICOLON) { @@ -221,6 +224,7 @@ TOKEN_MGR_DECLS: { parameters.clear(); positions.clear(); settings.clear(); + tempTables.clear(); } ClickHouseSqlStatement build(ParseHandler handler) { @@ -228,12 +232,12 @@ TOKEN_MGR_DECLS: { ClickHouseSqlStatement s = null; if (handler != null) { s = handler.handleStatement( - sqlStmt, stmtType, cluster, database, table, input, format, outfile, parameters, positions, settings); + sqlStmt, stmtType, cluster, database, table, input, format, outfile, parameters, positions, settings, tempTables); } if (s == null) { s = new ClickHouseSqlStatement( - sqlStmt, stmtType, cluster, database, table, input, format, outfile, parameters, positions, settings); + sqlStmt, stmtType, cluster, database, table, input, format, outfile, parameters, positions, settings, tempTables); } // reset variables @@ -247,22 +251,12 @@ TOKEN_MGR_DECLS: { } void addPosition(Token t) { - if (t == null) { + String keyword = null; + if (t == null || (keyword = t.image) == null || keyword.isEmpty()) { return; } - addPosition(t.image, t.beginLine == 1 ? t.beginColumn - 1 : -1); - } - - void addPosition(String keyword, int startIndex) { - if (keyword == null || keyword.isEmpty()) { - return; - } - - if (startIndex < 0) { - startIndex = builder.lastIndexOf(keyword); - } - this.positions.put(keyword.toUpperCase(Locale.ROOT), startIndex); + this.positions.put(keyword.toUpperCase(Locale.ROOT), builder.lastIndexOf(keyword)); } void addSetting(String key, String value) { @@ -279,6 +273,42 @@ SKIP: { | "\u2000" | "\u200a" | "\u200b" | "\u200c" | "\u200d" | "\u2028" | "\u2029" | "\u202f" | "\u205f" | "\u2060" | "\u3000" | "\ufeff"> { append(image); } + | { + int startIndex = image.indexOf("\'"); + int endIndex = image.lastIndexOf("\'"); + if (startIndex < 0 || endIndex < 0 || endIndex <= startIndex) { + // skip invalid content + } else if (image.charAt(1) == 'd') { // date + builder.append("date") + .append(image.substring(startIndex, endIndex + 1)); + } else { + char ch = image.charAt(2); + if (ch == 's') { // timestamp + int scale = image.lastIndexOf("."); + if (scale > 0) { + scale = endIndex - scale - 1; + } else { + scale = 0; + } + + if (scale > 0) { + // or downgrade to DateTime32 for better compatibility? + builder.append("toDateTime64(") + .append(image.substring(startIndex, endIndex + 1)).append(',').append(scale).append(')'); + } else { + builder.append("timestamp") + .append(image.substring(startIndex, endIndex + 1)); + } + } else if (ch == 't') { // temp table, either session-level temporary table or external table + String tableName = ClickHouseSqlUtils.unescape(image.substring(startIndex, endIndex + 1)); + builder.append('`').append(tableName).append('`'); + tempTables.add(tableName); + } else { + builder.append("timestamp'1970-01-01 ") + .append(image.substring(startIndex + 1, endIndex + 1)); + } + } + } | { append(image); } | "/*" { commentNestingDepth = 1; append(image); }: MULTI_LINE_COMMENT } @@ -476,7 +506,7 @@ void dataClause(): {} { columnExprList() )* | (LOOKAHEAD(2) ( { token_source.format = ClickHouseSqlUtils.unescape(token.image); } )? + { token_source.input = ClickHouseSqlUtils.unescape(token.image); } )? { token_source.format = token.image; })? (anyExprList())? } catch (ParseException e) { // FIXME introduce a lexical state in next release with consideration of delimiter from the context @@ -805,7 +835,7 @@ void interval(): {} { | | | | | | | } -Token literal(): { Token t; } { +Token literal(): { Token t = null; } { ( t = dateLiteral() | t = numberLiteral() diff --git a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseConnectionTest.java b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseConnectionTest.java new file mode 100644 index 000000000..f60ed84a1 --- /dev/null +++ b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseConnectionTest.java @@ -0,0 +1,18 @@ +package com.clickhouse.jdbc; + +import java.sql.Array; +import java.sql.SQLException; +import java.util.Properties; + +import org.testng.Assert; +import org.testng.annotations.Test; + +public class ClickHouseConnectionTest extends JdbcIntegrationTest { + @Test(groups = "integration") + public void testCreateArray() throws SQLException { + try (ClickHouseConnection conn = newConnection(new Properties())) { + Array array = conn.createArrayOf("Array(Int8)", null); + Assert.assertEquals(array.getArray(), new byte[0]); + } + } +} 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 e43e94791..916211cce 100644 --- a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHousePreparedStatementTest.java +++ b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHousePreparedStatementTest.java @@ -1,13 +1,30 @@ package com.clickhouse.jdbc; +import java.io.ByteArrayInputStream; import java.sql.PreparedStatement; +import java.sql.ResultSet; import java.sql.SQLException; +import java.sql.Statement; +import java.time.LocalDateTime; +import java.time.ZoneOffset; import java.util.Properties; +import com.clickhouse.client.ClickHouseFormat; +import com.clickhouse.client.data.ClickHouseBitmap; +import com.clickhouse.client.data.ClickHouseExternalTable; + import org.testng.Assert; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; public class ClickHousePreparedStatementTest extends JdbcIntegrationTest { + @DataProvider(name = "typedParameters") + private Object[][] getTypedParameters() { + return new Object[][] { + new Object[] { "Array(DateTime32)", new LocalDateTime[] { LocalDateTime.of(2021, 11, 1, 1, 2, 3), + LocalDateTime.of(2021, 11, 2, 2, 3, 4) } } }; + } + @Test(groups = "integration") public void testBatchInsert() throws SQLException { try (ClickHouseConnection conn = newConnection(new Properties()); @@ -31,24 +48,42 @@ public void testBatchInsert() throws SQLException { @Test(groups = "integration") public void testBatchInput() throws SQLException { try (ClickHouseConnection conn = newConnection(new Properties()); + Statement s = conn.createStatement(); PreparedStatement stmt = conn.prepareStatement( - "insert into test_batch_input select id, name from input('id Int32, name Nullable(String), desc Nullable(String)')")) { - conn.createStatement().execute("drop table if exists test_batch_input;" - + "create table test_batch_input(id Int32, name Nullable(String))engine=Memory"); - stmt.setInt(1, 1); - stmt.setString(2, "a"); - stmt.setString(3, "aaaaa"); - stmt.addBatch(); - stmt.setInt(1, 2); - stmt.setString(2, "b"); - stmt.setString(3, null); - stmt.addBatch(); - stmt.setInt(1, 3); - stmt.setString(2, null); - stmt.setString(3, "33333"); - stmt.addBatch(); - int[] results = stmt.executeBatch(); - Assert.assertEquals(results, new int[] { 0, 0, 0 }); + "insert into test_batch_input select id, name, value from input('id Int32, name Nullable(String), desc Nullable(String), value AggregateFunction(groupBitmap, UInt32)')")) { + s.execute("drop table if exists test_batch_input;" + + "create table test_batch_input(id Int32, name Nullable(String), value AggregateFunction(groupBitmap, UInt32))engine=Memory"); + Object[][] objs = new Object[][] { + new Object[] { 1, "a", "aaaaa", ClickHouseBitmap.wrap(1, 2, 3, 4, 5) }, + new Object[] { 2, "b", null, ClickHouseBitmap.wrap(6, 7, 8, 9, 10) }, + new Object[] { 3, null, "33333", ClickHouseBitmap.wrap(11, 12, 13) } + }; + for (Object[] v : objs) { + stmt.setInt(1, (int) v[0]); + stmt.setString(2, (String) v[1]); + stmt.setString(3, (String) v[2]); + stmt.setObject(4, v[3]); + stmt.addBatch(); + } + stmt.executeBatch(); + + try (ResultSet rs = s.executeQuery("select * from test_batch_input order by id")) { + Object[][] values = new Object[objs.length][]; + int index = 0; + while (rs.next()) { + values[index++] = new Object[] { + rs.getObject(1), rs.getObject(2), rs.getObject(3) + }; + } + Assert.assertEquals(index, objs.length); + for (int i = 0; i < objs.length; i++) { + Object[] actual = values[i]; + Object[] expected = objs[i]; + Assert.assertEquals(actual[0], expected[0]); + Assert.assertEquals(actual[1], expected[1]); + Assert.assertEquals(actual[2], expected[3]); + } + } } } @@ -68,4 +103,54 @@ public void testBatchQuery() throws SQLException { Assert.assertEquals(results, new int[] { 0, 0 }); } } + + @Test(groups = "integration") + public void testQueryWithExternalTable() throws SQLException { + try (ClickHouseConnection conn = newConnection(new Properties()); + PreparedStatement stmt = conn.prepareStatement( + "SELECT bitmapContains(my_bitmap, toUInt32(1)) as v1, bitmapContains(my_bitmap, toUInt32(2)) as v2 from {tt 'ext_table'}")) { + stmt.setObject(1, ClickHouseExternalTable.builder().name("ext_table") + .columns("my_bitmap AggregateFunction(groupBitmap,UInt32)").format(ClickHouseFormat.RowBinary) + .content(new ByteArrayInputStream(ClickHouseBitmap.wrap(1, 3, 5).toBytes())) + .asTempTable() + .build()); + ResultSet rs = stmt.executeQuery(); + Assert.assertTrue(rs.next()); + Assert.assertEquals(rs.getInt(1), 1); + Assert.assertEquals(rs.getInt(2), 0); + Assert.assertFalse(rs.next()); + } + } + + @Test(dataProvider = "typedParameters", groups = "integration") + public void testArrayParameter(String t, Object v) throws SQLException { + try (ClickHouseConnection conn = newConnection(new Properties()); + PreparedStatement stmt = conn.prepareStatement("select ?::?")) { + stmt.setObject(1, v); + // stmt.setString(2, t) or stmt.setObject(2, t) will result in quoted string + stmt.setObject(2, new StringBuilder(t)); + ResultSet rs = stmt.executeQuery(); + Assert.assertTrue(rs.next()); + Assert.assertEquals(rs.getObject(1), v); + Assert.assertFalse(rs.next()); + } + } + + @Test(groups = "integration") + public void testQueryWithNamedParameter() throws SQLException { + Properties props = new Properties(); + props.setProperty(JdbcConfig.PROP_NAMED_PARAM, "true"); + LocalDateTime ts = LocalDateTime.ofEpochSecond(10000, 123456789, ZoneOffset.UTC); + try (ClickHouseConnection conn = newConnection(props); + PreparedStatement stmt = conn + .prepareStatement("select :ts1 ts1, :ts2(DateTime32) ts2")) { + 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.assertFalse(rs.next()); + } + } } diff --git a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseResultSetTest.java b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseResultSetTest.java new file mode 100644 index 000000000..814e2c4aa --- /dev/null +++ b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseResultSetTest.java @@ -0,0 +1,81 @@ +package com.clickhouse.jdbc; + +import java.math.BigDecimal; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.time.LocalDateTime; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Properties; + +import org.testng.Assert; +import org.testng.annotations.Test; + +public class ClickHouseResultSetTest extends JdbcIntegrationTest { + @Test(groups = "integration") + public void testBigDecimal() throws SQLException { + try (ClickHouseConnection conn = newConnection(new Properties()); + Statement stmt = conn.createStatement()) { + ResultSet rs = stmt.executeQuery("select toDecimal64(number / 10, 1) from numbers(10)"); + BigDecimal v = BigDecimal.valueOf(0L).setScale(1); + while (rs.next()) { + Assert.assertEquals(rs.getBigDecimal(1), v); + Assert.assertEquals(rs.getObject(1), v); + v = v.add(new BigDecimal("0.1")); + } + } + } + + @Test(groups = "integration") + public void testArray() throws SQLException { + try (ClickHouseConnection conn = newConnection(new Properties()); + Statement stmt = conn.createStatement()) { + ResultSet rs = stmt.executeQuery( + "select [1,2,3] v1, ['a','b', 'c'] v2, arrayZip(v1, v2) v3, ['2021-11-01 01:02:03', '2021-11-02 02:03:04']::Array(DateTime32) as v4"); + Assert.assertTrue(rs.next()); + + Assert.assertEquals(rs.getObject(1), new short[] { 1, 2, 3 }); + Assert.assertEquals(rs.getArray(1).getArray(), new short[] { 1, 2, 3 }); + Assert.assertTrue(rs.getArray(1).getArray() == rs.getObject(1)); + + Assert.assertEquals(rs.getObject(2), new String[] { "a", "b", "c" }); + Assert.assertEquals(rs.getArray(2).getArray(), new String[] { "a", "b", "c" }); + Assert.assertTrue(rs.getArray(2).getArray() == rs.getObject(2)); + + Assert.assertEquals(rs.getObject(3), new List[] { Arrays.asList((short) 1, "a"), + Arrays.asList((short) 2, "b"), Arrays.asList((short) 3, "c") }); + Assert.assertEquals(rs.getArray(3).getArray(), new List[] { Arrays.asList((short) 1, "a"), + Arrays.asList((short) 2, "b"), Arrays.asList((short) 3, "c") }); + Assert.assertTrue(rs.getArray(3).getArray() == rs.getObject(3)); + + Assert.assertEquals(rs.getObject(4), new LocalDateTime[] { LocalDateTime.of(2021, 11, 1, 1, 2, 3), + LocalDateTime.of(2021, 11, 2, 2, 3, 4) }); + Assert.assertEquals(rs.getArray(4).getArray(), new LocalDateTime[] { LocalDateTime.of(2021, 11, 1, 1, 2, 3), + LocalDateTime.of(2021, 11, 2, 2, 3, 4) }); + Assert.assertTrue(rs.getArray(4).getArray() == rs.getObject(4)); + + Assert.assertFalse(rs.next()); + } + } + + @Test(groups = "integration") + public void testTuple() throws SQLException { + try (ClickHouseConnection conn = newConnection(new Properties()); + Statement stmt = conn.createStatement()) { + ResultSet rs = stmt + .executeQuery( + "select (1::Int16, 'a', 1.2::Float32, [1,2]::Array(Nullable(UInt8)), map(toUInt32(1),'a')) v"); + Assert.assertTrue(rs.next()); + List v = rs.getObject(1, List.class); + Assert.assertEquals(v.size(), 5); + Assert.assertEquals(v.get(0), Short.valueOf((short) 1)); + Assert.assertEquals(v.get(1), "a"); + Assert.assertEquals(v.get(2), Float.valueOf(1.2F)); + Assert.assertEquals(v.get(3), new Short[] { 1, 2 }); + Assert.assertEquals(v.get(4), Collections.singletonMap(1L, "a")); + Assert.assertFalse(rs.next()); + } + } +} 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 f88b3372b..911b28e1f 100644 --- a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseStatementTest.java +++ b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseStatementTest.java @@ -5,7 +5,12 @@ import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; +import java.sql.Time; import java.sql.Timestamp; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.OffsetDateTime; import java.util.Calendar; import java.util.GregorianCalendar; import java.util.Locale; @@ -21,6 +26,21 @@ import org.testng.annotations.Test; public class ClickHouseStatementTest extends JdbcIntegrationTest { + @Test(groups = "integration") + public void testJdbcEscapeSyntax() throws SQLException { + try (ClickHouseConnection conn = newConnection(new Properties()); + ClickHouseStatement stmt = conn.createStatement()) { + ResultSet rs = stmt.executeQuery( + "select * from (select {d '2021-11-01'} as D, {t '12:34:56'} as T, " + + "{ts '2021-11-01 12:34:56'} as TS) as {tt 'temp_table'}"); + Assert.assertTrue(rs.next()); + Assert.assertEquals(rs.getObject("ts", LocalDateTime.class), LocalDateTime.of(2021, 11, 1, 12, 34, 56)); + Assert.assertEquals(rs.getTime("t"), Time.valueOf(LocalTime.of(12, 34, 56))); + Assert.assertEquals(rs.getObject("d"), LocalDate.of(2021, 11, 1)); + Assert.assertFalse(rs.next()); + } + } + @Test(groups = "local") public void testLogComment() throws SQLException { Properties props = new Properties(); @@ -74,6 +94,22 @@ public void testQuery() throws SQLException { } } + @Test(groups = "integration") + public void testTimestamp() throws SQLException { + try (ClickHouseConnection conn = newConnection(new Properties()); + ClickHouseStatement stmt = conn.createStatement()) { + ResultSet rs = stmt.executeQuery("select now(), now('Asia/Chongqing')"); + Assert.assertTrue(rs.next()); + LocalDateTime dt1 = (LocalDateTime) rs.getObject(1); + LocalDateTime dt2 = rs.getObject(1, LocalDateTime.class); + Assert.assertTrue(dt1 == dt2); + OffsetDateTime ot1 = (OffsetDateTime) rs.getObject(2); + OffsetDateTime ot2 = rs.getObject(2, OffsetDateTime.class); + Assert.assertTrue(ot1 == ot2); + 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/CombinedResultSetTest.java b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/CombinedResultSetTest.java index 905974162..b3d76584a 100644 --- a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/CombinedResultSetTest.java +++ b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/CombinedResultSetTest.java @@ -15,7 +15,7 @@ import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -public class CombinedResultSetTest extends JdbcIntegrationTest { +public class CombinedResultSetTest { @DataProvider(name = "multipleResultSetsProvider") private Object[][] getMultipleResultSets() { return new Object[][] { @@ -129,18 +129,4 @@ public void testSingleResultSet(CombinedResultSet combined) throws SQLException combined.close(); Assert.assertTrue(combined.isClosed()); } - - @Test(groups = "integration") - public void testBigDecimal() throws SQLException { - try (ClickHouseConnection conn = newConnection(new Properties()); - Statement stmt = conn.createStatement()) { - ResultSet rs = stmt.executeQuery("select toDecimal64(number / 10, 1) from numbers(10)"); - BigDecimal v = BigDecimal.valueOf(0L).setScale(1); - while (rs.next()) { - Assert.assertEquals(rs.getBigDecimal(1), v); - Assert.assertEquals(rs.getObject(1), v); - v = v.add(new BigDecimal("0.1")); - } - } - } } 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 78488f013..0e3ffcc4b 100644 --- a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/JdbcIntegrationTest.java +++ b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/JdbcIntegrationTest.java @@ -73,19 +73,19 @@ public String getServerAddress(ClickHouseProtocol protocol, String customHostOrI .append(':').append(server.getPort()).toString(); } - public ClickHouseDataSource newDataSource() { + public ClickHouseDataSource newDataSource() throws SQLException { return newDataSource(null, new Properties()); } - public ClickHouseDataSource newDataSource(Properties properties) { + public ClickHouseDataSource newDataSource(Properties properties) throws SQLException { return newDataSource(null, properties); } - public ClickHouseDataSource newDataSource(String url) { + public ClickHouseDataSource newDataSource(String url) throws SQLException { return newDataSource(url, new Properties()); } - public ClickHouseDataSource newDataSource(String url, Properties properties) { + public ClickHouseDataSource newDataSource(String url, Properties properties) throws SQLException { return new ClickHouseDataSource(buildJdbcUrl(ClickHouseProtocol.HTTP, null, url), properties); } diff --git a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/JdbcParameterizedQueryTest.java b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/JdbcParameterizedQueryTest.java index 40e20308e..1fce97965 100644 --- a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/JdbcParameterizedQueryTest.java +++ b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/JdbcParameterizedQueryTest.java @@ -27,7 +27,7 @@ public void testParseJdbcQueries() { JdbcParameterizedQuery q = JdbcParameterizedQuery.of(sql); Assert.assertEquals(q.getOriginalQuery(), sql); Assert.assertEquals(q.hasParameter(), true); - Assert.assertEquals(q.getNamedParameters(), Arrays.asList("0", "1")); + Assert.assertEquals(q.getParameters(), Arrays.asList("0", "1")); Assert.assertEquals(q.apply("sum", "1"), "select sum(number % 2 == 0 ? 1 : 0) from numbers(100) where number > 1"); @@ -38,7 +38,14 @@ public void testParseJdbcQueries() { q = JdbcParameterizedQuery.of(sql); Assert.assertEquals(q.getOriginalQuery(), sql); Assert.assertEquals(q.hasParameter(), true); - Assert.assertEquals(q.getNamedParameters(), Arrays.asList("0")); + Assert.assertEquals(q.getParameters(), Arrays.asList("0")); Assert.assertEquals(q.apply("3"), "select 1 ? 'a' : 'b', 2 ? (select 1) : 2, 3"); + + sql = "select ?::?"; + q = JdbcParameterizedQuery.of(sql); + Assert.assertEquals(q.getOriginalQuery(), sql); + Assert.assertEquals(q.hasParameter(), true); + Assert.assertEquals(q.getParameters(), Arrays.asList("0", "1")); + Assert.assertEquals(q.apply(1, new StringBuilder("Int8")), "select 1::Int8"); } } diff --git a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/JdbcParseHandlerTest.java b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/JdbcParseHandlerTest.java index 8cad2cfdf..edd5312de 100644 --- a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/JdbcParseHandlerTest.java +++ b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/JdbcParseHandlerTest.java @@ -14,17 +14,22 @@ public class JdbcParseHandlerTest { @Test(groups = "unit") public void testParseDeleteStatement() { - Assert.assertEquals(ClickHouseSqlParser.parse("delete from tbl", null, JdbcParseHandler.INSTANCE)[0].getSQL(), + Assert.assertEquals( + ClickHouseSqlParser.parse("delete from tbl", null, JdbcParseHandler.INSTANCE)[0] + .getSQL(), "TRUNCATE TABLE tbl"); Assert.assertEquals( - ClickHouseSqlParser.parse("delete from tbl where 1", null, JdbcParseHandler.INSTANCE)[0].getSQL(), + ClickHouseSqlParser.parse("delete from tbl where 1", null, JdbcParseHandler.INSTANCE)[0] + .getSQL(), "ALTER TABLE `tbl` DELETE where 1 SETTINGS mutations_sync=1"); Assert.assertEquals( ClickHouseSqlParser.parse("delete from tbl where 1 and 1 settings a=1 format CSV", null, JdbcParseHandler.INSTANCE)[0].getSQL(), "ALTER TABLE `tbl` DELETE where 1 and 1 SETTINGS mutations_sync=1, a=1 format CSV"); Assert.assertEquals( - ClickHouseSqlParser.parse("delete from tbl where 1 and 1 settings mutations_sync=0 format CSV", null, + ClickHouseSqlParser.parse( + "delete from tbl where 1 and 1 settings mutations_sync=0 format CSV", + null, JdbcParseHandler.INSTANCE)[0].getSQL(), "ALTER TABLE `tbl` DELETE where 1 and 1 settings mutations_sync=0 format CSV"); } @@ -36,18 +41,22 @@ public void testParseInsertStatement() { @Test(groups = "unit") public void testParseUpdateStatement() { Assert.assertEquals( - ClickHouseSqlParser.parse("update tbl set a=1", null, JdbcParseHandler.INSTANCE)[0].getSQL(), + ClickHouseSqlParser.parse("update tbl set a=1", null, JdbcParseHandler.INSTANCE)[0] + .getSQL(), "ALTER TABLE `tbl` UPDATE a=1 SETTINGS mutations_sync=1"); Assert.assertEquals( - ClickHouseSqlParser.parse("update tbl set a=1,b=2 where 1", null, JdbcParseHandler.INSTANCE)[0] - .getSQL(), + ClickHouseSqlParser.parse("update tbl set a=1,b=2 where 1", null, + JdbcParseHandler.INSTANCE)[0] + .getSQL(), "ALTER TABLE `tbl` UPDATE a=1,b=2 where 1 SETTINGS mutations_sync=1"); Assert.assertEquals( - ClickHouseSqlParser.parse("update tbl set x=1, y = 2 where 1 and 1 settings a=1 format CSV", null, + ClickHouseSqlParser.parse( + "update tbl set x=1, y = 2 where 1 and 1 settings a=1 format CSV", null, JdbcParseHandler.INSTANCE)[0].getSQL(), "ALTER TABLE `tbl` UPDATE x=1, y = 2 where 1 and 1 SETTINGS mutations_sync=1, a=1 format CSV"); Assert.assertEquals( - ClickHouseSqlParser.parse("update tbl set y = 2 where 1 and 1 settings mutations_sync=0 format CSV", + ClickHouseSqlParser.parse( + "update tbl set y = 2 where 1 and 1 settings mutations_sync=0 format CSV", null, JdbcParseHandler.INSTANCE)[0].getSQL(), "ALTER TABLE `tbl` UPDATE y = 2 where 1 and 1 settings mutations_sync=0 format CSV"); } @@ -55,21 +64,27 @@ public void testParseUpdateStatement() { @Test(groups = "unit") public void testDeleteStatementWithoutWhereClause() { Assert.assertEquals(JdbcParseHandler.INSTANCE.handleStatement("delete from `a\\`' a` . tbl", - StatementType.DELETE, null, "a\\`' a", "tbl", null, null, null, null, new HashMap() { + StatementType.DELETE, null, "a\\`' a", "tbl", null, null, null, null, + new HashMap() { { put("DELETE", 0); put("FROM", 8); } - }, null), new ClickHouseSqlStatement("TRUNCATE TABLE `a\\`' a` . tbl", StatementType.DELETE, null, - "a\\`' a", "tbl", null, null, null, null, null, null)); - Assert.assertEquals(JdbcParseHandler.INSTANCE.handleStatement("delete from `table\\`'1`", StatementType.DELETE, + }, null, null), + new ClickHouseSqlStatement("TRUNCATE TABLE `a\\`' a` . tbl", StatementType.DELETE, + null, + "a\\`' a", "tbl", null, null, null, null, null, null, null)); + Assert.assertEquals(JdbcParseHandler.INSTANCE.handleStatement("delete from `table\\`'1`", + StatementType.DELETE, null, null, "table1", null, null, null, null, new HashMap() { { put("DELETE", 0); put("FROM", 7); } - }, null), new ClickHouseSqlStatement("TRUNCATE TABLE `table\\`'1`", StatementType.DELETE, null, null, - "table1", null, null, null, null, null, null), + }, null, null), + new ClickHouseSqlStatement("TRUNCATE TABLE `table\\`'1`", StatementType.DELETE, null, + null, + "table1", null, null, null, null, null, null, null), null); } @@ -81,17 +96,23 @@ public void testDeleteStatementWithWhereClause() { positions.put("WHERE", 28); Assert.assertEquals( JdbcParseHandler.INSTANCE.handleStatement("delete from `a\\`' a` . tbl where a = b", - StatementType.DELETE, null, "a\\`' a", "tbl", null, null, null, null, positions, null), - new ClickHouseSqlStatement("ALTER TABLE `a\\`' a`.`tbl` DELETE where a = b SETTINGS mutations_sync=1", - StatementType.DELETE, null, "a\\`' a", "tbl", null, null, null, null, null, null)); + StatementType.DELETE, null, "a\\`' a", "tbl", null, null, null, null, + positions, null, null), + new ClickHouseSqlStatement( + "ALTER TABLE `a\\`' a`.`tbl` DELETE where a = b SETTINGS mutations_sync=1", + StatementType.DELETE, null, "a\\`' a", "tbl", null, null, null, null, + null, null, null)); positions.put("DELETE", 0); positions.put("FROM", 8); positions.put("WHERE", 26); Assert.assertEquals( - JdbcParseHandler.INSTANCE.handleStatement("delete from `table\\`'1` where 1", StatementType.DELETE, - null, null, "table\\`'1", null, null, null, null, positions, null), - new ClickHouseSqlStatement("ALTER TABLE `table\\`'1` DELETE where 1 SETTINGS mutations_sync=1", - StatementType.DELETE, null, null, "table\\`'1", null, null, null, null, null, null)); + JdbcParseHandler.INSTANCE.handleStatement("delete from `table\\`'1` where 1", + StatementType.DELETE, + null, null, "table\\`'1", null, null, null, null, positions, null, null), + new ClickHouseSqlStatement( + "ALTER TABLE `table\\`'1` DELETE where 1 SETTINGS mutations_sync=1", + StatementType.DELETE, null, null, "table\\`'1", null, null, null, null, + null, null, null)); } @Test(groups = "unit") @@ -106,10 +127,12 @@ public void testDeleteStatementWithSettings() { }; Map settings = Collections.singletonMap("a", "1"); Assert.assertEquals( - JdbcParseHandler.INSTANCE.handleStatement(sql1, StatementType.DELETE, null, null, "tbl", null, null, - null, null, positions, settings), - new ClickHouseSqlStatement("TRUNCATE TABLE tbl settings a=1", StatementType.DELETE, null, null, "tbl", - null, null, null, null, null, settings)); + JdbcParseHandler.INSTANCE.handleStatement(sql1, StatementType.DELETE, null, null, "tbl", + null, null, + null, null, positions, settings, null), + new ClickHouseSqlStatement("TRUNCATE TABLE tbl settings a=1", StatementType.DELETE, + null, null, "tbl", + null, null, null, null, null, settings, null)); String sql2 = "delete from tbl where a != 1 and b != 2 settings a=1,b='a'"; positions = new HashMap() { @@ -127,11 +150,13 @@ public void testDeleteStatementWithSettings() { } }; Assert.assertEquals( - JdbcParseHandler.INSTANCE.handleStatement(sql2, StatementType.DELETE, null, null, "tbl", null, null, - null, null, positions, settings), + JdbcParseHandler.INSTANCE.handleStatement(sql2, StatementType.DELETE, null, null, "tbl", + null, null, + null, null, positions, settings, null), new ClickHouseSqlStatement( "ALTER TABLE `tbl` DELETE where a != 1 and b != 2 SETTINGS mutations_sync=1, a=1,b='a'", - StatementType.DELETE, null, null, "tbl", null, null, null, null, null, settings)); + StatementType.DELETE, null, null, "tbl", null, null, null, null, null, + settings, null)); String sql3 = "delete from tbl where a != 1 and b != 2 settings a=1,mutations_sync=2,b='a'"; positions = new HashMap() { @@ -150,31 +175,40 @@ public void testDeleteStatementWithSettings() { } }; Assert.assertEquals( - JdbcParseHandler.INSTANCE.handleStatement(sql3, StatementType.DELETE, null, null, "tbl", null, null, - null, null, positions, settings), + JdbcParseHandler.INSTANCE.handleStatement(sql3, StatementType.DELETE, null, null, "tbl", + null, null, + null, null, positions, settings, null), new ClickHouseSqlStatement( "ALTER TABLE `tbl` DELETE where a != 1 and b != 2 settings a=1,mutations_sync=2,b='a'", - StatementType.DELETE, null, null, "tbl", null, null, null, null, null, settings)); + StatementType.DELETE, null, null, "tbl", null, null, null, null, null, + settings, null)); } @Test(groups = "unit") public void testUpdateStatementWithoutWhereClause() { Assert.assertEquals(JdbcParseHandler.INSTANCE.handleStatement("update `a\\`' a` . tbl set a=1", - StatementType.UPDATE, null, "a\\`' a", "tbl", null, null, null, null, new HashMap() { + StatementType.UPDATE, null, "a\\`' a", "tbl", null, null, null, null, + new HashMap() { { put("UPDATE", 0); put("SET", 23); } - }, null), new ClickHouseSqlStatement("ALTER TABLE `a\\`' a`.`tbl` UPDATE a=1 SETTINGS mutations_sync=1", - StatementType.UPDATE, null, "a\\`' a", "tbl", null, null, null, null, null, null)); + }, null, null), + new ClickHouseSqlStatement( + "ALTER TABLE `a\\`' a`.`tbl` UPDATE a=1 SETTINGS mutations_sync=1", + StatementType.UPDATE, null, "a\\`' a", "tbl", null, null, null, null, + null, null, null)); Assert.assertEquals(JdbcParseHandler.INSTANCE.handleStatement("update `table\\`'1` set a=1", - StatementType.UPDATE, null, null, "table1", null, null, null, null, new HashMap() { + StatementType.UPDATE, null, null, "table1", null, null, null, null, + new HashMap() { { put("UPDATE", 0); put("SET", 20); } - }, null), new ClickHouseSqlStatement("ALTER TABLE `table1` UPDATE a=1 SETTINGS mutations_sync=1", - StatementType.UPDATE, null, null, "table1", null, null, null, null, null, null)); + }, null, null), + new ClickHouseSqlStatement("ALTER TABLE `table1` UPDATE a=1 SETTINGS mutations_sync=1", + StatementType.UPDATE, null, null, "table1", null, null, null, null, + null, null, null)); } @Test(groups = "unit") @@ -183,18 +217,24 @@ public void testUpdateStatementWithWhereClause() { positions.put("UPDATE", 0); positions.put("SET", 23); Assert.assertEquals( - JdbcParseHandler.INSTANCE.handleStatement("Update `a\\`' a` . tbl set a = 2 where a = b", - StatementType.UPDATE, null, "a\\`' a", "tbl", null, null, null, null, positions, null), + JdbcParseHandler.INSTANCE.handleStatement( + "Update `a\\`' a` . tbl set a = 2 where a = b", + StatementType.UPDATE, null, "a\\`' a", "tbl", null, null, null, null, + positions, null, null), new ClickHouseSqlStatement( "ALTER TABLE `a\\`' a`.`tbl` UPDATE a = 2 where a = b SETTINGS mutations_sync=1", - StatementType.UPDATE, null, "a\\`' a", "tbl", null, null, null, null, null, null)); + StatementType.UPDATE, null, "a\\`' a", "tbl", null, null, null, null, + null, null, null)); positions.put("UPDATE", 0); positions.put("SET", 19); Assert.assertEquals( - JdbcParseHandler.INSTANCE.handleStatement("update `table\\`'1` set a = b where 1", StatementType.UPDATE, - null, null, "table\\`'1", null, null, null, null, positions, null), - new ClickHouseSqlStatement("ALTER TABLE `table\\`'1` UPDATE a = b where 1 SETTINGS mutations_sync=1", - StatementType.UPDATE, null, null, "table\\`'1", null, null, null, null, null, null)); + JdbcParseHandler.INSTANCE.handleStatement("update `table\\`'1` set a = b where 1", + StatementType.UPDATE, + null, null, "table\\`'1", null, null, null, null, positions, null, null), + new ClickHouseSqlStatement( + "ALTER TABLE `table\\`'1` UPDATE a = b where 1 SETTINGS mutations_sync=1", + StatementType.UPDATE, null, null, "table\\`'1", null, null, null, null, + null, null, null)); } @Test(groups = "unit") @@ -209,10 +249,13 @@ public void testUpdateStatementWithSettings() { }; Map settings = Collections.singletonMap("a", "1"); Assert.assertEquals( - JdbcParseHandler.INSTANCE.handleStatement(sql1, StatementType.UPDATE, null, null, "tbl", null, null, - null, null, positions, settings), - new ClickHouseSqlStatement("ALTER TABLE `tbl` UPDATE x=1 SETTINGS mutations_sync=1, a=1", - StatementType.UPDATE, null, null, "tbl", null, null, null, null, null, settings)); + JdbcParseHandler.INSTANCE.handleStatement(sql1, StatementType.UPDATE, null, null, "tbl", + null, null, + null, null, positions, settings, null), + new ClickHouseSqlStatement( + "ALTER TABLE `tbl` UPDATE x=1 SETTINGS mutations_sync=1, a=1", + StatementType.UPDATE, null, null, "tbl", null, null, null, null, null, + settings, null)); String sql2 = "update tbl set x=1, y=2 where a != 1 and b != 2 settings a=1,b='a'"; positions = new HashMap() { @@ -230,11 +273,13 @@ public void testUpdateStatementWithSettings() { } }; Assert.assertEquals( - JdbcParseHandler.INSTANCE.handleStatement(sql2, StatementType.UPDATE, null, null, "tbl", null, null, - null, null, positions, settings), + JdbcParseHandler.INSTANCE.handleStatement(sql2, StatementType.UPDATE, null, null, "tbl", + null, null, + null, null, positions, settings, null), new ClickHouseSqlStatement( "ALTER TABLE `tbl` UPDATE x=1, y=2 where a != 1 and b != 2 SETTINGS mutations_sync=1, a=1,b='a'", - StatementType.UPDATE, null, null, "tbl", null, null, null, null, null, settings)); + StatementType.UPDATE, null, null, "tbl", null, null, null, null, null, + settings, null)); String sql3 = "update tbl set x=1,y=2 where a != 1 and b != 2 settings a=1,mutations_sync=2,b='a'"; positions = new HashMap() { @@ -253,10 +298,12 @@ public void testUpdateStatementWithSettings() { } }; Assert.assertEquals( - JdbcParseHandler.INSTANCE.handleStatement(sql3, StatementType.UPDATE, null, null, "tbl", null, null, - null, null, positions, settings), + JdbcParseHandler.INSTANCE.handleStatement(sql3, StatementType.UPDATE, null, null, "tbl", + null, null, + null, null, positions, settings, null), new ClickHouseSqlStatement( "ALTER TABLE `tbl` UPDATE x=1,y=2 where a != 1 and b != 2 settings a=1,mutations_sync=2,b='a'", - StatementType.UPDATE, null, null, "tbl", null, null, null, null, null, settings)); + StatementType.UPDATE, null, null, "tbl", null, null, null, null, null, + settings, null)); } } diff --git a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/internal/ClickHouseJdbcUrlParserTest.java b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/internal/ClickHouseJdbcUrlParserTest.java index 6a5a0dae0..a2d2217f4 100644 --- a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/internal/ClickHouseJdbcUrlParserTest.java +++ b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/internal/ClickHouseJdbcUrlParserTest.java @@ -2,6 +2,7 @@ import java.net.URI; import java.net.URISyntaxException; +import java.sql.SQLException; import java.util.Properties; import com.clickhouse.client.ClickHouseCredentials; @@ -31,23 +32,23 @@ public void testRemoveCredentialsFromQuery() { @Test(groups = "unit") public void testParseInvalidUri() { - Assert.assertThrows(IllegalArgumentException.class, () -> ClickHouseJdbcUrlParser.parse(null, null)); - Assert.assertThrows(IllegalArgumentException.class, () -> ClickHouseJdbcUrlParser.parse("", null)); - Assert.assertThrows(URISyntaxException.class, () -> ClickHouseJdbcUrlParser.parse("some_invalid_uri", null)); - Assert.assertThrows(URISyntaxException.class, () -> ClickHouseJdbcUrlParser.parse("jdbc:clickhouse:.", null)); - Assert.assertThrows(URISyntaxException.class, () -> ClickHouseJdbcUrlParser.parse("jdbc:clickhouse://", null)); - Assert.assertThrows(IllegalArgumentException.class, + Assert.assertThrows(SQLException.class, () -> ClickHouseJdbcUrlParser.parse(null, null)); + Assert.assertThrows(SQLException.class, () -> ClickHouseJdbcUrlParser.parse("", null)); + Assert.assertThrows(SQLException.class, () -> ClickHouseJdbcUrlParser.parse("some_invalid_uri", null)); + Assert.assertThrows(SQLException.class, () -> ClickHouseJdbcUrlParser.parse("jdbc:clickhouse:.", null)); + Assert.assertThrows(SQLException.class, () -> ClickHouseJdbcUrlParser.parse("jdbc:clickhouse://", null)); + Assert.assertThrows(SQLException.class, () -> ClickHouseJdbcUrlParser.parse("jdbc:clickhouse:///db", null)); - Assert.assertThrows(URISyntaxException.class, + Assert.assertThrows(SQLException.class, () -> ClickHouseJdbcUrlParser.parse("jdbc:clickhouse://server/ ", null)); - Assert.assertThrows(URISyntaxException.class, + Assert.assertThrows(SQLException.class, () -> ClickHouseJdbcUrlParser.parse("clickhouse://a:b:c@aaa", null)); - Assert.assertThrows(URISyntaxException.class, + Assert.assertThrows(SQLException.class, () -> ClickHouseJdbcUrlParser.parse("clickhouse://::1:1234/a", null)); } @Test(groups = "unit") - public void testParseIpv6() throws URISyntaxException { + public void testParseIpv6() throws SQLException, URISyntaxException { ConnectionInfo info = ClickHouseJdbcUrlParser.parse("jdbc:clickhouse://[::1]:1234", null); Assert.assertEquals(info.getUri(), new URI("jdbc:clickhouse:http://[::1]:1234/default")); Assert.assertEquals(info.getServer(), @@ -60,7 +61,7 @@ public void testParseIpv6() throws URISyntaxException { } @Test(groups = "unit") - public void testParseAbbrevation() throws URISyntaxException { + public void testParseAbbrevation() throws SQLException, URISyntaxException { ConnectionInfo info = ClickHouseJdbcUrlParser.parse("jdbc:ch://localhost", null); Assert.assertEquals(info.getUri(), new URI("jdbc:clickhouse:http://localhost:8123/default")); Assert.assertEquals(info.getServer(), @@ -89,7 +90,7 @@ public void testParseAbbrevation() throws URISyntaxException { } @Test(groups = "unit") - public void testParse() throws URISyntaxException { + public void testParse() throws SQLException, URISyntaxException { ConnectionInfo info = ClickHouseJdbcUrlParser.parse("jdbc:ch://localhost", null); Assert.assertEquals(info.getUri(), new URI("jdbc:clickhouse:http://localhost:8123/default")); Assert.assertEquals(info.getServer(), @@ -120,7 +121,7 @@ public void testParse() throws URISyntaxException { } @Test(groups = "unit") - public void testParseWithProperties() throws URISyntaxException { + public void testParseWithProperties() throws SQLException, URISyntaxException { ConnectionInfo info = ClickHouseJdbcUrlParser.parse("jdbc:clickhouse://localhost/", null); Assert.assertEquals(info.getUri(), new URI("jdbc:clickhouse:http://localhost:8123/default")); Assert.assertEquals(info.getServer(), 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 9dcbb3d72..f87f625d8 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 @@ -11,6 +11,7 @@ import java.io.InputStream; import java.io.InputStreamReader; import java.nio.charset.StandardCharsets; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.regex.Matcher; @@ -265,13 +266,14 @@ 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) }); + 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) }); + 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) }); + 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); @@ -311,13 +313,13 @@ public void testSelectStatement() { { put("TOTALS", 208); } - }, null) }); + }, 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, "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) }); + "unknown", null, null, null, null, null, null, null) }); } @Test(groups = "unit") @@ -394,12 +396,12 @@ public void testComments() throws ParseException { 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, 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", - "b;c", null, null, null, null, null, null), + "b;c", null, null, null, null, null, null, null), new ClickHouseSqlStatement("select 1 as `a ; a`", StatementType.SELECT) }); } @@ -590,6 +592,45 @@ public void testExtractDBAndTableName() { StatementType.SELECT, "db", ".inner.a"); } + @Test(groups = "unit") + public void testJdbcEscapeSyntax() { + String sql = "select {d '123'}"; + ClickHouseSqlStatement[] stmts = parse(sql); + assertEquals(stmts.length, 1); + assertEquals(stmts[0].getSQL(), "select date'123'"); + assertEquals(stmts[0].hasTempTable(), false); + + sql = "select {t '123'}"; + stmts = parse(sql); + assertEquals(stmts.length, 1); + assertEquals(stmts[0].getSQL(), "select timestamp'1970-01-01 123'"); + assertEquals(stmts[0].hasTempTable(), false); + + sql = "select {ts '123'}"; + stmts = parse(sql); + assertEquals(stmts.length, 1); + assertEquals(stmts[0].getSQL(), "select timestamp'123'"); + assertEquals(stmts[0].hasTempTable(), false); + + sql = "select {ts '123.1'}"; + stmts = parse(sql); + assertEquals(stmts.length, 1); + assertEquals(stmts[0].getSQL(), "select toDateTime64('123.1',1)"); + assertEquals(stmts[0].hasTempTable(), false); + + sql = "select {tt '1''2\\'3'}"; + stmts = parse(sql); + assertEquals(stmts.length, 1); + assertEquals(stmts[0].getSQL(), "select `1'2'3`"); + assertEquals(stmts[0].hasTempTable(), true); + assertEquals(stmts[0].getTempTables(), Collections.singleton("1'2'3")); + + sql = "select {d 1} {t} {tt} {ts 123.1'}"; + stmts = parse(sql); + assertEquals(stmts.length, 1); + assertEquals(stmts[0].getSQL(), "select "); + } + // known issue public void testTernaryOperator() { String sql = "select x > 2 ? 'a' : 'b' from (select number as x from system.numbers limit ?)"; From b59646d244e8591c3fdbc42055be071c8c15826e Mon Sep 17 00:00:00 2001 From: Zhichun Wu Date: Wed, 8 Dec 2021 20:33:15 +0800 Subject: [PATCH 4/8] Enable JDBC-specific configuration and disable wrapper object by default --- .../src/main/java/com/clickhouse/jdbc/JdbcConfig.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) 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 08b5b1c41..9466e1af0 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcConfig.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/JdbcConfig.java @@ -20,7 +20,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_WRAPPER_OBJ = BOOLEAN_TRUE; + private static final String DEFAULT_WRAPPER_OBJ = BOOLEAN_FALSE; static boolean extractBooleanValue(Properties props, String key, String defaultValue) { if (props == null || props.isEmpty() || key == null || key.isEmpty()) { @@ -45,21 +45,26 @@ public static List getDriverProperties() { DriverPropertyInfo info = new DriverPropertyInfo(PROP_AUTO_COMMIT, DEFAULT_AUTO_COMMIT); info.choices = new String[] { BOOLEAN_TRUE, BOOLEAN_FALSE }; info.description = "Whether to enable auto commit when connection is created."; + list.add(info); info = new DriverPropertyInfo(PROP_FETCH_SIZE, DEFAULT_FETCH_SIZE); info.description = "Default fetch size, negative or zero means no preferred option."; + list.add(info); info = new DriverPropertyInfo(PROP_JDBC_COMPLIANT, DEFAULT_JDBC_COMPLIANT); info.choices = new String[] { BOOLEAN_TRUE, BOOLEAN_FALSE }; info.description = "Whether to enable JDBC-compliant features like fake transaction and standard UPDATE and DELETE statements."; + list.add(info); info = new DriverPropertyInfo(PROP_NAMED_PARAM, DEFAULT_NAMED_PARAM); info.choices = new String[] { BOOLEAN_TRUE, BOOLEAN_FALSE }; 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_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."; + list.add(info); return Collections.unmodifiableList(list); } From 45acb4e18b74c10590e733bf9eaf6d7d0bfe7b29 Mon Sep 17 00:00:00 2001 From: Zhichun Wu Date: Wed, 8 Dec 2021 20:44:30 +0800 Subject: [PATCH 5/8] Stick with JDK 8 --- .pom.xml.swp | Bin 0 -> 16384 bytes .../jdbc/internal/ClickHouseStatementImpl.java | 4 ++-- 2 files changed, 2 insertions(+), 2 deletions(-) create mode 100644 .pom.xml.swp diff --git a/.pom.xml.swp b/.pom.xml.swp new file mode 100644 index 0000000000000000000000000000000000000000..4e088c4fea1ae54eb42298013800065e3750e66c GIT binary patch literal 16384 zcmeHNON<;x8E%J|M{FkqkRU+{?X|@vneLg{HQB7nZ9D6AY~x)UuWe!!9yQ%HGrc?0 z-SlJE>%`%J0w)d#1X6MV2>}Pdk%Ev9z$FL};=my&JPsiuJPt%iNXRBQhOertpEI*N z>nuc;x}{IEUH_y0`s=T%|L?6<{oJv6by{mGc;2Zf(}%uHANgrZ@%L;~E_Q8mHR6Qz zpO1mB8#zI3o|+H4?sDyv?`l2Qq`jitld(d5(T#kQ1^Od)*o|7c*|W{%t{Vlcwq&(T z-E;feg?>r4sHu9ZJWzRH^B&lxOg0;JF1Kf5xBBjPo!z{ON}}>W<$=lrl?N&hR34~2 zPzfF#S%HM6J-~Y(xBeK0!qTi78^RoT968&{aKPTIF$_|o# zu1orYq_;}+e@ptjq(57tzb5IYCH?gh{a=#)DM>#m2b}bOtKCEze{53n7$9r))?@Bv>3 zJ_$5{`+z%vUBLAnit<4zXN^(d;@q4m;(+24d8Cz=WkP#?*q>PUjf#DPXIfBSKq2AzX5&( zSik~M2Xx>*;2+x+9~~ zd@NOVZ_d{f5g`EKa8qISxS;XJ5K z5`BP>$#lce!}S*CgsIe&wohw7Y6vNl>jbWoN;KhyX`&a}a;&~*rmPKZs;*(woqS`) z+onMng?2CHpN28;?Ld1bVmZMQU7@MOB;u?|^m21fzz+P#3thh;f=;XfLMCL=cAFtG zb24>q*g$(GLW6$Fjj1Mjz9GlvIBrNoERmo_Jv$Xa)NA`N4Ri!u$y8u|mzWu9KJ#2Z zM679U6narO=a{aASL5uNBQ;FWhJ1r6Tan$fv_Ra48l`S1&!8$Z@_im>YmE9vv(95b zF{nVwBz?AGi@l?+O=`{B;=-X*i$_nM!J0`3f|o#z$1rdf7_nExT!y~gLM*w?JauS? z_0hMH>4px^4Y2tAwX9XeLn6Ot@YoM#bUmmyG&cy~0EdQa9@AXEqX*1iu}!9n3uJl_ z(hhvATg>aZYkHrqFsG>#ovTT^6N+uifG)pFbo`Z)$S^~&n>f@9x^8&-rp@HLCB~c$ z6#hxo2AeQ&lqm!YW~h_J*swP2p`OqCw<9jtX{OuPXZa<&qx_QHz>2U*pb;eM^JvAc)u&*(7w8y|}JP1HDVD0<)is21Ju_bneh$oRs)Wu&8*5 z#a394hxx4QI9TlbNQLAKJa{nk;>>6-YT&}U7@{WY1R?VVbmycqm7AXJ+$@W$>N0cr=u_lVHWp zzq-1rQIDG4c+XFI)K46nom*I(t2KFQIk?A*ah~Zi^^lNN7u|NaN`0oDh)sS#Js0Q0 zs!&jL3n`2tdCP{4!-qQdMUgCImKP4JY%z#UwPYd@{mr7Ac&B|pJ?5BNes)N)4>4oJ zc1-5+9p`*}bT{^4$_i~UU*bM~a%OwdIy-yLJWO+7`nKrCIF5u)g_$_MRO(pj@xu?# zs+Mn~mQdTS&q;Cwd`HXVVl%`bGe^{TdYFtc=D^%6i&NXcRc@pq>1#|*N|wEkCyB-- zq9=_p3q`uo#<_IN%I`mMPdQr5-0|JW%dHlbqw78&dQ z36Zi+9557K|9=;C=gX)^dHsJx7Mwps?H>ROzz2aFsP&%*o(4V*d>GgRyoUPz1>hMV z1Qvkz0I#C1e;#-mcmm*c|NDU#P{&^eJ`eC3e>ZRqwfnDtuK_l25V#k(jynD&;2GdN zFa_)cUPR6QEnp2e0Q?FG?t91p5%^cab_mYsKvp7FE7~{Kf1T%3`gFS9J=udEF&;?$`2jj zdXe}h#_5<&%nyk>%_Uo8ricwHW4 zNa8)5PFS1bb;AJT6Upm>2EKQ0#T@bgLD8KYPa7|`H)!`mTVwWkXTw0~NpIg`@wbWH z6Q5gLw`X?oZ?AO@&2(4Lm#pQDr(DEHQn2#COa4|8MWcSnSo+CLn1MLjLMzBPUK@AM x%`K-5<{-zp@hrr7+Xfnwi4e(~k&bFBbG~fpjkY=CS;yYNlqt10_@5mR{{?K?{1gBH literal 0 HcmV?d00001 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 d71911d27..28b1ea9c0 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 @@ -119,7 +119,7 @@ protected ClickHouseResponse executeStatement(String stmt, request.options(options); } if (settings != null && !settings.isEmpty()) { - if (request.getSessionId().isEmpty()) { + if (!request.getSessionId().isPresent()) { request.session(UUID.randomUUID().toString()); } for (Entry e : settings.entrySet()) { @@ -130,7 +130,7 @@ protected ClickHouseResponse executeStatement(String stmt, List list = new ArrayList<>(tables.size()); for (ClickHouseExternalTable t : tables) { if (t.isTempTable()) { - if (request.getSessionId().isEmpty()) { + if (!request.getSessionId().isPresent()) { request.session(UUID.randomUUID().toString()); } request.query("drop temporary table if exists `" + t.getName() + "`").execute().get(); From e9a3bc7e63e0db86888ffc50333579de78bceff3 Mon Sep 17 00:00:00 2001 From: Zhichun Wu Date: Wed, 8 Dec 2021 21:15:49 +0800 Subject: [PATCH 6/8] Improve UT to support CH 21.3 --- .../jdbc/ClickHousePreparedStatementTest.java | 10 +++++++++- .../com/clickhouse/jdbc/ClickHouseResultSetTest.java | 4 ++-- 2 files changed, 11 insertions(+), 3 deletions(-) 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 916211cce..e4d0eed06 100644 --- a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHousePreparedStatementTest.java +++ b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHousePreparedStatementTest.java @@ -65,7 +65,11 @@ public void testBatchInput() throws SQLException { stmt.setObject(4, v[3]); stmt.addBatch(); } - stmt.executeBatch(); + int[] results = stmt.executeBatch(); + Assert.assertEquals(results.length, objs.length); + for (int result : results) { + Assert.assertNotEquals(result, PreparedStatement.EXECUTE_FAILED); + } try (ResultSet rs = s.executeQuery("select * from test_batch_input order by id")) { Object[][] values = new Object[objs.length][]; @@ -126,6 +130,10 @@ public void testQueryWithExternalTable() throws SQLException { public void testArrayParameter(String t, Object v) throws SQLException { try (ClickHouseConnection conn = newConnection(new Properties()); PreparedStatement stmt = conn.prepareStatement("select ?::?")) { + if (conn.getServerVersion().check("(,21.3]")) { + return; + } + stmt.setObject(1, v); // stmt.setString(2, t) or stmt.setObject(2, t) will result in quoted string stmt.setObject(2, new StringBuilder(t)); diff --git a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseResultSetTest.java b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseResultSetTest.java index 814e2c4aa..24adbafca 100644 --- a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseResultSetTest.java +++ b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseResultSetTest.java @@ -33,7 +33,7 @@ public void testArray() throws SQLException { try (ClickHouseConnection conn = newConnection(new Properties()); Statement stmt = conn.createStatement()) { ResultSet rs = stmt.executeQuery( - "select [1,2,3] v1, ['a','b', 'c'] v2, arrayZip(v1, v2) v3, ['2021-11-01 01:02:03', '2021-11-02 02:03:04']::Array(DateTime32) as v4"); + "select [1,2,3] v1, ['a','b', 'c'] v2, arrayZip(v1, v2) v3, cast(['2021-11-01 01:02:03', '2021-11-02 02:03:04'] as Array(DateTime32)) v4"); Assert.assertTrue(rs.next()); Assert.assertEquals(rs.getObject(1), new short[] { 1, 2, 3 }); @@ -66,7 +66,7 @@ public void testTuple() throws SQLException { Statement stmt = conn.createStatement()) { ResultSet rs = stmt .executeQuery( - "select (1::Int16, 'a', 1.2::Float32, [1,2]::Array(Nullable(UInt8)), map(toUInt32(1),'a')) v"); + "select (toInt16(1), 'a', toFloat32(1.2), cast([1,2] as Array(Nullable(UInt8))), map(toUInt32(1),'a')) v"); Assert.assertTrue(rs.next()); List v = rs.getObject(1, List.class); Assert.assertEquals(v.size(), 5); From e7b0e6a7eedde7cba23e4b763de42a5e94ed3afe Mon Sep 17 00:00:00 2001 From: Zhichun Wu Date: Thu, 9 Dec 2021 00:13:40 +0800 Subject: [PATCH 7/8] Revert changes made on streaming --- .../client/ClickHouseInputStream.java | 20 ++++++------------- .../client/data/ClickHouseLZ4InputStream.java | 4 ++-- .../http/ClickHouseResponseHandler.java | 1 - .../client/http/DefaultHttpConnection.java | 7 ++++--- 4 files changed, 12 insertions(+), 20 deletions(-) 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 b956f040e..76c83d976 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseInputStream.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseInputStream.java @@ -40,7 +40,7 @@ private void ensureOpen() throws IOException { throw new IOException("Stream has been closed"); } - if (buffer == null || (buffer != EMPTY && buffer.limit() > 0 && !buffer.hasRemaining())) { + if (buffer == null || (buffer != EMPTY && !buffer.hasRemaining())) { updateBuffer(); } } @@ -65,17 +65,9 @@ private int updateBuffer() throws IOException { @Override public int available() throws IOException { - if (closed || buffer == EMPTY) { - return 0; - } + ensureOpen(); - int available = 0; - if (buffer == null || (buffer.limit() > 0 && !buffer.hasRemaining())) { - available = updateBuffer(); - } else { - available = buffer.remaining(); - } - return available; + return buffer.remaining(); } @Override @@ -95,7 +87,7 @@ public void close() throws IOException { public byte readByte() throws IOException { ensureOpen(); - if (buffer == EMPTY || buffer.limit() == 0) { + if (buffer == EMPTY) { close(); throw new EOFException(); } @@ -107,7 +99,7 @@ public byte readByte() throws IOException { public int read() throws IOException { ensureOpen(); - if (buffer == EMPTY || buffer.limit() == 0) { + if (buffer == EMPTY) { return -1; } @@ -120,7 +112,7 @@ public int read(byte[] b, int off, int len) throws IOException { int counter = 0; while (len > 0) { - if (buffer == EMPTY || buffer.limit() == 0) { + if (buffer == EMPTY) { return counter > 0 ? counter : -1; } 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 cf835187d..d563f8096 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 @@ -28,10 +28,10 @@ public class ClickHouseLZ4InputStream extends ClickHouseInputStream { private boolean closed; private boolean checkNext() throws IOException { - if (currentBlock == null) { + if (currentBlock == null || !currentBlock.hasRemaining()) { currentBlock = readNextBlock(); } - return currentBlock != null && currentBlock.hasRemaining(); + return currentBlock != null; } // every block is: 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 e9cc17a78..a04b3539a 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 @@ -9,7 +9,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; import java.util.concurrent.LinkedBlockingDeque; -import java.util.concurrent.TimeUnit; import java.util.concurrent.Flow.Subscription; import java.util.concurrent.atomic.AtomicBoolean; 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 bd96b11b8..099593c5f 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 @@ -183,14 +183,15 @@ private ClickHouseHttpResponse postString(HttpRequest.Builder reqBuilder, String reqBuilder.POST(HttpRequest.BodyPublishers.ofString(sql)); HttpResponse r; try { - // r = httpClient.send(reqBuilder.build(), responseInfo -> new - // ExtendedResponseInputStream()); - r = httpClient.send(reqBuilder.build(), + CompletableFuture> f = httpClient.sendAsync(reqBuilder.build(), responseInfo -> new ClickHouseResponseHandler(config.getMaxBufferSize(), config.getSocketTimeout())); + r = f.get(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new IOException("Thread was interrupted when posting request or receiving response", e); + } catch (ExecutionException e) { + throw new IOException("Failed to post query", e); } return buildResponse(r); } From d2fa508486edcfc57f42136060b616783884492b Mon Sep 17 00:00:00 2001 From: Zhichun Wu Date: Thu, 9 Dec 2021 00:13:52 +0800 Subject: [PATCH 8/8] Validate results in benchmark --- .pom.xml.swp | Bin 16384 -> 0 bytes .../benchmark/jdbc/ConsumeValueFunction.java | 2 +- .../benchmark/jdbc/DriverState.java | 6 +- .../com/clickhouse/benchmark/jdbc/Query.java | 98 +++++++++++++----- 4 files changed, 78 insertions(+), 28 deletions(-) delete mode 100644 .pom.xml.swp diff --git a/.pom.xml.swp b/.pom.xml.swp deleted file mode 100644 index 4e088c4fea1ae54eb42298013800065e3750e66c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 16384 zcmeHNON<;x8E%J|M{FkqkRU+{?X|@vneLg{HQB7nZ9D6AY~x)UuWe!!9yQ%HGrc?0 z-SlJE>%`%J0w)d#1X6MV2>}Pdk%Ev9z$FL};=my&JPsiuJPt%iNXRBQhOertpEI*N z>nuc;x}{IEUH_y0`s=T%|L?6<{oJv6by{mGc;2Zf(}%uHANgrZ@%L;~E_Q8mHR6Qz zpO1mB8#zI3o|+H4?sDyv?`l2Qq`jitld(d5(T#kQ1^Od)*o|7c*|W{%t{Vlcwq&(T z-E;feg?>r4sHu9ZJWzRH^B&lxOg0;JF1Kf5xBBjPo!z{ON}}>W<$=lrl?N&hR34~2 zPzfF#S%HM6J-~Y(xBeK0!qTi78^RoT968&{aKPTIF$_|o# zu1orYq_;}+e@ptjq(57tzb5IYCH?gh{a=#)DM>#m2b}bOtKCEze{53n7$9r))?@Bv>3 zJ_$5{`+z%vUBLAnit<4zXN^(d;@q4m;(+24d8Cz=WkP#?*q>PUjf#DPXIfBSKq2AzX5&( zSik~M2Xx>*;2+x+9~~ zd@NOVZ_d{f5g`EKa8qISxS;XJ5K z5`BP>$#lce!}S*CgsIe&wohw7Y6vNl>jbWoN;KhyX`&a}a;&~*rmPKZs;*(woqS`) z+onMng?2CHpN28;?Ld1bVmZMQU7@MOB;u?|^m21fzz+P#3thh;f=;XfLMCL=cAFtG zb24>q*g$(GLW6$Fjj1Mjz9GlvIBrNoERmo_Jv$Xa)NA`N4Ri!u$y8u|mzWu9KJ#2Z zM679U6narO=a{aASL5uNBQ;FWhJ1r6Tan$fv_Ra48l`S1&!8$Z@_im>YmE9vv(95b zF{nVwBz?AGi@l?+O=`{B;=-X*i$_nM!J0`3f|o#z$1rdf7_nExT!y~gLM*w?JauS? z_0hMH>4px^4Y2tAwX9XeLn6Ot@YoM#bUmmyG&cy~0EdQa9@AXEqX*1iu}!9n3uJl_ z(hhvATg>aZYkHrqFsG>#ovTT^6N+uifG)pFbo`Z)$S^~&n>f@9x^8&-rp@HLCB~c$ z6#hxo2AeQ&lqm!YW~h_J*swP2p`OqCw<9jtX{OuPXZa<&qx_QHz>2U*pb;eM^JvAc)u&*(7w8y|}JP1HDVD0<)is21Ju_bneh$oRs)Wu&8*5 z#a394hxx4QI9TlbNQLAKJa{nk;>>6-YT&}U7@{WY1R?VVbmycqm7AXJ+$@W$>N0cr=u_lVHWp zzq-1rQIDG4c+XFI)K46nom*I(t2KFQIk?A*ah~Zi^^lNN7u|NaN`0oDh)sS#Js0Q0 zs!&jL3n`2tdCP{4!-qQdMUgCImKP4JY%z#UwPYd@{mr7Ac&B|pJ?5BNes)N)4>4oJ zc1-5+9p`*}bT{^4$_i~UU*bM~a%OwdIy-yLJWO+7`nKrCIF5u)g_$_MRO(pj@xu?# zs+Mn~mQdTS&q;Cwd`HXVVl%`bGe^{TdYFtc=D^%6i&NXcRc@pq>1#|*N|wEkCyB-- zq9=_p3q`uo#<_IN%I`mMPdQr5-0|JW%dHlbqw78&dQ z36Zi+9557K|9=;C=gX)^dHsJx7Mwps?H>ROzz2aFsP&%*o(4V*d>GgRyoUPz1>hMV z1Qvkz0I#C1e;#-mcmm*c|NDU#P{&^eJ`eC3e>ZRqwfnDtuK_l25V#k(jynD&;2GdN zFa_)cUPR6QEnp2e0Q?FG?t91p5%^cab_mYsKvp7FE7~{Kf1T%3`gFS9J=udEF&;?$`2jj zdXe}h#_5<&%nyk>%_Uo8ricwHW4 zNa8)5PFS1bb;AJT6Upm>2EKQ0#T@bgLD8KYPa7|`H)!`mTVwWkXTw0~NpIg`@wbWH z6Q5gLw`X?oZ?AO@&2(4Lm#pQDr(DEHQn2#COa4|8MWcSnSo+CLn1MLjLMzBPUK@AM x%`K-5<{-zp@hrr7+Xfnwi4e(~k&bFBbG~fpjkY=CS;yYNlqt10_@5mR{{?K?{1gBH diff --git a/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/jdbc/ConsumeValueFunction.java b/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/jdbc/ConsumeValueFunction.java index e5b3f5a36..5a390d360 100644 --- a/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/jdbc/ConsumeValueFunction.java +++ b/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/jdbc/ConsumeValueFunction.java @@ -7,5 +7,5 @@ @FunctionalInterface public interface ConsumeValueFunction { - void consume(Blackhole blackhole, ResultSet rs, int columnIndex) throws SQLException; + void consume(Blackhole blackhole, ResultSet rs, int rowIndex, int columnIndex) throws SQLException; } diff --git a/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/jdbc/DriverState.java b/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/jdbc/DriverState.java index f3d38b3df..618f382f3 100644 --- a/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/jdbc/DriverState.java +++ b/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/jdbc/DriverState.java @@ -129,11 +129,11 @@ public boolean usePreparedStatement() { public ConsumeValueFunction getConsumeFunction(ConsumeValueFunction defaultFunc) { if ("string".equals(type)) { - return (b, r, i) -> b.consume(r.getString(i)); + return (b, r, l, i) -> b.consume(r.getString(i)); } else if ("object".equals(type)) { - return (b, r, i) -> b.consume(r.getObject(i)); + return (b, r, l, i) -> b.consume(r.getObject(i)); } else if (defaultFunc == null) { - return (b, r, i) -> b.consume(i); + return (b, r, l, i) -> b.consume(i); } else { return defaultFunc; } diff --git a/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/jdbc/Query.java b/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/jdbc/Query.java index 09d2554a7..ef5f49eb2 100644 --- a/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/jdbc/Query.java +++ b/clickhouse-benchmark/src/main/java/com/clickhouse/benchmark/jdbc/Query.java @@ -2,6 +2,8 @@ import java.sql.ResultSet; import java.sql.Statement; +import java.util.Locale; + import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.infra.Blackhole; @@ -10,162 +12,210 @@ public class Query extends DriverBenchmark { public void selectArrayOfUInt16(Blackhole blackhole, DriverState state) throws Throwable { int num = state.getRandomNumber(); int rows = state.getSampleSize() + num; - ConsumeValueFunction func = state.getConsumeFunction((b, r, i) -> b.consume(r.getArray(i))); + ConsumeValueFunction func = state.getConsumeFunction((b, r, l, i) -> b.consume(r.getArray(i))); + int l = 0; try (Statement stmt = executeQuery(state, "select range(100, number % 600) as v from numbers(?)", rows)) { ResultSet rs = stmt.getResultSet(); while (rs.next()) { - func.consume(blackhole, rs, 1); + func.consume(blackhole, rs, l++, 1); } } + if (l != rows) { + throw new IllegalStateException(String.format(Locale.ROOT, "Expected %d rows but got %d", rows, l)); + } } @Benchmark public void selectMapOfInt32(Blackhole blackhole, DriverState state) throws Throwable { int num = state.getRandomNumber(); int rows = state.getSampleSize() + num; - ConsumeValueFunction func = state.getConsumeFunction((b, r, i) -> b.consume(r.getObject(i))); + ConsumeValueFunction func = state.getConsumeFunction((b, r, l, i) -> b.consume(r.getObject(i))); + int l = 0; try (Statement stmt = executeQuery(state, "select cast((arrayMap(x->x+1000, range(1, number % 100)), arrayMap(x->x+10000, range(1, number %100))) as Map(Int32, Int32)) as v from numbers(?)", rows)) { ResultSet rs = stmt.getResultSet(); while (rs.next()) { - func.consume(blackhole, rs, 1); + func.consume(blackhole, rs, l++, 1); } } + if (l != rows) { + throw new IllegalStateException(String.format(Locale.ROOT, "Expected %d rows but got %d", rows, l)); + } } @Benchmark public void selectTupleOfInt16(Blackhole blackhole, DriverState state) throws Throwable { int num = state.getRandomNumber(); int rows = state.getSampleSize() + num; - ConsumeValueFunction func = state.getConsumeFunction((b, r, i) -> b.consume(r.getObject(i))); + ConsumeValueFunction func = state.getConsumeFunction((b, r, l, i) -> b.consume(r.getObject(i))); + int l = 0; try (Statement stmt = executeQuery(state, "select tuple(arrayMap(x -> cast(x as Int16), range(100, number % 600))) as v from numbers(?)", rows)) { ResultSet rs = stmt.getResultSet(); while (rs.next()) { - func.consume(blackhole, rs, 1); + func.consume(blackhole, rs, l++, 1); } } + if (l != rows) { + throw new IllegalStateException(String.format(Locale.ROOT, "Expected %d rows but got %d", rows, l)); + } } @Benchmark public void selectDateTime32(Blackhole blackhole, DriverState state) throws Throwable { int num = state.getRandomNumber(); int rows = state.getSampleSize() + num; - ConsumeValueFunction func = state.getConsumeFunction((b, r, i) -> b.consume(r.getTimestamp(i))); + ConsumeValueFunction func = state.getConsumeFunction((b, r, l, i) -> b.consume(r.getTimestamp(i))); + int l = 0; try (Statement stmt = executeQuery(state, "select toDateTime32(1613826920 + number) as v from numbers(?)", rows)) { ResultSet rs = stmt.getResultSet(); while (rs.next()) { - func.consume(blackhole, rs, 1); + func.consume(blackhole, rs, l++, 1); } } + if (l != rows) { + throw new IllegalStateException(String.format(Locale.ROOT, "Expected %d rows but got %d", rows, l)); + } } @Benchmark public void selectDateTime64(Blackhole blackhole, DriverState state) throws Throwable { int num = state.getRandomNumber(); int rows = state.getSampleSize() + num; - ConsumeValueFunction func = state.getConsumeFunction((b, r, i) -> b.consume(r.getTimestamp(i))); + ConsumeValueFunction func = state.getConsumeFunction((b, r, l, i) -> b.consume(r.getTimestamp(i))); + int l = 0; try (Statement stmt = executeQuery(state, "select toDateTime64(1613826920 + number / 1000000000, 9) as v from numbers(?)", rows)) { ResultSet rs = stmt.getResultSet(); while (rs.next()) { - func.consume(blackhole, rs, 1); + func.consume(blackhole, rs, l++, 1); } } + if (l != rows) { + throw new IllegalStateException(String.format(Locale.ROOT, "Expected %d rows but got %d", rows, l)); + } } @Benchmark public void selectInt8(Blackhole blackhole, DriverState state) throws Throwable { int num = state.getRandomNumber(); int rows = state.getSampleSize() + num; - ConsumeValueFunction func = state.getConsumeFunction((b, r, i) -> b.consume(r.getByte(i))); + ConsumeValueFunction func = state.getConsumeFunction((b, r, l, i) -> b.consume(r.getByte(i))); + int l = 0; try (Statement stmt = executeQuery(state, "select toInt8(number % 256) as v from numbers(?)", rows)) { ResultSet rs = stmt.getResultSet(); while (rs.next()) { - func.consume(blackhole, rs, 1); + func.consume(blackhole, rs, l++, 1); } } + if (l != rows) { + throw new IllegalStateException(String.format(Locale.ROOT, "Expected %d rows but got %d", rows, l)); + } } @Benchmark public void selectUInt8(Blackhole blackhole, DriverState state) throws Throwable { int num = state.getRandomNumber(); int rows = state.getSampleSize() + num; - ConsumeValueFunction func = state.getConsumeFunction((b, r, i) -> b.consume(r.getShort(i))); + ConsumeValueFunction func = state.getConsumeFunction((b, r, l, i) -> b.consume(r.getShort(i))); + int l = 0; try (Statement stmt = executeQuery(state, "select toUInt8(number % 256) as v from numbers(?)", rows)) { ResultSet rs = stmt.getResultSet(); while (rs.next()) { - func.consume(blackhole, rs, 1); + func.consume(blackhole, rs, l++, 1); } } + if (l != rows) { + throw new IllegalStateException(String.format(Locale.ROOT, "Expected %d rows but got %d", rows, l)); + } } @Benchmark public void selectUuid(Blackhole blackhole, DriverState state) throws Throwable { int num = state.getRandomNumber(); int rows = state.getSampleSize() + num; - ConsumeValueFunction func = state.getConsumeFunction((b, r, i) -> b.consume(r.getString(i))); + ConsumeValueFunction func = state.getConsumeFunction((b, r, l, i) -> b.consume(r.getString(i))); + int l = 0; try (Statement stmt = executeQuery(state, "select generateUUIDv4() as v from numbers(?)", rows)) { ResultSet rs = stmt.getResultSet(); while (rs.next()) { - func.consume(blackhole, rs, 1); + func.consume(blackhole, rs, l++, 1); } } + if (l != rows) { + throw new IllegalStateException(String.format(Locale.ROOT, "Expected %d rows but got %d", rows, l)); + } } @Benchmark public void selectInt32(Blackhole blackhole, DriverState state) throws Throwable { int num = state.getRandomNumber(); int rows = state.getSampleSize() + num; - ConsumeValueFunction func = state.getConsumeFunction((b, r, i) -> b.consume(r.getInt(i))); + ConsumeValueFunction func = state.getConsumeFunction((b, r, l, i) -> b.consume(r.getInt(i))); + int l = 0; try (Statement stmt = executeQuery(state, "select toInt32(number) as v from numbers(?)", rows)) { ResultSet rs = stmt.getResultSet(); while (rs.next()) { - func.consume(blackhole, rs, 1); + func.consume(blackhole, rs, l++, 1); } } + if (l != rows) { + throw new IllegalStateException(String.format(Locale.ROOT, "Expected %d rows but got %d", rows, l)); + } } @Benchmark public void selectString(Blackhole blackhole, DriverState state) throws Throwable { int num = state.getRandomNumber(); int rows = state.getSampleSize() + num; - ConsumeValueFunction func = state.getConsumeFunction((b, r, i) -> b.consume(r.getString(i))); + ConsumeValueFunction func = state.getConsumeFunction((b, r, l, i) -> b.consume(r.getString(i))); + int l = 0; try (Statement stmt = executeQuery(state, "select toString(number/3) as v from numbers(?)", rows)) { ResultSet rs = stmt.getResultSet(); while (rs.next()) { - func.consume(blackhole, rs, 1); + func.consume(blackhole, rs, l++, 1); } } + if (l != rows) { + throw new IllegalStateException(String.format(Locale.ROOT, "Expected %d rows but got %d", rows, l)); + } } @Benchmark public void selectUInt64(Blackhole blackhole, DriverState state) throws Throwable { int num = state.getRandomNumber(); int rows = state.getSampleSize() + num; - ConsumeValueFunction func = state.getConsumeFunction((b, r, i) -> b.consume(r.getLong(i))); + ConsumeValueFunction func = state.getConsumeFunction((b, r, l, i) -> b.consume(r.getLong(i))); + int l = 0; try (Statement stmt = executeQuery(state, "select number as v from numbers(?)", rows)) { ResultSet rs = stmt.getResultSet(); while (rs.next()) { - func.consume(blackhole, rs, 1); + func.consume(blackhole, rs, l++, 1); } } + if (l != rows) { + throw new IllegalStateException(String.format(Locale.ROOT, "Expected %d rows but got %d", rows, l)); + } } @Benchmark public void selectDecimal64(Blackhole blackhole, DriverState state) throws Throwable { int num = state.getRandomNumber(); int rows = state.getSampleSize() + num; - ConsumeValueFunction func = state.getConsumeFunction((b, r, i) -> b.consume(r.getBigDecimal(i))); + ConsumeValueFunction func = state.getConsumeFunction((b, r, l, i) -> b.consume(r.getBigDecimal(i))); + int l = 0; try (Statement stmt = executeQuery(state, "select toDecimal64(number + number / 10000, 4) as v from numbers(?)", rows)) { ResultSet rs = stmt.getResultSet(); while (rs.next()) { - func.consume(blackhole, rs, 1); + func.consume(blackhole, rs, l++, 1); } } + if (l != rows) { + throw new IllegalStateException(String.format(Locale.ROOT, "Expected %d rows but got %d", rows, l)); + } } }