diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml
new file mode 100644
index 000000000..027ec6e4e
--- /dev/null
+++ b/.github/workflows/nightly.yml
@@ -0,0 +1,65 @@
+name: Nightly
+
+on:
+ schedule:
+ # Build and publish nightly snapshot at 3:55pm every day
+ - cron: "55 15 * * *"
+
+env:
+ CHC_BRANCH: "develop"
+ CHC_VERSION: "0.3.3"
+
+jobs:
+ nightly:
+ if: ${{ startsWith(github.repository, 'ClickHouse/') }}
+ name: "Build and Publish Nightly Snapshot"
+ runs-on: "ubuntu-latest"
+
+ steps:
+ - name: Check out Git repository
+ uses: actions/checkout@v3
+ with:
+ ref: ${{ env.CHC_BRANCH }}
+ - name: Install JDK 11
+ uses: AdoptOpenJDK/install-jdk@v1
+ with:
+ version: "11"
+ targets: "JDK11_HOME"
+ - name: Setup Toolchain
+ shell: bash
+ run: |
+ mkdir -p $HOME/.m2 \
+ && cat << EOF > $HOME/.m2/toolchains.xml
+
+
+
+ jdk
+
+ 11
+
+
+ ${{ env.JDK11_HOME }}
+
+
+
+ EOF
+ - name: Install JDK 8 and Maven
+ uses: actions/setup-java@v2
+ with:
+ distribution: "temurin"
+ java-version: 8
+ - name: Update Configuration
+ run: |
+ find . -type f -name "pom.xml" -exec sed -i -e 's|${revision}|${{ env.CHC_VERSION }}-SNAPSHOT|g' \
+ -e 's|^\( \).*\(\)$|\1${{ env.CHC_VERSION }}-SNAPSHOT\2|' \
+ -e 's|${parent.groupId}|com.clickhouse|g' -e 's|${project.parent.groupId}|com.clickhouse|g' '{}' \;
+ find . -type f -name "simplelogger.*" -exec rm -fv '{}' \;
+ - name: Release Snapshot
+ uses: samuelmeuli/action-maven-publish@v1
+ with:
+ maven_profiles: release
+ maven_args: -q --batch-mode
+ gpg_private_key: ${{ secrets.GPG_PRIVATE_KEY }}
+ gpg_passphrase: ${{ secrets.GPG_PASSPHRASE }}
+ nexus_username: ${{ secrets.SONATYPE_USER }}
+ nexus_password: ${{ secrets.SONATYPE_PASSWD }}
diff --git a/.gitignore b/.gitignore
index 3109e7d25..e0c1be403 100644
--- a/.gitignore
+++ b/.gitignore
@@ -8,6 +8,8 @@
*.ear
# VSCode
+.bloop
+.metals
.vscode
.factorypath
@@ -33,6 +35,7 @@ target/
# Generated files
.flattened-pom.xml
dependency-reduced-pom.xml
+**/parser/javacc/*
**/parser/*CharStream.java
**/parser/ClickHouseSqlParser.java
**/parser/ClickHouseSqlParserConstants.java
diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseColumn.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseColumn.java
index 4c88ffb3a..29320c2b9 100644
--- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseColumn.java
+++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseColumn.java
@@ -202,8 +202,10 @@ protected static int readColumn(String args, int startIndex, int len, String nam
List nestedColumns = new LinkedList<>();
for (String p : params) {
if (isFirst) {
- int pIndex = p.indexOf('(');
- aggFunc = ClickHouseAggregateFunction.of(pIndex > 0 ? p.substring(0, pIndex) : p);
+ if (matchedKeyword == KEYWORD_AGGREGATE_FUNCTION) {
+ int pIndex = p.indexOf('(');
+ aggFunc = ClickHouseAggregateFunction.of(pIndex > 0 ? p.substring(0, pIndex) : p);
+ }
isFirst = false;
} else {
nestedColumns.add(ClickHouseColumn.of("", p));
diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseNode.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseNode.java
index 3eb2559bc..21cfe6779 100644
--- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseNode.java
+++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseNode.java
@@ -274,6 +274,20 @@ public Builder addOption(String option, String value) {
return this;
}
+ /**
+ * Removes an option from this node.
+ *
+ * @param option option to be removed, null value will be ignored
+ * @return this builder
+ */
+ public Builder removeOption(String option) {
+ if (!ClickHouseChecker.isNullOrEmpty(option)) {
+ options.remove(option);
+ }
+
+ return this;
+ }
+
/**
* Sets all options for this node. Use null or empty value to clear all existing
* options.
diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseResponseSummary.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseResponseSummary.java
index 33bc639b6..19790e61a 100644
--- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseResponseSummary.java
+++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseResponseSummary.java
@@ -257,4 +257,12 @@ public int getUpdateCount() {
public boolean isEmpty() {
return progress.get().isEmpty() && stats.get().isEmpty();
}
+
+ @Override
+ public String toString() {
+ return new StringBuilder().append("ClickHouseResponseSummary [readBytes=").append(getReadBytes())
+ .append(", readRows=").append(getReadRows()).append(", totalRowsToRead=").append(getTotalRowsToRead())
+ .append(", writtenBytes=").append(getWrittenBytes()).append(", writtenRows=").append(getWrittenRows())
+ .append(", updates=").append(getUpdateCount()).append(']').toString();
+ }
}
diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseLongValue.java b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseLongValue.java
index 755536235..3e4fd938a 100644
--- a/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseLongValue.java
+++ b/clickhouse-client/src/main/java/com/clickhouse/client/data/ClickHouseLongValue.java
@@ -246,12 +246,12 @@ public ClickHouseLongValue update(double value) {
@Override
public ClickHouseLongValue update(BigInteger value) {
- return value == null ? resetToNullOrEmpty() : set(false, unsigned, value.longValueExact());
+ return value == null ? resetToNullOrEmpty() : set(false, unsigned, value.longValue());
}
@Override
public ClickHouseLongValue update(BigDecimal value) {
- return value == null ? resetToNullOrEmpty() : set(false, unsigned, value.longValueExact());
+ return value == null ? resetToNullOrEmpty() : set(false, unsigned, value.longValue());
}
@Override
diff --git a/clickhouse-client/src/test/java/com/clickhouse/client/data/ClickHouseLongValueTest.java b/clickhouse-client/src/test/java/com/clickhouse/client/data/ClickHouseLongValueTest.java
index 7be6e3f7e..442a88bf5 100644
--- a/clickhouse-client/src/test/java/com/clickhouse/client/data/ClickHouseLongValueTest.java
+++ b/clickhouse-client/src/test/java/com/clickhouse/client/data/ClickHouseLongValueTest.java
@@ -321,5 +321,11 @@ public void testUnsignedValue() throws Exception {
Assert.assertEquals(v.asBigInteger(), new BigInteger("10223372036854775101"));
Assert.assertEquals(v.asBigDecimal(), new BigDecimal("10223372036854775101"));
Assert.assertEquals(v.asString(), "10223372036854775101");
+
+ v.update(new BigDecimal("10223372036854775101"));
+ Assert.assertEquals(v.asLong(), -8223372036854776515L);
+ Assert.assertEquals(v.asBigInteger(), new BigInteger("10223372036854775101"));
+ Assert.assertEquals(v.asBigDecimal(), new BigDecimal("10223372036854775101"));
+ Assert.assertEquals(v.asString(), "10223372036854775101");
}
}
diff --git a/clickhouse-jdbc/pom.xml b/clickhouse-jdbc/pom.xml
index edccc6bc5..83deec78e 100644
--- a/clickhouse-jdbc/pom.xml
+++ b/clickhouse-jdbc/pom.xml
@@ -228,6 +228,10 @@
org.apache${shade.base}.apache
+
+ net.jpountz
+ ${shade.base}.jpountz
+
@@ -277,6 +281,20 @@
truetrueall
+
+
+ com.google.gson
+ ${shade.base}.gson
+
+
+ org.apache
+ ${shade.base}.apache
+
+
+ net.jpountz
+ ${shade.base}.jpountz
+
+
@@ -293,9 +311,7 @@
*:*
- com/google/**mozilla/**
- org/****/module-info.classMETA-INF/DEPENDENCIESMETA-INF/MANIFEST.MF
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 d1b47522b..effc90e3c 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
@@ -11,6 +11,8 @@
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
import com.clickhouse.client.ClickHouseChecker;
import com.clickhouse.client.ClickHouseClient;
@@ -18,6 +20,7 @@
import com.clickhouse.client.ClickHouseDataStreamFactory;
import com.clickhouse.client.ClickHouseDeserializer;
import com.clickhouse.client.ClickHouseFormat;
+import com.clickhouse.client.ClickHouseNode;
import com.clickhouse.client.ClickHouseRequest;
import com.clickhouse.client.ClickHouseResponse;
import com.clickhouse.client.ClickHouseResponseSummary;
@@ -427,14 +430,27 @@ public void cancel() throws SQLException {
final String qid;
if ((qid = this.queryId) != null) {
- ClickHouseClient.send(request.getServer(), String.format("KILL QUERY WHERE query_id='%s'", qid))
- .whenComplete((summary, exception) -> {
- if (exception != null) {
- log.warn("Failed to kill query [%s] due to: %s", qid, exception.getMessage());
- } else if (summary != null) {
- log.debug("Killed query [%s]", qid);
- }
- });
+ String sessionIdKey = ClickHouseClientOption.SESSION_ID.getKey();
+ ClickHouseNode server = request.getServer();
+ if (server.getOptions().containsKey(sessionIdKey)) {
+ server = ClickHouseNode.builder(request.getServer()).removeOption(sessionIdKey)
+ .removeOption(ClickHouseClientOption.SESSION_CHECK.getKey())
+ .removeOption(ClickHouseClientOption.SESSION_TIMEOUT.getKey()).build();
+ }
+ try {
+ List summaries = ClickHouseClient
+ .send(server, String.format("KILL QUERY WHERE query_id='%s'", qid))
+ .get(request.getConfig().getConnectionTimeout(), TimeUnit.MILLISECONDS);
+ log.info("Killed query [%s]: %s", qid, summaries.get(0));
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ log.warn("Interrupted for killing query [%s]", qid);
+ } catch (TimeoutException e) {
+ log.warn("Timed out after waiting %d ms for killing query [%s]",
+ request.getConfig().getConnectionTimeout(), qid);
+ } catch (Exception e) { // unexpected
+ throw SqlExceptionUtils.handle(e.getCause());
+ }
}
if (request.getTransaction() != null) {
request.getTransaction().abort();
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 681bdecaf..9349cf9dc 100644
--- a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseStatementTest.java
+++ b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseStatementTest.java
@@ -52,6 +52,15 @@ private Object[][] getTimeZoneTestOptions() {
new Object[] { true }, new Object[] { false } };
}
+ @DataProvider(name = "connectionProperties")
+ private Object[][] getConnectionProperties() {
+ Properties emptyProps = new Properties();
+ Properties sessionProps = new Properties();
+ sessionProps.setProperty(ClickHouseClientOption.SESSION_ID.getKey(), UUID.randomUUID().toString());
+ return new Object[][] {
+ new Object[] { emptyProps }, new Object[] { sessionProps } };
+ }
+
@Test(groups = "integration")
public void testJdbcEscapeSyntax() throws SQLException {
try (ClickHouseConnection conn = newConnection(new Properties());
@@ -237,13 +246,15 @@ public void testAsyncInsert() throws SQLException {
}
}
- @Test(groups = "integration")
- public void testCancelQuery() throws Exception {
- try (ClickHouseConnection conn = newConnection(new Properties());
+ @Test(dataProvider = "connectionProperties", groups = "integration")
+ public void testCancelQuery(Properties props) throws Exception {
+ try (ClickHouseConnection conn = newConnection(props);
ClickHouseStatement stmt = conn.createStatement();) {
CountDownLatch c = new CountDownLatch(1);
ClickHouseClient.submit(() -> stmt.executeQuery("select * from numbers(100000000)")).whenComplete(
(rs, e) -> {
+ Assert.assertNull(e, "Should NOT have any exception");
+
int index = 0;
try {
@@ -252,8 +263,9 @@ public void testCancelQuery() throws Exception {
c.countDown();
}
}
+ Assert.fail("Query should have been cancelled");
} catch (SQLException ex) {
- // ignore
+ Assert.assertNotNull(ex, "Should end up with exception");
}
});
try {
@@ -261,6 +273,12 @@ public void testCancelQuery() throws Exception {
} finally {
stmt.cancel();
}
+
+ try (ResultSet rs = stmt.executeQuery("select 5")) {
+ Assert.assertTrue(rs.next(), "Should have at least one record");
+ Assert.assertEquals(rs.getInt(1), 5);
+ Assert.assertFalse(rs.next(), "Should have only one record");
+ }
}
}
diff --git a/examples/jdbc/pom.xml b/examples/jdbc/pom.xml
index b7fe4ad9e..5db22317f 100644
--- a/examples/jdbc/pom.xml
+++ b/examples/jdbc/pom.xml
@@ -7,7 +7,7 @@
1.0.0jar
- jdbc-examples
+ JDBC ExamplesJDBC Exampleshttps://github.com/ClickHouse/clickhouse-jdbc2022
@@ -56,7 +56,7 @@
UTF-8UTF-8
- 0.3.2-patch9
+ 0.3.3-SNAPSHOT3.8.1
diff --git a/examples/jdbc/src/main/java/com/clickhouse/examples/jdbc/Advance.java b/examples/jdbc/src/main/java/com/clickhouse/examples/jdbc/Advance.java
deleted file mode 100644
index b092aae03..000000000
--- a/examples/jdbc/src/main/java/com/clickhouse/examples/jdbc/Advance.java
+++ /dev/null
@@ -1,79 +0,0 @@
-package com.clickhouse.examples.jdbc;
-
-import java.io.ByteArrayInputStream;
-import java.nio.charset.StandardCharsets;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.time.LocalDateTime;
-import java.util.Properties;
-
-import com.clickhouse.client.ClickHouseFormat;
-import com.clickhouse.client.data.ClickHouseExternalTable;
-
-public class Advance {
- static String exteralTables(String url, String user, String password) throws SQLException {
- String sql = "select a.name as n1, b.name as n2 from {tt 'table1'} a inner join {tt 'table2'} b on a.id=b.id";
- try (Connection conn = DriverManager.getConnection(url, user, password);
- PreparedStatement ps = conn.prepareStatement(sql)) {
- ps.setObject(1,
- ClickHouseExternalTable.builder().name("table1").columns("id Int32, name Nullable(String)")
- .format(ClickHouseFormat.CSV)
- .content(new ByteArrayInputStream("1,a\n2,b".getBytes(StandardCharsets.US_ASCII))).build());
- ps.setObject(2,
- ClickHouseExternalTable.builder().name("table2").columns("id Int32, name String")
- .format(ClickHouseFormat.JSONEachRow)
- .content(new ByteArrayInputStream("{\"id\":3,\"name\":\"c\"}\n{\"id\":1,\"name\":\"d\"}"
- .getBytes(StandardCharsets.US_ASCII)))
- .build());
- try (ResultSet rs = ps.executeQuery()) {
- if (!rs.next()) {
- throw new IllegalStateException("Should have at least one record");
- }
-
- // n1=a, n2=d
- return String.format("n1=%s, n2=%s", rs.getString(1), rs.getString(2));
- }
- }
- }
-
- static String namedParameter(String url, String user, String password) throws SQLException {
- Properties props = new Properties();
- props.setProperty("user", user);
- props.setProperty("password", password);
- props.setProperty("namedParameter", "true");
- // two parameters:
- // * a - String
- // * b - DateTime64(3)
- String sql = "select :a as a1, :a(String) as a2, :b(DateTime64(3)) as b";
- try (Connection conn = DriverManager.getConnection(url, props);
- PreparedStatement ps = conn.prepareStatement(sql)) {
- ps.setString(1, "a");
- ps.setObject(2, LocalDateTime.of(2022, 1, 7, 22, 48, 17, 123000000));
-
- try (ResultSet rs = ps.executeQuery()) {
- if (!rs.next()) {
- throw new IllegalStateException("Should have at least one record");
- }
- // a1=a, a2=a, b=2022-01-07 22:48:17.123
- return String.format("a1=%s, a2=%s, b=%s", rs.getString(1), rs.getString(2), rs.getString("B"));
- }
- }
- }
-
- public static void main(String[] args) {
- String url = String.format("jdbc:ch://%s:%d/system", System.getProperty("chHost", "localhost"),
- Integer.parseInt(System.getProperty("chPort", "8123")));
- String user = System.getProperty("chUser", "default");
- String password = System.getProperty("chPassword", "");
-
- try {
- exteralTables(url, user, password);
- namedParameter(url, user, password);
- } catch (SQLException e) {
- e.printStackTrace();
- }
- }
-}
diff --git a/examples/jdbc/src/main/java/com/clickhouse/examples/jdbc/Advanced.java b/examples/jdbc/src/main/java/com/clickhouse/examples/jdbc/Advanced.java
new file mode 100644
index 000000000..ef9d74fd1
--- /dev/null
+++ b/examples/jdbc/src/main/java/com/clickhouse/examples/jdbc/Advanced.java
@@ -0,0 +1,192 @@
+package com.clickhouse.examples.jdbc;
+
+import java.io.ByteArrayInputStream;
+import java.io.UncheckedIOException;
+import java.nio.charset.StandardCharsets;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.time.LocalDateTime;
+import java.util.Properties;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.clickhouse.client.ClickHouseException;
+import com.clickhouse.client.ClickHouseFormat;
+import com.clickhouse.client.ClickHouseRecord;
+import com.clickhouse.client.ClickHouseRequest;
+import com.clickhouse.client.ClickHouseRequestManager;
+import com.clickhouse.client.ClickHouseResponse;
+import com.clickhouse.client.data.ClickHouseExternalTable;
+import com.clickhouse.jdbc.ClickHouseConnection;
+import com.clickhouse.jdbc.SqlExceptionUtils;
+
+public class Advanced {
+ private static Connection getConnection(String url, Properties props) throws SQLException {
+ Connection conn = DriverManager.getConnection(url, props);
+ System.out.println("Connected to: " + conn.getMetaData().getURL());
+ return conn;
+ }
+
+ private static Connection getConnection(String url) throws SQLException {
+ return getConnection(url, new Properties());
+ }
+
+ private static ByteArrayInputStream newInputStream(String content) {
+ return new ByteArrayInputStream(content.getBytes(StandardCharsets.US_ASCII));
+ }
+
+ static String customQueryId(String url) throws SQLException {
+ String sql = "select 1";
+ String queryId = "my-query-id";
+ String result = "";
+ try (Connection conn = getConnection(url); Statement stmt = conn.createStatement()) {
+ stmt.unwrap(ClickHouseRequest.class).manager(new ClickHouseRequestManager() {
+ private final AtomicInteger id = new AtomicInteger(0);
+
+ @Override
+ public String createQueryId() {
+ return "my-query-" + id.incrementAndGet();
+ }
+ });
+ try (ClickHouseResponse resp = stmt.unwrap(ClickHouseRequest.class).query(sql, queryId).executeAndWait()) {
+ result = resp.firstRecord().getValue(1).asString();
+ } catch (ClickHouseException | UncheckedIOException e) {
+ throw SqlExceptionUtils.handle(e);
+ }
+ }
+ return result;
+ }
+
+ static String exteralTables(String url) throws SQLException {
+ String sql = "select a.name as n1, b.name as n2 from {tt 'table1'} a inner join {tt 'table2'} b on a.id=b.id";
+ try (Connection conn = getConnection(url); PreparedStatement ps = conn.prepareStatement(sql)) {
+ ps.setObject(1,
+ ClickHouseExternalTable.builder().name("table1").columns("id Int32, name Nullable(String)")
+ .format(ClickHouseFormat.CSV)
+ .content(newInputStream("1,a\n2,b")).build());
+ ps.setObject(2,
+ ClickHouseExternalTable.builder().name("table2").columns("id Int32, name String")
+ .format(ClickHouseFormat.JSONEachRow)
+ .content(newInputStream("{\"id\":3,\"name\":\"c\"}\n{\"id\":1,\"name\":\"d\"}")).build());
+ try (ResultSet rs = ps.executeQuery()) {
+ if (!rs.next()) {
+ throw new IllegalStateException("Should have at least one record");
+ }
+
+ // n1=a, n2=d
+ return String.format("n1=%s, n2=%s", rs.getString(1), rs.getString(2));
+ }
+ }
+ }
+
+ static String manualTransaction(String url) throws SQLException {
+ Properties props = new Properties();
+ // props.setProperty(JdbcConfig.PROP_AUTO_COMMIT, "false");
+ props.setProperty("autoCommit", "false");
+ // props.setProperty(JdbcConfig.PROP_TX_SUPPORT, "true");
+ props.setProperty("transactionSupport", "true");
+ try (Connection conn = getConnection(url, props)) {
+ if (!((ClickHouseConnection) conn).isTransactionSupported()) {
+ System.out.println("Re-establishing connection until transaction is supported...");
+ return manualTransaction(url);
+ }
+
+ conn.commit();
+ return "Transaction committed!";
+ }
+ }
+
+ static String namedParameter(String url) throws SQLException {
+ Properties props = new Properties();
+ // props.setProperty(JdbcConfig.PROP_NAMED_PARAM, "true");
+ props.setProperty("namedParameter", "true");
+ // two parameters:
+ // * a - String
+ // * b - DateTime64(3)
+ String sql = "select :a as a1, :a(String) as a2, :b(DateTime64(3)) as b";
+ try (Connection conn = getConnection(url, props); PreparedStatement ps = conn.prepareStatement(sql)) {
+ ps.setString(1, "a");
+ ps.setObject(2, LocalDateTime.of(2022, 1, 7, 22, 48, 17, 123000000));
+
+ try (ResultSet rs = ps.executeQuery()) {
+ if (!rs.next()) {
+ throw new IllegalStateException("Should have at least one record");
+ }
+
+ // a1=a, a2=a, b=2022-01-07 22:48:17.123
+ return String.format("a1=%s, a2=%s, b=%s", rs.getString(1), rs.getString(2), rs.getString("B"));
+ }
+ }
+ }
+
+ static String renameResponseColumn(String url) throws SQLException {
+ Properties props = new Properties();
+ // props.setProperty(ClickHouseClientOption.RENAME_RESPONSE_COLUMN.getKey(),
+ // ClickHouseRenameMethod.TO_CAMELCASE_WITHOUT_PREFIX.name());
+ props.setProperty("rename_response_column", "TO_CAMELCASE_WITHOUT_PREFIX");
+
+ String sql = "SELECT c.`simple.string_value`, n.number\n"
+ + "FROM (SELECT 1 number, 'string' `simple.string_value`) c\n"
+ + "INNER JOIN (SELECT 1 number) n ON n.number = c.number";
+ try (Connection conn = getConnection(url, props);
+ Statement stmt = conn.createStatement();
+ ResultSet rs = stmt.executeQuery(sql)) {
+ String firstColumn = rs.getMetaData().getColumnName(1);
+ String secondColumn = rs.getMetaData().getColumnName(2);
+
+ if (!rs.next()) {
+ throw new IllegalStateException("Should have at least one record");
+ }
+
+ // return "stringValue=string, number=1"
+ // instead of "simple.string_value=string, n.number=1",
+ return String.format("%s=%s, %s=%s", firstColumn, rs.getString(1), secondColumn, rs.getInt(2));
+ }
+ }
+
+ static String unwrapToUseClientApi(String url) throws SQLException {
+ String sql = "select 1 n union all select 2 n";
+ try (Connection conn = getConnection(url); Statement stmt = conn.createStatement()) {
+ ClickHouseRequest> request = stmt.unwrap(ClickHouseRequest.class);
+ // server setting is not allowed in read-only mode
+ if (!conn.isReadOnly()) {
+ // not required for ClickHouse 22.7+, only works for HTTP protocol
+ request.set("send_progress_in_http_headers", 1);
+ }
+ try (ClickHouseResponse response = request.query(sql).executeAndWait()) {
+ int count = 0;
+ // may throw UncheckedIOException (due to restriction of Iterable interface)
+ for (ClickHouseRecord r : response.records()) {
+ count++;
+ }
+ return String.format("Result Rows: %d (read bytes: %d)", count, response.getSummary().getReadBytes());
+ } catch (ClickHouseException e) {
+ throw SqlExceptionUtils.handle(e);
+ }
+ }
+ }
+
+ public static void main(String[] args) {
+ // randomly pick one of the two endpoints to connect to,
+ // fail over to the other when there's connection issue
+ String url = System.getProperty("chUrl",
+ "jdbc:ch://(https://explorer@play.clickhouse.com:443),"
+ + "(https://demo:demo@github.demo.trial.altinity.cloud)"
+ + "/default?failover=1&load_balancing_policy=random");
+
+ try {
+ System.out.println(exteralTables(url));
+ System.out.println(namedParameter(url));
+ System.out.println(renameResponseColumn(url));
+ System.out.println(unwrapToUseClientApi(url));
+
+ // requires ClickHouse 22.6+ with transaction enabled
+ System.out.println(manualTransaction(url));
+ } catch (SQLException e) {
+ e.printStackTrace();
+ }
+ }
+}
diff --git a/examples/jdbc/src/main/java/com/clickhouse/examples/jdbc/Basic.java b/examples/jdbc/src/main/java/com/clickhouse/examples/jdbc/Basic.java
index c25383c80..a0cc834b9 100644
--- a/examples/jdbc/src/main/java/com/clickhouse/examples/jdbc/Basic.java
+++ b/examples/jdbc/src/main/java/com/clickhouse/examples/jdbc/Basic.java
@@ -6,83 +6,147 @@
import java.sql.ResultSet;
import java.sql.SQLException;
import java.sql.Statement;
+import java.time.LocalDateTime;
+import java.util.Properties;
public class Basic {
- static void dropAndCreateTable(String url, String user, String password, String table) throws SQLException {
- try (Connection conn = DriverManager.getConnection(url, user, password);
- Statement stmt = conn.createStatement()) {
- stmt.execute(
- String.format(
- "drop table if exists %1$s; create table %1$s(a String, b Nullable(String)) engine=Memory",
- table));
+ static final String TABLE_NAME = "jdbc_example_basic";
+
+ private static Connection getConnection(String url) throws SQLException {
+ return getConnection(url, new Properties());
+ }
+
+ private static Connection getConnection(String url, Properties properties) throws SQLException {
+ final Connection conn;
+ // Driver driver = new ClickHouseDriver();
+ // conn = driver.connect(url, properties);
+
+ // ClickHouseDataSource dataSource = new ClickHouseDataSource(url, properties);
+ // conn = dataSource.getConnection();
+
+ conn = DriverManager.getConnection(url, properties);
+ System.out.println("Connected to: " + conn.getMetaData().getURL());
+ return conn;
+ }
+
+ static int dropAndCreateTable(Connection conn) throws SQLException {
+ try (Statement stmt = conn.createStatement()) {
+ // multi-statement query is supported by default
+ // session will be created automatically during execution
+ stmt.execute(String.format(
+ "drop table if exists %1$s; create table %1$s(a String, b Nullable(String)) engine=Memory",
+ TABLE_NAME));
+ return stmt.getUpdateCount();
}
}
- static void batchInsert(String url, String user, String password, String table) throws SQLException {
- try (Connection conn = DriverManager.getConnection(url, user, password)) {
- // not that fast as it's based on string substitution and large sql statement
- String sql = String.format("insert into %1$s values(?, ?)", table);
- try (PreparedStatement ps = conn.prepareStatement(sql)) {
- ps.setString(1, "a");
- ps.setString(2, "b");
- ps.addBatch();
- ps.setString(1, "c");
- ps.setString(2, null);
- ps.addBatch();
- ps.executeBatch();
+ static int batchInsert(Connection conn) throws SQLException {
+ // 1. NOT recommended when inserting lots of rows, because it's based on a large statement
+ String sql = String.format("insert into %s values(? || ' - 1', ?)", TABLE_NAME);
+ int count = 0;
+ try (PreparedStatement ps = conn.prepareStatement(sql)) {
+ ps.setString(1, "a");
+ ps.setString(2, "b");
+ ps.addBatch();
+ ps.setString(1, "c");
+ // ps.setNull(2, Types.VARCHAR);
+ // ps.setObject(2, null);
+ ps.setString(2, null);
+ ps.addBatch();
+ // same as below query:
+ // insert into
values ('a' || ' - 1', 'b'), ('c' || ' - 1', null)
+ for (int i : ps.executeBatch()) {
+ if (i > 0) {
+ count += i;
+ }
}
+ }
- // faster when inserting massive data
- sql = String.format("insert into %1$s", table);
- try (PreparedStatement ps = conn.prepareStatement(sql)) {
- ps.setString(1, "a");
- ps.setString(2, "b");
- ps.addBatch();
- ps.setString(1, "c");
- ps.setString(2, null);
- ps.addBatch();
- ps.executeBatch();
+ // 2. faster and ease of use, with additional query for getting table structure
+ // sql = String.format("insert into %s (a)", TABLE_NAME);
+ // sql = String.format("insert into %s (a) values (?)", TABLE_NAME);
+ sql = String.format("insert into %s (* except b)", TABLE_NAME);
+ // Note: below query will be issued to get table structure:
+ // select * except b from
where 0
+ try (PreparedStatement ps = conn.prepareStatement(sql)) {
+ // implicit type conversion: int -> String
+ ps.setInt(1, 1);
+ ps.addBatch();
+ // implicit type conversion: LocalDateTime -> string
+ ps.setObject(1, LocalDateTime.now());
+ ps.addBatch();
+ // same as below query:
+ // insert into
format RowBinary
+ for (int i : ps.executeBatch()) {
+ if (i > 0) {
+ count += i;
+ }
}
+ }
- // fastest approach as it does not need to issue additional query for metadata
- sql = String.format("insert into %1$s select a, b from input('a String, b Nullable(String)')", table);
- try (PreparedStatement ps = conn.prepareStatement(sql)) {
- ps.setString(1, "a");
- ps.setString(2, "b");
- ps.addBatch();
- ps.setString(1, "c");
- ps.setString(2, null);
- ps.addBatch();
- ps.executeBatch();
+ // 3. fastest but inconvenient and NOT portable(as it's limited to ClickHouse)
+ // see https://clickhouse.com/docs/en/sql-reference/table-functions/input/
+ sql = String.format("insert into %s select a, b from input('a String, b Nullable(String)')", TABLE_NAME);
+ try (PreparedStatement ps = conn.prepareStatement(sql)) {
+ ps.setString(1, "a");
+ ps.setString(2, "b");
+ ps.addBatch();
+ ps.setString(1, "c");
+ ps.setString(2, null);
+ ps.addBatch();
+ // same as below query:
+ // insert into