diff --git a/client-v2/src/main/java/com/clickhouse/client/api/Client.java b/client-v2/src/main/java/com/clickhouse/client/api/Client.java index 8883eb221..5b37a8f88 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/Client.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/Client.java @@ -41,6 +41,7 @@ import com.clickhouse.data.ClickHouseFormat; import org.apache.hc.core5.concurrent.DefaultThreadFactory; import org.apache.hc.core5.http.ClassicHttpResponse; +import org.apache.hc.core5.http.Header; import org.apache.hc.core5.http.HttpHeaders; import org.apache.hc.core5.http.HttpStatus; import org.slf4j.Logger; @@ -1639,8 +1640,13 @@ public CompletableFuture query(String sqlQuery, Map next(); diff --git a/client-v2/src/main/java/com/clickhouse/client/api/query/QueryResponse.java b/client-v2/src/main/java/com/clickhouse/client/api/query/QueryResponse.java index 532e68a28..735fe6f58 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/query/QueryResponse.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/query/QueryResponse.java @@ -133,6 +133,15 @@ public long getResultRows() { return operationMetrics.getMetric(ServerMetrics.RESULT_ROWS).getLong(); } + + /** + * Alias for {@link ServerMetrics#TOTAL_ROWS_TO_READ} + * @return estimated number of rows to read + */ + public long getTotalRowsToRead() { + return operationMetrics.getMetric(ServerMetrics.TOTAL_ROWS_TO_READ).getLong(); + } + /** * Alias for {@link OperationMetrics#getQueryId()} * @return query id of the request diff --git a/client-v2/src/test/java/com/clickhouse/client/insert/SamplePOJO.java b/client-v2/src/test/java/com/clickhouse/client/insert/SamplePOJO.java index 47d260256..9a55e823b 100644 --- a/client-v2/src/test/java/com/clickhouse/client/insert/SamplePOJO.java +++ b/client-v2/src/test/java/com/clickhouse/client/insert/SamplePOJO.java @@ -85,6 +85,8 @@ public class SamplePOJO { private ClickHouseBitmap groupBitmapUint32; private ClickHouseBitmap groupBitmapUint64; + private String keyword; + public SamplePOJO() { final Random random = new Random(); byteValue = (byte) random.nextInt(); @@ -180,6 +182,8 @@ public SamplePOJO() { groupBitmapUint32 = ClickHouseBitmap.wrap(random.ints(5, Integer.MAX_VALUE - 100, Integer.MAX_VALUE).toArray()); groupBitmapUint64 = ClickHouseBitmap.wrap(random.longs(5, Long.MAX_VALUE - 100, Long.MAX_VALUE).toArray()); + + keyword = "database"; } public byte getByteValue() { @@ -574,6 +578,14 @@ public void setGroupBitmapUint64(ClickHouseBitmap groupBitmapUint64) { this.groupBitmapUint64 = groupBitmapUint64; } + public String getKeyword() { + return keyword; + } + + public void setKeyword(String keyword) { + this.keyword = keyword; + } + @Override public String toString() { return "SamplePOJO{" + @@ -683,7 +695,8 @@ public static String generateTableCreateSQL(String tableName) { "nested Nested (innerInt Int32, innerString String, " + "innerNullableInt Nullable(Int32)), " + "groupBitmapUint32 AggregateFunction(groupBitmap, UInt32), " + - "groupBitmapUint64 AggregateFunction(groupBitmap, UInt64) " + + "groupBitmapUint64 AggregateFunction(groupBitmap, UInt64), " + + "keyword LowCardinality(String) " + ") ENGINE = MergeTree ORDER BY ()"; } } diff --git a/client-v2/src/test/java/com/clickhouse/client/query/QueryTests.java b/client-v2/src/test/java/com/clickhouse/client/query/QueryTests.java index acfce3f50..2e318b75c 100644 --- a/client-v2/src/test/java/com/clickhouse/client/query/QueryTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/query/QueryTests.java @@ -34,6 +34,7 @@ import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.MappingIterator; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.io.BaseEncoding; import org.apache.commons.lang3.RandomStringUtils; import org.apache.commons.lang3.StringEscapeUtils; import org.testng.Assert; @@ -55,6 +56,7 @@ import java.net.Inet4Address; import java.net.Inet6Address; import java.net.InetAddress; +import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.time.LocalDate; import java.time.LocalDateTime; @@ -62,6 +64,7 @@ import java.time.ZonedDateTime; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -486,7 +489,8 @@ record = reader.next(); "col1 Array(UInt32)", "col2 Array(Array(Int32))", "col3 Array(UInt64)", - "col4 Array(Bool)" + "col4 Array(Bool)", + "col5 Array(String)" ); private final static List> ARRAY_VALUE_GENERATORS = Arrays.asList( @@ -505,8 +509,17 @@ record = reader.next(); RANDOM.longs(10, 0, Long.MAX_VALUE) .mapToObj(BigInteger::valueOf).collect(Collectors.toList()), c -> RANDOM.ints(10, 0, 1) - .mapToObj(i -> i == 0 ).collect(Collectors.toList()) - + .mapToObj(i -> i == 0 ).collect(Collectors.toList()), + c -> { + UUID uuid = UUID.randomUUID(); + byte[] bts = ByteBuffer.allocate(16) + .putLong(uuid.getMostSignificantBits()) + .putLong(uuid.getLeastSignificantBits()) + .array(); + String sep = "\\x"; + String hex = sep + BaseEncoding.base16().withSeparator(sep, 2).encode(bts); + return Arrays.asList(hex); + } ); @Test(groups = {"integration"}) @@ -546,6 +559,7 @@ public void testArrayValues() throws Exception { Assert.assertEquals(col4Values, data.get(0).get("col4")); boolean[] col4Array = reader.getBooleanArray("col4"); Assert.assertEquals(col4Array, ((List)data.get(0).get("col4")).toArray()); + Assert.assertEquals(reader.getList("col5"), ((List)data.get(0).get("col5"))); } @Test @@ -1452,6 +1466,13 @@ private Map writeValuesRow(StringBuilder insertStmtBuilder, List } insertStmtBuilder.setLength(insertStmtBuilder.length() - 2); insertStmtBuilder.append("}, "); + } else if (value instanceof List) { + insertStmtBuilder.append("["); + for (Object item : (List)value) { + insertStmtBuilder.append(quoteValue(item)).append(", "); + } + insertStmtBuilder.setLength(insertStmtBuilder.length() - 2); + insertStmtBuilder.append("], "); } else { insertStmtBuilder.append(value).append(", "); } @@ -1463,7 +1484,9 @@ private Map writeValuesRow(StringBuilder insertStmtBuilder, List private String quoteValue(Object value) { if (value instanceof String) { - return '\'' + value.toString() + '\''; + String strVal = (String)value; + + return '\'' + strVal.replaceAll("\\\\", "\\\\\\\\") + '\''; } return value.toString(); } @@ -2000,4 +2023,37 @@ public void testServerTimezone() throws Exception { Assert.assertEquals(serverUtcTime.withZoneSameInstant(ZoneId.of("America/New_York")), serverEstTime); } } + + @Test(groups = {"integration"}) + public void testLowCardinalityValues() throws Exception { + final String table = "test_low_cardinality_values"; + final String tableCreate = "CREATE TABLE " + table + "(rowID Int32, keyword LowCardinality(String)) Engine = MergeTree ORDER BY ()"; + + client.execute("DROP TABLE IF EXISTS " + table); + client.execute(tableCreate); + + client.execute("INSERT INTO " + table + " VALUES (0, 'db'), (1, 'fast'), (2, 'not a java')"); + String[] values = new String[] {"db", "fast", "not a java"}; + Collection records = client.queryAll("SELECT * FROM " + table); + for (GenericRecord record : records) { + int rowId = record.getInteger("rowID"); + Assert.assertEquals(record.getString("keyword"), values[rowId]); + } + } + + @Test(groups = {"integration"}) + public void testGettingRowsBeforeLimit() throws Exception { + int expectedTotalRowsToRead = 100; + List serverVersion = client.queryAll("SELECT version()"); + if (ClickHouseVersion.of(serverVersion.get(0).getString(1)).check("(,23.8]")) { + // issue in prev. release. + expectedTotalRowsToRead = 0; + } + + try (QueryResponse response = client.query("SELECT number FROM system.numbers LIMIT 100").get()) { + Assert.assertTrue(response.getResultRows() < 1000); + + Assert.assertEquals(response.getTotalRowsToRead(), expectedTotalRowsToRead); + } + } } diff --git a/jdbc-v2/src/main/java/com/clickhouse/jdbc/StatementImpl.java b/jdbc-v2/src/main/java/com/clickhouse/jdbc/StatementImpl.java index 0756fe847..16dadc003 100644 --- a/jdbc-v2/src/main/java/com/clickhouse/jdbc/StatementImpl.java +++ b/jdbc-v2/src/main/java/com/clickhouse/jdbc/StatementImpl.java @@ -182,6 +182,11 @@ public ResultSetImpl executeQuery(String sql, QuerySettings settings) throws SQL } else { response = connection.client.query(lastSql, mergedSettings).get(queryTimeout, TimeUnit.SECONDS); } + + if (response.getFormat().isText()) { + throw new SQLException("Only RowBinaryWithNameAndTypes is supported for output format. Please check your query.", + ExceptionUtils.SQL_STATE_CLIENT_ERROR); + } ClickHouseBinaryFormatReader reader = connection.client.newBinaryFormatReader(response); currentResultSet = new ResultSetImpl(this, response, reader); @@ -225,7 +230,7 @@ public int executeUpdate(String sql, QuerySettings settings) throws SQLException try (QueryResponse response = queryTimeout == 0 ? connection.client.query(lastSql, mergedSettings).get() : connection.client.query(lastSql, mergedSettings).get(queryTimeout, TimeUnit.SECONDS)) { currentResultSet = null; - updateCount = (int) response.getWrittenRows(); + updateCount = Math.max(0, (int) response.getWrittenRows()); // when statement alters schema no result rows returned. metrics = response.getMetrics(); lastQueryId = response.getQueryId(); } catch (Exception e) { @@ -601,4 +606,12 @@ public String enquoteNCharLiteral(String val) throws SQLException { checkClosed(); return Statement.super.enquoteNCharLiteral(val); } + + /** + * Return query ID of last executed statement. It is not guaranteed when statements is used concurrently. + * @return query ID + */ + public String getLastQueryId() { + return lastQueryId; + } } diff --git a/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java b/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java index 7868ccbbd..2e2627076 100644 --- a/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java +++ b/jdbc-v2/src/test/java/com/clickhouse/jdbc/DataTypeTests.java @@ -1,6 +1,5 @@ package com.clickhouse.jdbc; -import com.clickhouse.client.api.ClientConfigProperties; import com.clickhouse.data.Tuple; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -11,7 +10,6 @@ import java.math.BigInteger; import java.sql.Connection; import java.sql.Date; -import java.sql.DriverManager; import java.sql.JDBCType; import java.sql.PreparedStatement; import java.sql.ResultSet; @@ -26,7 +24,6 @@ import java.util.GregorianCalendar; import java.util.HashMap; import java.util.Map; -import java.util.Properties; import java.util.Random; import java.util.TimeZone; import java.util.UUID; @@ -318,7 +315,8 @@ public void testStringTypes() throws SQLException { runQuery("CREATE TABLE test_strings (order Int8, " + "str String, fixed FixedString(6), " + "enum Enum8('a' = 6, 'b' = 7, 'c' = 8), enum8 Enum8('a' = 1, 'b' = 2, 'c' = 3), enum16 Enum16('a' = 1, 'b' = 2, 'c' = 3), " - + "uuid UUID, ipv4 IPv4, ipv6 IPv6" + + "uuid UUID, ipv4 IPv4, ipv6 IPv6, " + + "escaped String " + ") ENGINE = MergeTree ORDER BY ()"); // Insert random (valid) values @@ -333,10 +331,10 @@ public void testStringTypes() throws SQLException { String uuid = UUID.randomUUID().toString(); String ipv4 = rand.nextInt(256) + "." + rand.nextInt(256) + "." + rand.nextInt(256) + "." + rand.nextInt(256); String ipv6 = "2001:adb8:85a3:1:2:8a2e:370:7334"; - + String escaped = "\\xA3\\xA3\\x12\\xA0\\xDF\\x13\\x4E\\x8C\\x87\\x74\\xD4\\x53\\xDB\\xFC\\x34\\x95"; try (Connection conn = getConnection()) { - try (PreparedStatement stmt = conn.prepareStatement("INSERT INTO test_strings VALUES ( 1, ?, ?, ?, ?, ?, ?, ?, ? )")) { + try (PreparedStatement stmt = conn.prepareStatement("INSERT INTO test_strings VALUES ( 1, ?, ?, ?, ?, ?, ?, ?, ?, ? )")) { stmt.setString(1, str); stmt.setString(2, fixed); stmt.setString(3, enum8); @@ -345,6 +343,7 @@ public void testStringTypes() throws SQLException { stmt.setString(6, uuid); stmt.setString(7, ipv4); stmt.setString(8, ipv6); + stmt.setString(9, escaped); stmt.executeUpdate(); } } @@ -365,7 +364,7 @@ public void testStringTypes() throws SQLException { assertEquals(rs.getString("uuid"), uuid); assertEquals(rs.getString("ipv4"), "/" + ipv4); assertEquals(rs.getString("ipv6"), "/" + ipv6); - + assertEquals(rs.getString("escaped"), escaped); assertFalse(rs.next()); } } diff --git a/jdbc-v2/src/test/java/com/clickhouse/jdbc/StatementTest.java b/jdbc-v2/src/test/java/com/clickhouse/jdbc/StatementTest.java index 6ac7f5086..6c31a81a3 100644 --- a/jdbc-v2/src/test/java/com/clickhouse/jdbc/StatementTest.java +++ b/jdbc-v2/src/test/java/com/clickhouse/jdbc/StatementTest.java @@ -57,6 +57,7 @@ public void testExecuteQuerySimpleNumbers() throws Exception { assertEquals(rs.getLong("num"), 1); assertFalse(rs.next()); } + Assert.assertFalse(((StatementImpl)stmt).getLastQueryId().isEmpty()); } } } @@ -530,4 +531,12 @@ public void testConcurrentCancel() throws Exception { } } } + + @Test(groups = {"integration"}) + public void testTextFormatInResponse() throws Exception { + try (Connection conn = getJdbcConnection(); + Statement stmt = conn.createStatement()) { + Assert.expectThrows(SQLException.class, () ->stmt.executeQuery("SELECT 1 FORMAT JSON")); + } + } }