diff --git a/CHANGELOG.md b/CHANGELOG.md index 40587e234..160c2b7c3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,6 +1,7 @@ ## 0.4.2 ### New Features +* support `BEGIN TRANSACTION`, `COMMIT`, and `ROLLBACK` in JDBC driver. [#975](https://github.com/ClickHouse/clickhouse-java/issues/975) * new options for JDBC driver * databaseTerm(catalog or schema, defaults to schema) [#1273](https://github.com/ClickHouse/clickhouse-java/issues/1273) * externalDatabase(true or false, defaults to true) [#1245](https://github.com/ClickHouse/clickhouse-java/issues/1245) diff --git a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseTransaction.java b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseTransaction.java index 1abd2cce9..159453257 100644 --- a/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseTransaction.java +++ b/clickhouse-client/src/main/java/com/clickhouse/client/ClickHouseTransaction.java @@ -119,6 +119,10 @@ public String toString() { static final String QUERY_SELECT_TX_ID = "SELECT transactionID()"; + public static final String COMMAND_BEGIN = "BEGIN"; + public static final String COMMAND_COMMIT = "COMMIT"; + public static final String COMMAND_ROLLBACK = "ROLLBACK"; + // transaction state public static final int NEW = 0; public static final int ACTIVE = 1; @@ -526,7 +530,7 @@ public void commit(Map settings) throws ClickHouseExceptio boolean success = false; try { ensureTransactionId(); - issue("COMMIT", true, settings); + issue(COMMAND_COMMIT, true, settings); success = state.compareAndSet(currentState, COMMITTED); return x; } catch (ClickHouseException e) { @@ -582,7 +586,7 @@ public void rollback(Map settings) throws ClickHouseExcept boolean success = false; try { ensureTransactionId(); - issue("ROLLBACK", true, settings); + issue(COMMAND_ROLLBACK, true, settings); success = state.compareAndSet(currentState, ROLLED_BACK); return x; } catch (ClickHouseException e) { 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 722c617c3..70fe0110d 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseConnection.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/ClickHouseConnection.java @@ -151,6 +151,14 @@ default PreparedStatement prepareStatement(String sql, int resultSetType, int re return prepareStatement(sql, resultSetType, resultSetConcurrency, ResultSet.HOLD_CURSORS_OVER_COMMIT); } + /** + * Starts a new transaction. It's no-op for a newly started transaction. + * + * @throws SQLException when current transaction is active state or not able to + * start new transaction + */ + void begin() throws SQLException; + /** * Gets configuration tied to this connection. * 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 3a6a9bea7..1ffeea2a2 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 @@ -436,7 +436,7 @@ public void setAutoCommit(boolean autoCommit) throws SQLException { } } else { // start new transaction if (!txRef.compareAndSet(null, createTransaction())) { - log.warn("[JDBC Compliant Mode] not able to start a new transaction, reuse the exist one"); + log.warn("Not able to start a new transaction, reuse the exist one: %s", txRef.get()); } } } @@ -449,9 +449,22 @@ public boolean getAutoCommit() throws SQLException { } @Override - public void commit() throws SQLException { - ensureOpen(); + public void begin() throws SQLException { + if (getAutoCommit()) { + throw SqlExceptionUtils.clientError("Cannot start new transaction in auto-commit mode"); + } + ensureTransactionSupport(); + + JdbcTransaction tx = txRef.get(); + if (tx == null || !tx.isNew()) { + // invalid transaction state + throw new SQLException(JdbcTransaction.ERROR_TX_STARTED, SqlExceptionUtils.SQL_STATE_INVALID_TX_STATE); + } + } + + @Override + public void commit() throws SQLException { if (getAutoCommit()) { throw SqlExceptionUtils.clientError("Cannot commit in auto-commit mode"); } @@ -475,8 +488,6 @@ public void commit() throws SQLException { @Override public void rollback() throws SQLException { - ensureOpen(); - if (getAutoCommit()) { throw SqlExceptionUtils.clientError("Cannot rollback in auto-commit mode"); } 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 e01e8f8ab..145dcaa64 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 @@ -5,6 +5,7 @@ import java.io.Serializable; import java.sql.ResultSet; import java.sql.SQLException; +import java.sql.SQLFeatureNotSupportedException; import java.sql.SQLWarning; import java.sql.Statement; import java.util.ArrayList; @@ -22,6 +23,7 @@ import com.clickhouse.client.ClickHouseRequest; import com.clickhouse.client.ClickHouseResponse; import com.clickhouse.client.ClickHouseResponseSummary; +import com.clickhouse.client.ClickHouseTransaction; import com.clickhouse.client.ClickHouseRequest.Mutation; import com.clickhouse.client.config.ClickHouseClientOption; import com.clickhouse.config.ClickHouseConfigChangeListener; @@ -93,6 +95,12 @@ private ClickHouseResponse getLastResponse(Map o ClickHouseResponse response = null; for (int i = 0, len = parsedStmts.length; i < len; i++) { ClickHouseSqlStatement stmt = parsedStmts[i]; + response = processSqlStatement(stmt); + if (response != null) { + updateResult(stmt, response); + continue; + } + if (stmt.hasFormat()) { request.format(ClickHouseFormat.valueOf(stmt.getFormat())); } @@ -124,6 +132,23 @@ protected void ensureOpen() throws SQLException { } } + protected ClickHouseResponse processSqlStatement(ClickHouseSqlStatement stmt) throws SQLException { + if (stmt.isTCL()) { + if (stmt.containsKeyword(ClickHouseTransaction.COMMAND_BEGIN)) { + connection.begin(); + } else if (stmt.containsKeyword(ClickHouseTransaction.COMMAND_COMMIT)) { + connection.commit(); + } else if (stmt.containsKeyword(ClickHouseTransaction.COMMAND_ROLLBACK)) { + connection.rollback(); + } else { + throw new SQLFeatureNotSupportedException("Unsupported TCL: " + stmt.getSQL()); + } + return ClickHouseResponse.EMPTY; + } + + return null; + } + protected ClickHouseResponse executeStatement(String stmt, Map options, List tables, Map settings) throws SQLException { boolean autoTx = connection.getAutoCommit() && connection.isTransactionSupported(); @@ -175,6 +200,10 @@ protected ClickHouseResponse executeStatement(String stmt, Map options, List tables, Map settings) throws SQLException { + ClickHouseResponse resp = processSqlStatement(stmt); + if (resp != null) { + return resp; + } return executeStatement(stmt.getSQL(), options, tables, settings); } @@ -261,7 +290,7 @@ protected ResultSet updateResult(ClickHouseSqlStatement stmt, ClickHouseResponse stmt.getTable(), this, response); } else { response.close(); - currentUpdateCount = stmt.isDDL() ? 0L + currentUpdateCount = stmt.isDDL() || stmt.isTCL() ? 0L : (response.getSummary().isEmpty() ? 1L : response.getSummary().getWrittenRows()); currentResult = null; } diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/JdbcTransaction.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/JdbcTransaction.java index 34f30d7a2..440bfb546 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/JdbcTransaction.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/internal/JdbcTransaction.java @@ -19,6 +19,7 @@ public class JdbcTransaction { static final String ACTION_ROLLBACK = "rolled back"; static final String ERROR_TX_NOT_STARTED = "Transaction not started"; + static final String ERROR_TX_STARTED = "Transaction has been started"; protected final ClickHouseTransaction tx; protected final String id; @@ -36,6 +37,11 @@ public JdbcTransaction(ClickHouseTransaction tx) { this.savepoints = new LinkedList<>(); } + public boolean isNew() { + return this.queries.isEmpty() && this.savepoints.isEmpty() + && (this.tx == null || this.tx.isNew() || this.tx.isActive()); + } + public void commit(Logger log) throws SQLException { if (this.tx != null) { try { 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 6e1b1875c..61a173fbd 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 @@ -130,6 +130,10 @@ public boolean isMutation() { return this.stmtType.getOperationType() == OperationType.WRITE || this.hasOutfile(); } + public boolean isTCL() { + return this.stmtType.getLanguageType() == LanguageType.TCL; + } + public boolean isIdemponent() { boolean result = this.stmtType.isIdempotent() && !this.hasOutfile(); diff --git a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/parser/StatementType.java b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/parser/StatementType.java index 3649d7a6a..30f7fa209 100644 --- a/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/parser/StatementType.java +++ b/clickhouse-jdbc/src/main/java/com/clickhouse/jdbc/parser/StatementType.java @@ -27,7 +27,8 @@ public enum StatementType { TRUNCATE(LanguageType.DDL, OperationType.UNKNOWN, true), // truncate statement UPDATE(LanguageType.DML, OperationType.WRITE, false), // the upcoming light-weight update statement USE(LanguageType.DDL, OperationType.UNKNOWN, true), // use statement - WATCH(LanguageType.DDL, OperationType.UNKNOWN, true); // watch statement + WATCH(LanguageType.DDL, OperationType.UNKNOWN, true), // watch statement + TRANSACTION(LanguageType.TCL, OperationType.WRITE, true); // TCL statement private LanguageType langType; private OperationType opType; diff --git a/clickhouse-jdbc/src/main/javacc/ClickHouseSqlParser.jj b/clickhouse-jdbc/src/main/javacc/ClickHouseSqlParser.jj index 60c3c9d58..331490e00 100644 --- a/clickhouse-jdbc/src/main/javacc/ClickHouseSqlParser.jj +++ b/clickhouse-jdbc/src/main/javacc/ClickHouseSqlParser.jj @@ -382,6 +382,7 @@ void stmt(): {} { | updateStmt() { token_source.stmtType = StatementType.UPDATE; } | useStmt() { token_source.stmtType = StatementType.USE; } | watchStmt() { token_source.stmtType = StatementType.WATCH; } + | txStmt() { token_source.stmtType = StatementType.TRANSACTION; } } // https://clickhouse.tech/docs/en/sql-reference/statements/alter/ @@ -641,6 +642,13 @@ void watchStmt(): {} { // not interested anyExprList() } +// TCL +void txStmt(): {} { // not interested + { token_source.addPosition(token); } + | { token_source.addPosition(token); } + | ( { token_source.addPosition(token); } ) +} + // columns void columnExprList(): {} { columnsExpr() ( columnsExpr())* @@ -995,6 +1003,9 @@ TOKEN: { |

> | > | > + | > + | > + | > | > | > @@ -1060,6 +1071,7 @@ TOKEN: { |

> |

> | > + | > | > | > | > diff --git a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseConnectionTest.java b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseConnectionTest.java index 71e703c64..701534853 100644 --- a/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseConnectionTest.java +++ b/clickhouse-jdbc/src/test/java/com/clickhouse/jdbc/ClickHouseConnectionTest.java @@ -14,6 +14,7 @@ import com.clickhouse.data.value.UnsignedByte; import org.testng.Assert; +import org.testng.SkipException; import org.testng.annotations.Test; public class ClickHouseConnectionTest extends JdbcIntegrationTest { @@ -68,13 +69,16 @@ public void testCreateArray() throws SQLException { } } - @Test // (groups = "integration") + @Test(groups = "integration") public void testAutoCommitMode() throws SQLException { Properties props = new Properties(); props.setProperty("transactionSupport", "true"); for (int i = 0; i < 10; i++) { - try (Connection conn = newConnection(props); Statement stmt = conn.createStatement()) { + try (ClickHouseConnection conn = newConnection(props); Statement stmt = conn.createStatement()) { + if (!conn.getServerVersion().check("[22.7,)")) { + throw new SkipException("Skip the test as transaction is supported since 22.7"); + } stmt.execute("select 1, throwIf(" + i + " % 3 = 0)"); stmt.executeQuery("select number, toDateTime(number), toString(number), throwIf(" + i + " % 5 = 0)" + " from numbers(100000)"); @@ -237,20 +241,15 @@ public void testReadOnly() throws SQLException { } } - @Test // (groups = "integration") - public void testTransaction() throws SQLException { - testAutoCommit(); - testManualCommit(); - testNestedTransactions(); - testParallelTransactions(); - } - - @Test // (groups = "integration") + @Test(groups = "integration") public void testAutoCommit() throws SQLException { Properties props = new Properties(); props.setProperty("transactionSupport", "true"); String tableName = "test_jdbc_tx_auto_commit"; - try (Connection c = newConnection(props); Statement s = c.createStatement()) { + try (ClickHouseConnection c = newConnection(props); Statement s = c.createStatement()) { + if (!c.getServerVersion().check("[22.7,)")) { + throw new SkipException("Skip the test as transaction is supported since 22.7"); + } s.execute("drop table if exists " + tableName + "; " + "create table " + tableName + "(id UInt64) engine=MergeTree order by id"); } @@ -332,15 +331,18 @@ public void testAutoCommit() throws SQLException { } } - @Test // (groups = "integration") - public void testManualCommit() throws SQLException { + @Test(groups = "integration") + public void testManualTxApi() throws SQLException { Properties props = new Properties(); props.setProperty("autoCommit", "false"); Properties txProps = new Properties(); txProps.putAll(props); txProps.setProperty("transactionSupport", "true"); - String tableName = "test_jdbc_manual_tx"; - try (Connection c = newConnection(txProps); Statement s = c.createStatement()) { + String tableName = "test_jdbc_manual_tx_api"; + try (ClickHouseConnection c = newConnection(txProps); Statement s = c.createStatement()) { + if (!c.getServerVersion().check("[22.7,)")) { + throw new SkipException("Skip the test as transaction is supported since 22.7"); + } s.execute("drop table if exists " + tableName + "; " + "create table " + tableName + "(id UInt64, value String) engine=MergeTree order by id"); } @@ -364,10 +366,15 @@ public void testManualCommit() throws SQLException { "Implicit transaction is NOT supported before 22.7"); } + Assert.assertThrows(SQLException.class, () -> conn.begin()); + Assert.assertThrows(SQLException.class, () -> txConn.begin()); + checkRowCount(stmt, "select 1", 1); checkRowCount(txStmt, "select 1", 1); + Assert.assertThrows(SQLException.class, () -> txConn.begin()); txConn.commit(); + txConn.begin(); checkRowCount(stmt, "select 1", 1); checkRowCount(txStmt, "select 1", 1); txConn.rollback(); @@ -435,13 +442,158 @@ public void testManualCommit() throws SQLException { } } - @Test // (groups = "integration") + @Test(groups = "integration") + public void testManualTxTcl() throws SQLException { + Properties props = new Properties(); + props.setProperty("autoCommit", "false"); + Properties txProps = new Properties(); + txProps.putAll(props); + txProps.setProperty("transactionSupport", "true"); + String tableName = "test_jdbc_manual_tx_tcl"; + try (ClickHouseConnection c = newConnection(txProps); Statement s = c.createStatement()) { + if (!c.getServerVersion().check("[22.7,)")) { + throw new SkipException("Skip the test as transaction is supported since 22.7"); + } + s.execute("drop table if exists " + tableName + "; " + + "create table " + tableName + "(id UInt64, value String) engine=MergeTree order by id"); + } + + try (ClickHouseConnection conn = newConnection(props); + ClickHouseConnection txConn = newConnection(txProps); + Statement stmt = conn.createStatement(); + Statement txStmt = txConn.createStatement(); + PreparedStatement ps = conn.prepareStatement("insert into " + tableName); + PreparedStatement txPs = txConn.prepareStatement("insert into " + tableName)) { + Assert.assertFalse(conn.getAutoCommit()); + Assert.assertFalse(txConn.getAutoCommit()); + Assert.assertFalse(conn.isTransactionSupported()); + Assert.assertTrue(txConn.isTransactionSupported()); + Assert.assertFalse(conn.isImplicitTransactionSupported()); + if (txConn.getServerVersion().check("[22.7,)")) { + Assert.assertTrue(txConn.isImplicitTransactionSupported(), + "Implicit transaction is supported since 22.7"); + } else { + Assert.assertFalse(txConn.isImplicitTransactionSupported(), + "Implicit transaction is NOT supported before 22.7"); + } + + Assert.assertThrows(SQLException.class, () -> stmt.execute("begin transaction")); + Assert.assertThrows(SQLException.class, () -> txStmt.execute("begin transaction")); + + checkRowCount(stmt, "select 1", 1); + checkRowCount(txStmt, "select 1", 1); + try (Statement s = conn.createStatement()) { + Assert.assertEquals(s.executeUpdate("commit"), 0); + } + try (Statement s = txConn.createStatement()) { + Assert.assertEquals(s.executeUpdate("commit"), 0); + } + + Assert.assertEquals(stmt.executeUpdate("begin transaction"), 0); + Assert.assertEquals(txStmt.executeUpdate("begin transaction"), 0); + checkRowCount(stmt, "begin transaction; select 1", 1); + checkRowCount(txStmt, "begin transaction; select 1", 1); + try (Statement s = txConn.createStatement()) { + Assert.assertEquals(s.executeUpdate("rollback"), 0); + } + + checkRowCount(stmt, tableName, 0); + checkRowCount(txStmt, tableName, 0); + + txStmt.executeUpdate("insert into " + tableName + " values(0, '0')"); + checkRowCount(stmt, tableName, 1); + checkRowCount(txStmt, tableName, 1); + try (Statement s = txConn.createStatement()) { + Assert.assertEquals(s.executeUpdate("rollback"), 0); + } + checkRowCount(stmt, tableName, 0); + checkRowCount(txStmt, tableName, 0); + + stmt.executeUpdate("insert into " + tableName + " values(1, 'a')"); + checkRowCount(stmt, tableName, 1); + checkRowCount(txStmt, tableName, 1); + + txStmt.executeUpdate("insert into " + tableName + " values(2, 'b')"); + checkRowCount(stmt, tableName, 2); + checkRowCount(txStmt, tableName, 2); + + try (Connection c = newConnection(txProps); Statement s = c.createStatement()) { + s.executeUpdate("insert into " + tableName + " values(-1, '-1')"); + checkRowCount(stmt, tableName, 3); + checkRowCount(txStmt, tableName, 2); + checkRowCount(s, tableName, 2); + try (Statement ss = c.createStatement()) { + Assert.assertEquals(ss.executeUpdate("rollback"), 0); + } + checkRowCount(stmt, tableName, 2); + checkRowCount(txStmt, tableName, 2); + checkRowCount(s, tableName, 1); + } + checkRowCount(stmt, tableName, 2); + checkRowCount(txStmt, tableName, 2); + + try (Connection c = newConnection(txProps); Statement s = c.createStatement()) { + s.executeUpdate("insert into " + tableName + " values(3, 'c')"); + checkRowCount(stmt, tableName, 3); + checkRowCount(txStmt, tableName, 2); + checkRowCount(s, tableName, 2); + try (Statement ss = txConn.createStatement()) { + Assert.assertEquals(ss.executeUpdate("commit"), 0); + } + checkRowCount(stmt, tableName, 3); + checkRowCount(txStmt, tableName, 2); + checkRowCount(s, tableName, 2); + } + checkRowCount(stmt, tableName, 3); + checkRowCount(txStmt, tableName, 2); + try (Statement s = txConn.createStatement()) { + Assert.assertEquals(s.executeUpdate("commit"), 0); + } + checkRowCount(txStmt, tableName, 3); + + try (Statement s = conn.createStatement()) { + Assert.assertEquals(s.executeUpdate("commit"), 0); + } + try (Statement s = txConn.createStatement()) { + Assert.assertEquals(s.executeUpdate("commit"), 0); + } + txStmt.addBatch("begin transaction"); + txStmt.addBatch("insert into " + tableName + " values(4, 'd')"); + txStmt.addBatch("insert into " + tableName + " values(5, 'e')"); + txStmt.addBatch("commit"); + txStmt.executeBatch(); + + txStmt.addBatch("insert into " + tableName + " values(6, 'f')"); + txStmt.addBatch("rollback"); + txStmt.executeBatch(); + + txConn.setAutoCommit(true); + Assert.assertTrue(txConn.getAutoCommit()); + try (Statement s = txConn.createStatement()) { + s.executeUpdate("insert into " + tableName + " values(6, 'f')"); + checkRowCount(stmt, tableName, 6); + checkRowCount(txStmt, tableName, 6); + checkRowCount(s, tableName, 6); + } + + try (Statement s = txConn.createStatement()) { + checkRowCount(stmt, tableName, 6); + checkRowCount(txStmt, tableName, 6); + checkRowCount(s, tableName, 6); + } + } + } + + @Test(groups = "integration") public void testNestedTransactions() throws SQLException { Properties props = new Properties(); props.setProperty("autoCommit", "false"); props.setProperty("transactionSupport", "true"); String tableName = "test_jdbc_nested_tx"; - try (Connection c = newConnection(props); Statement s = c.createStatement()) { + try (ClickHouseConnection c = newConnection(props); Statement s = c.createStatement()) { + if (!c.getServerVersion().check("[22.7,)")) { + throw new SkipException("Skip the test as transaction is supported since 22.7"); + } s.execute("drop table if exists " + tableName + "; " + "create table " + tableName + "(id UInt64) engine=MergeTree order by id"); } @@ -474,13 +626,16 @@ public void testNestedTransactions() throws SQLException { } } - @Test // (groups = "integration") + @Test(groups = "integration") public void testParallelTransactions() throws SQLException { Properties props = new Properties(); props.setProperty("autoCommit", "false"); props.setProperty("transactionSupport", "true"); String tableName = "test_jdbc_parallel_tx"; - try (Connection c = newConnection(props); Statement s = c.createStatement()) { + try (ClickHouseConnection c = newConnection(props); Statement s = c.createStatement()) { + if (!c.getServerVersion().check("[22.7,)")) { + throw new SkipException("Skip the test as transaction is supported since 22.7"); + } s.execute("drop table if exists " + tableName + "; " + "create table " + tableName + "(id UInt64) engine=MergeTree order by id"); } 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 b47726710..05376e9ae 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 @@ -729,6 +729,21 @@ public void testNewParameterSyntax() { assertEquals(stmts[0].getSQL(), sql); } + @Test(groups = "unit") + public void testTcl() { + ClickHouseSqlStatement[] stmts = parse("begin transaction; commit;rollback;"); + assertEquals(stmts.length, 3); + assertEquals(stmts[0].isTCL(), true); + assertEquals(stmts[0].containsKeyword("bEGin"), true); + assertEquals(stmts[0].getSQL(), "begin transaction"); + assertEquals(stmts[1].isTCL(), true); + assertEquals(stmts[1].containsKeyword("Commit"), true); + assertEquals(stmts[1].getSQL(), " commit"); + assertEquals(stmts[2].isTCL(), true); + assertEquals(stmts[2].containsKeyword("RollBack"), true); + assertEquals(stmts[2].getSQL(), "rollback"); + } + // known issue public void testTernaryOperator() { String sql = "select x > 2 ? 'a' : 'b' from (select number as x from system.numbers limit ?)";