diff --git a/.github/workflows/benchmark.yml b/.github/workflows/benchmark.yml index 6a76764c3..49700cd17 100644 --- a/.github/workflows/benchmark.yml +++ b/.github/workflows/benchmark.yml @@ -62,7 +62,7 @@ jobs: mvn --batch-mode --update-snapshots -q -DskipTests install cd clickhouse-benchmark mvn --batch-mode --update-snapshots install - java -jar target/benchmarks.jar -rf text -p client=clickhouse-jdbc Basic + java -DclickhouseVersion="21.3" -jar target/benchmarks.jar -rf text -p client=clickhouse-jdbc Basic echo "BENCHMARK_REPORT<> $GITHUB_ENV cat jmh-result.text >> $GITHUB_ENV echo "EOF" >> $GITHUB_ENV @@ -110,7 +110,7 @@ jobs: run: | mvn --batch-mode --update-snapshots -DskipTests -pl clickhouse-benchmark -am package cd clickhouse-benchmark - java -jar target/benchmarks.jar -rf json ${{ github.event.inputs.options }} > output.txt + java -DclickhouseVersion="21.3" -jar target/benchmarks.jar -rf json ${{ github.event.inputs.options }} > output.txt echo "BENCHMARK_REPORT<> $GITHUB_ENV tail -n +$(grep -n '^REMEMBER:' output.txt | tail -1 | awk -F: '{print $1+6}') output.txt | head -n -2 | grep -v ':ยท' >> $GITHUB_ENV echo "EOF" >> $GITHUB_ENV diff --git a/.gitignore b/.gitignore index 1c98dd67a..8e4ab4f9f 100644 --- a/.gitignore +++ b/.gitignore @@ -36,6 +36,7 @@ target/ **/parser/ClickHouseSqlParserTokenManager.java **/parser/Token*.java **/parser/ParseException.java +jmh-result.* # Shell scripts *.sh diff --git a/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/ClientState.java b/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/ClientState.java index 62d44bf6b..13d787529 100644 --- a/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/ClientState.java +++ b/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/ClientState.java @@ -33,8 +33,7 @@ public void doSetup(ServerState serverState) throws Exception { serverState.getUser(), serverState.getPassword()), new Properties()); try (Statement s = conn.createStatement()) { - s.execute( - "create table if not exists test_insert(i Nullable(UInt64), s Nullable(String), t Nullable(DateTime))engine=Memory"); + s.execute("create table if not exists system.test_insert(i Nullable(UInt64), s Nullable(String), t Nullable(DateTime))engine=Memory"); } } catch (SQLException e) { e.printStackTrace(); @@ -43,9 +42,9 @@ public void doSetup(ServerState serverState) throws Exception { } @TearDown(Level.Trial) - public void doTearDown() throws SQLException { + public void doTearDown(ServerState serverState) throws SQLException { try (Statement s = conn.createStatement()) { - s.execute("drop table if exists test_insert"); + s.execute("drop table if exists system.test_insert"); } conn.close(); } diff --git a/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/Insertion.java b/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/Insertion.java index 44eea7efe..b6b2cfe17 100644 --- a/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/Insertion.java +++ b/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/Insertion.java @@ -19,7 +19,7 @@ public int insert10kUInt64Rows(ClientState state) throws Throwable { final int rows = 10000; final int num = new Random().nextInt(rows); - return executeInsert(state, "insert into test_insert(i) values(?)", new Enumeration() { + return executeInsert(state, "insert into system.test_insert(i) values(?)", new Enumeration() { int counter = 0; @Override @@ -39,7 +39,7 @@ public int insert10kStringRows(ClientState state) throws Throwable { final int rows = 10000; final int num = new Random().nextInt(rows); - return executeInsert(state, "insert into test_insert(s) values(?)", new Enumeration() { + return executeInsert(state, "insert into system.test_insert(s) values(?)", new Enumeration() { int counter = 0; @Override @@ -59,7 +59,7 @@ public int insert10kTimestampRows(ClientState state) throws Throwable { final int rows = 10000; final int num = new Random().nextInt(rows); - return executeInsert(state, "insert into test_insert(t) values(?)", new Enumeration() { + return executeInsert(state, "insert into system.test_insert(t) values(?)", new Enumeration() { int counter = 0; @Override diff --git a/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/ClickHousePreparedStatementImpl.java b/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/ClickHousePreparedStatementImpl.java index 9a384d5dc..4abb48f95 100644 --- a/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/ClickHousePreparedStatementImpl.java +++ b/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/ClickHousePreparedStatementImpl.java @@ -19,7 +19,6 @@ import java.sql.RowId; import java.sql.SQLException; import java.sql.SQLFeatureNotSupportedException; -import java.sql.SQLSyntaxErrorException; import java.sql.SQLXML; import java.sql.Time; import java.sql.Timestamp; @@ -31,15 +30,14 @@ import java.util.List; import java.util.Map; import java.util.TimeZone; -import java.util.regex.Matcher; -import java.util.regex.Pattern; import org.apache.http.entity.AbstractHttpEntity; import org.apache.http.impl.client.CloseableHttpClient; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import ru.yandex.clickhouse.jdbc.parser.ClickHouseSqlStatement; import ru.yandex.clickhouse.jdbc.parser.StatementType; - import ru.yandex.clickhouse.response.ClickHouseResponse; import ru.yandex.clickhouse.settings.ClickHouseProperties; import ru.yandex.clickhouse.settings.ClickHouseQueryParam; @@ -47,6 +45,7 @@ import ru.yandex.clickhouse.util.ClickHouseValueFormatter; public class ClickHousePreparedStatementImpl extends ClickHouseStatementImpl implements ClickHousePreparedStatement { + private static final Logger log = LoggerFactory.getLogger(ClickHousePreparedStatementImpl.class); static final String PARAM_MARKER = "?"; static final String NULL_MARKER = "\\N"; @@ -95,18 +94,19 @@ public void clearParameters() { @Override public ClickHouseResponse executeQueryClickhouseResponse() throws SQLException { - return super.executeQueryClickhouseResponse(buildSql()); + return executeQueryClickhouseResponse(buildSql(), null, null); } @Override public ClickHouseResponse executeQueryClickhouseResponse(Map additionalDBParams) throws SQLException { - return super.executeQueryClickhouseResponse(buildSql(), additionalDBParams); + return executeQueryClickhouseResponse(buildSql(), additionalDBParams, null); } - private String buildSql() throws SQLException { + private ClickHouseSqlStatement buildSql() throws SQLException { if (sqlParts.size() == 1) { - return sqlParts.get(0); + return new ClickHouseSqlStatement(sqlParts.get(0), parsedStmt.getStatementType()); } + checkBinded(); StringBuilder sb = new StringBuilder(sqlParts.get(0)); for (int i = 1, p = 0; i < sqlParts.size(); i++) { @@ -120,7 +120,7 @@ private String buildSql() throws SQLException { } sb.append(sqlParts.get(i)); } - return sb.toString(); + return new ClickHouseSqlStatement(sb.toString(), parsedStmt.getStatementType()); } private void checkBinded() throws SQLException { @@ -135,32 +135,34 @@ private void checkBinded() throws SQLException { @Override public boolean execute() throws SQLException { - return super.execute(buildSql()); + return executeQueryStatement(buildSql(), null, null, null) != null; } @Override public ResultSet executeQuery() throws SQLException { - return super.executeQuery(buildSql()); + return executeQueryStatement(buildSql(), null, null, null); } @Override public void clearBatch() throws SQLException { - batchRows.clear(); + super.clearBatch(); + + batchRows = new ArrayList<>(); } @Override public ResultSet executeQuery(Map additionalDBParams) throws SQLException { - return super.executeQuery(buildSql(), additionalDBParams); + return executeQuery(additionalDBParams, null); } @Override public ResultSet executeQuery(Map additionalDBParams, List externalData) throws SQLException { - return super.executeQuery(buildSql(), additionalDBParams, externalData); + return executeQueryStatement(buildSql(), additionalDBParams, externalData, null); } @Override public int executeUpdate() throws SQLException { - return super.executeUpdate(buildSql()); + return executeStatement(buildSql(), null, null, null); } private void setBind(int parameterIndex, String bind, boolean quote) { @@ -318,9 +320,18 @@ public void setObject(int parameterIndex, Object x) throws SQLException { } } + @Override + public void addBatch(String sql) throws SQLException { + throw new SQLException("addBatch(String) cannot be called in PreparedStatement or CallableStatement!"); + } + @Override public void addBatch() throws SQLException { - batchRows.addAll(buildBatch()); + if (parsedStmt.getStatementType() == StatementType.INSERT) { + batchRows.addAll(buildBatch()); + } else { + batchStmts.add(buildSql()); + } } private List buildBatch() throws SQLException { @@ -357,22 +368,27 @@ public int[] executeBatch() throws SQLException { public int[] executeBatch(Map additionalDBParams) throws SQLException { int valuePosition = -1; String sql = parsedStmt.getSQL(); - if (parsedStmt.getStatementType() == StatementType.INSERT && parsedStmt.hasValues()) { + StatementType type = parsedStmt.getStatementType(); + if (type == StatementType.INSERT && parsedStmt.hasValues()) { valuePosition = parsedStmt.getStartPosition(ClickHouseSqlStatement.KEYWORD_VALUES); } - if (valuePosition < 0) { - throw new SQLSyntaxErrorException( - "Query must be like 'INSERT INTO [db.]table [(c1, c2, c3)] VALUES (?, ?, ?)'. " + - "Got: " + sql - ); - } - String insertSql = sql.substring(0, valuePosition); - BatchHttpEntity entity = new BatchHttpEntity(batchRows); - sendStream(entity, insertSql, additionalDBParams); int[] result = new int[batchRows.size()]; Arrays.fill(result, 1); - batchRows = new ArrayList<>(); + if (valuePosition > 0) { // insert + String insertSql = sql.substring(0, valuePosition); + BatchHttpEntity entity = new BatchHttpEntity(batchRows); + sendStream(entity, insertSql, additionalDBParams); + } else { // others + if (type == StatementType.ALTER_DELETE || type == StatementType.ALTER_UPDATE) { + log.warn("UPDATE and DELETE should be used with caution, as they are expensive operations and not supposed to be used frequently."); + } else { + log.warn("PreparedStatement will slow down non-INSERT queries, so please consider to use Statement instead."); + } + result = super.executeBatch(); + } + + clearBatch(); return result; } @@ -623,7 +639,7 @@ private String getParameter(int paramIndex) { @Override public String asSql() { try { - return buildSql(); + return buildSql().getSQL(); } catch (SQLException e) { return parsedStmt.getSQL(); } 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 c182e4075..9e8da7c73 100644 --- a/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/ClickHouseStatementImpl.java +++ b/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/ClickHouseStatementImpl.java @@ -87,6 +87,8 @@ public class ClickHouseStatementImpl extends ConfigurableApi batchStmts; + /** * Current database name may be changed by {@link java.sql.Connection#setCatalog(String)} * between creation of this object and query execution, but javadoc does not allow @@ -132,21 +134,120 @@ protected ClickHouseSqlStatement parseSqlStatements( } ClickHouseSqlStatement lastStmt = getLastStatement(); + ClickHouseSqlStatement formattedStmt = applyFormat(lastStmt, preferredFormat); + if (formattedStmt != lastStmt) { + setLastStatement(lastStmt = formattedStmt); + } + + return lastStmt; + } + + protected ClickHouseSqlStatement applyFormat(ClickHouseSqlStatement stmt, ClickHouseFormat preferredFormat) { + if (Objects.requireNonNull(stmt).isQuery() && !stmt.hasFormat()) { + String sql = stmt.getSQL(); + String format = Objects.requireNonNull(preferredFormat).name(); - if (lastStmt.isQuery() && !lastStmt.hasFormat()) { - String format = preferredFormat.name(); Map positions = new HashMap<>(); - positions.putAll(lastStmt.getPositions()); + positions.putAll(stmt.getPositions()); positions.put(ClickHouseSqlStatement.KEYWORD_FORMAT, sql.length()); - - sql = new StringBuilder(lastStmt.getSQL()).append("\nFORMAT ").append(format).toString(); - lastStmt = new ClickHouseSqlStatement(sql, lastStmt.getStatementType(), - lastStmt.getCluster(), lastStmt.getDatabase(), lastStmt.getTable(), - format, lastStmt.getOutfile(), lastStmt.getParameters(), positions); - setLastStatement(lastStmt); + + sql = new StringBuilder(sql).append("\nFORMAT ").append(format).toString(); + stmt = new ClickHouseSqlStatement(sql, stmt.getStatementType(), + stmt.getCluster(), stmt.getDatabase(), stmt.getTable(), + format, stmt.getOutfile(), stmt.getParameters(), positions); } - return lastStmt; + return stmt; + } + + protected ResultSet updateResult(ClickHouseSqlStatement stmt, InputStream is) throws IOException, ClickHouseException { + ResultSet rs = null; + if (stmt.isQuery()) { + currentUpdateCount = -1; + currentResult = createResultSet( + properties.isCompress() ? new ClickHouseLZ4Stream(is) : is, properties.getBufferSize(), + stmt.getDatabaseOrDefault(properties.getDatabase()), + stmt.getTable(), + stmt.hasWithTotals(), + this, + getConnection().getTimeZone(), + properties + ); + currentResult.setMaxRows(maxRows); + rs = currentResult; + } else { + currentUpdateCount = 0; + try { + is.close(); + } catch (IOException e) { + log.error("can not close stream: {}", e.getMessage()); + } + } + + return rs; + } + + protected int executeStatement( + ClickHouseSqlStatement stmt, + Map additionalDBParams, + List externalData, + Map additionalRequestParams) throws SQLException { + stmt = applyFormat(stmt, ClickHouseFormat.TabSeparatedWithNamesAndTypes); + if (additionalDBParams == null || additionalDBParams.isEmpty()) { + additionalDBParams = new EnumMap<>(ClickHouseQueryParam.class); + } else { + additionalDBParams = new EnumMap<>(additionalDBParams); + } + try (InputStream is = getInputStream(stmt, additionalDBParams, externalData, additionalRequestParams)) { + //noinspection StatementWithEmptyBody + } catch (IOException e) { + log.error("can not close stream: {}", e.getMessage()); + } + + return currentSummary != null ? (int) currentSummary.getWrittenRows() : 1; + } + + protected ResultSet executeQueryStatement(ClickHouseSqlStatement stmt, + Map additionalDBParams, + List externalData, + Map additionalRequestParams) throws SQLException { + stmt = applyFormat(stmt, ClickHouseFormat.TabSeparatedWithNamesAndTypes); + if (additionalDBParams == null || additionalDBParams.isEmpty()) { + additionalDBParams = new EnumMap<>(ClickHouseQueryParam.class); + } else { + additionalDBParams = new EnumMap<>(additionalDBParams); + } + + InputStream is = getInputStream(stmt, additionalDBParams, externalData, additionalRequestParams); + try { + return updateResult(stmt, is); + } catch (Exception e) { + try { + is.close(); + } catch (IOException ioe) { + log.error("can not close stream: {}", ioe.getMessage()); + } + throw ClickHouseExceptionSpecifier.specify(e, properties.getHost(), properties.getPort()); + } + } + + protected ClickHouseResponse executeQueryClickhouseResponse( + ClickHouseSqlStatement stmt, + Map additionalDBParams, + Map additionalRequestParams) throws SQLException { + if (additionalDBParams == null || additionalDBParams.isEmpty()) { + additionalDBParams = new EnumMap<>(ClickHouseQueryParam.class); + } else { + additionalDBParams = new EnumMap<>(additionalDBParams); + } + stmt = applyFormat(stmt, ClickHouseFormat.JSONCompact); + + try (InputStream is = getInputStream(stmt, additionalDBParams, null, additionalRequestParams)) { + return Jackson.getObjectMapper().readValue( + properties.isCompress() ? new ClickHouseLZ4Stream(is) : is, ClickHouseResponse.class); + } catch (IOException e) { + throw new RuntimeException(e); + } } public ClickHouseStatementImpl(CloseableHttpClient client, ClickHouseConnection connection, @@ -158,6 +259,8 @@ public ClickHouseStatementImpl(CloseableHttpClient client, ClickHouseConnection this.properties = properties == null ? new ClickHouseProperties() : properties; this.initialDatabase = this.properties.getDatabase(); this.isResultSetScrollable = (resultSetType != ResultSet.TYPE_FORWARD_ONLY); + + this.batchStmts = new ArrayList<>(); } @Override @@ -196,28 +299,7 @@ public ResultSet executeQuery(String sql, ClickHouseSqlStatement parsedStmt = getLastStatement(); try { - if (parsedStmt.isQuery()) { - currentUpdateCount = -1; - currentResult = createResultSet(properties.isCompress() - ? new ClickHouseLZ4Stream(is) : is, properties.getBufferSize(), - parsedStmt.getDatabaseOrDefault(properties.getDatabase()), - parsedStmt.getTable(), - parsedStmt.hasWithTotals(), - this, - getConnection().getTimeZone(), - properties - ); - currentResult.setMaxRows(maxRows); - return currentResult; - } else { - currentUpdateCount = 0; - try { - is.close(); - } catch (IOException e) { - log.error("can not close stream: {}", e.getMessage()); - } - return null; - } + return updateResult(parsedStmt, is); } catch (Exception e) { try { is.close(); @@ -459,17 +541,27 @@ public int getResultSetType() throws SQLException { @Override public void addBatch(String sql) throws SQLException { - + for (ClickHouseSqlStatement s : ClickHouseSqlParser.parse(sql, properties)) { + this.batchStmts.add(s); + } } @Override public void clearBatch() throws SQLException { - + this.batchStmts = new ArrayList<>(); } @Override public int[] executeBatch() throws SQLException { - return new int[0]; + int len = batchStmts.size(); + int[] results = new int[len]; + for (int i = 0; i < len; i++) { + results[i] = executeStatement(batchStmts.get(i), null, null, null); + } + + clearBatch(); + + return results; } @Override @@ -555,8 +647,10 @@ public ClickHouseResponseSummary getResponseSummary() { return currentSummary; } - private InputStream getLastInputStream(Map additionalDBParams, - List externalData, Map additionalRequestParams) throws ClickHouseException { + private InputStream getLastInputStream( + Map additionalDBParams, + List externalData, + Map additionalRequestParams) throws ClickHouseException { InputStream is = null; for (int i = 0, len = parsedStmts.length; i < len; i++) { // TODO skip useless queries to reduce network calls and server load diff --git a/clickhouse-jdbc/src/main/javacc/ClickHouseSqlParser.jj b/clickhouse-jdbc/src/main/javacc/ClickHouseSqlParser.jj index 9c13c8a02..fbdd709ff 100644 --- a/clickhouse-jdbc/src/main/javacc/ClickHouseSqlParser.jj +++ b/clickhouse-jdbc/src/main/javacc/ClickHouseSqlParser.jj @@ -679,6 +679,7 @@ void anyExpr(): {} { anyNestedExpr() ( LOOKAHEAD(2) ( + // TODO needs to extract parameters(and exclude ternary operator) here | | | | | operator() )? anyNestedExpr() )* diff --git a/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/integration/ClickHousePreparedStatementTest.java b/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/integration/ClickHousePreparedStatementTest.java index 3f283b8d7..c70dddf7f 100644 --- a/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/integration/ClickHousePreparedStatementTest.java +++ b/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/integration/ClickHousePreparedStatementTest.java @@ -32,6 +32,10 @@ import ru.yandex.clickhouse.settings.ClickHouseProperties; import static java.util.Collections.singletonList; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; public class ClickHousePreparedStatementTest { @@ -676,6 +680,93 @@ public void testStaticNullValue() throws Exception { ps0.executeUpdate(); } + // known issue + public void testTernaryOperator() throws Exception { + String sql = "select x > 2 ? 'a' : 'b' from (select number as x from system.numbers limit ?)"; + try (PreparedStatement s = connection.prepareStatement(sql)) { + int len = 5; + s.setInt(1, len); + ResultSet rs = s.executeQuery(); + for (int i = 0; i < len; i++) { + assertTrue(rs.next()); + assertEquals(rs.getString(1), i > 2 ? "a" : "b"); + } + assertFalse(rs.next()); + rs.close(); + } + } + + @Test + public void testBatchProcess() throws Exception { + try (PreparedStatement s = connection.prepareStatement( + "create table if not exists test.batch_update(k UInt8, v String) engine=MergeTree order by k")) { + s.execute(); + } + + Object[][] data = new Object[][] { + new Object[] {1, "a"}, + new Object[] {1, "b"}, + new Object[] {3, "c"} + }; + + // insert + try (PreparedStatement s = connection.prepareStatement("insert into table test.batch_update values(?,?)")) { + for (int i = 0; i < data.length; i++) { + Object[] row = data[i]; + s.setInt(1, (int) row[0]); + s.setString(2, (String) row[1]); + s.addBatch(); + } + int[] results = s.executeBatch(); + assertNotNull(results); + assertEquals(results.length, 3); + } + + // select + try (PreparedStatement s = connection.prepareStatement( + "select * from test.batch_update where k in (?, ?) order by k, v")) { + s.setInt(1, 1); + s.setInt(2, 3); + ResultSet rs = s.executeQuery(); + int index = 0; + while (rs.next()) { + Object[] row = data[index++]; + assertEquals(rs.getInt(1), (int) row[0]); + assertEquals(rs.getString(2), (String) row[1]); + } + assertEquals(index, data.length); + } + + // update + try (PreparedStatement s = connection.prepareStatement( + "alter table test.batch_update update v = ? where k = ?")) { + s.setString(1, "x"); + s.setInt(2, 1); + s.addBatch(); + s.setString(1, "y"); + s.setInt(2, 3); + s.addBatch(); + int[] results = s.executeBatch(); + assertNotNull(results); + assertEquals(results.length, 2); + } + + // delete + try (PreparedStatement s = connection.prepareStatement("alter table test.batch_update delete where k = ?")) { + s.setInt(1, 1); + s.addBatch(); + s.setInt(1, 3); + s.addBatch(); + int[] results = s.executeBatch(); + assertNotNull(results); + assertEquals(results.length, 2); + } + + try (PreparedStatement s = connection.prepareStatement("drop table if exists test.batch_update")) { + s.execute(); + } + } + private static byte[] randomEncodedUUID() { UUID uuid = UUID.randomUUID(); return ByteBuffer.allocate(16) diff --git a/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/integration/ClickHouseStatementImplTest.java b/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/integration/ClickHouseStatementImplTest.java index 9987d7fb9..a51ef7ee2 100644 --- a/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/integration/ClickHouseStatementImplTest.java +++ b/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/integration/ClickHouseStatementImplTest.java @@ -432,6 +432,26 @@ public void testMultiStatements() throws SQLException { } } + @Test + public void testBatchProcessing() throws SQLException { + try (Statement s = connection.createStatement()) { + int[] results = s.executeBatch(); + assertNotNull(results); + assertEquals(results.length, 0); + + s.addBatch("select 1; select 2"); + s.addBatch("select 3"); + results = s.executeBatch(); + assertNotNull(results); + assertEquals(results.length, 3); + + s.clearBatch(); + results = s.executeBatch(); + assertNotNull(results); + assertEquals(results.length, 0); + } + } + private static Object readField(Object object, String fieldName, long timeoutSecs) { long start = System.currentTimeMillis(); Object value; diff --git a/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/jdbc/parser/ClickHouseSqlParserTest.java b/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/jdbc/parser/ClickHouseSqlParserTest.java index fb96cc75e..e95ef61d1 100644 --- a/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/jdbc/parser/ClickHouseSqlParserTest.java +++ b/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/jdbc/parser/ClickHouseSqlParserTest.java @@ -587,6 +587,15 @@ public void testExtractDBAndTableName() { StatementType.SELECT, "db", ".inner.a"); } + // known issue + public void testTernaryOperator() { + String sql = "select x > 2 ? 'a' : 'b' from (select number as x from system.numbers limit ?)"; + ClickHouseSqlStatement[] stmts = parse(sql); + assertEquals(stmts.length, 1); + assertEquals(stmts[0].getStatementType(), StatementType.SELECT); + assertEquals(stmts[0].getParameters().size(), 1); + } + static void parseAllSqlFiles(File f) throws IOException { if (f.isDirectory()) { File[] files = f.listFiles();