diff --git a/.github/workflows/analysis.yml b/.github/workflows/analysis.yml
index 85ad0ac9b..dc4341e55 100644
--- a/.github/workflows/analysis.yml
+++ b/.github/workflows/analysis.yml
@@ -61,6 +61,7 @@ jobs:
# added,diff_context,file,nofilter
filter_mode: 'added'
if: github.event_name == 'pull_request_target' || github.event.inputs.pr != ''
+ continue-on-error: true
- name: Update sonar config
run: |
sed -i -e 's|^\(.*\).*\(\)$|\1ClickHouse_clickhouse-jdbc\2|' \
@@ -73,3 +74,4 @@ jobs:
run: |
find . -type f -name "log4j.*" -exec rm -fv '{}' \;
mvn -q --batch-mode -Panalysis verify org.sonarsource.scanner.maven:sonar-maven-plugin:sonar
+ continue-on-error: true
diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml
index dff870629..32ff1dc46 100644
--- a/.github/workflows/build.yml
+++ b/.github/workflows/build.yml
@@ -34,7 +34,8 @@ jobs:
matrix:
java: [8, 11, 15]
# most recent LTS releases as well as latest stable builds
- clickhouse: ["20.8", "20.10", "20.12", "21.2", "latest"]
+ clickhouse: ["20.8", "21.3", "latest"]
+ fail-fast: false
name: Build using JDK ${{ matrix.java }} against ClickHouse ${{ matrix.clickhouse }}
steps:
- name: Check out Git repository
diff --git a/.github/workflows/timezone.yml b/.github/workflows/timezone.yml
index c7ceb2f97..78fd5f1f5 100644
--- a/.github/workflows/timezone.yml
+++ b/.github/workflows/timezone.yml
@@ -1,4 +1,4 @@
-name: TimeZone Test
+name: TimeZone
on:
push:
diff --git a/clickhouse-benchmark/pom.xml b/clickhouse-benchmark/pom.xml
index 97a0d63b5..c10861b59 100644
--- a/clickhouse-benchmark/pom.xml
+++ b/clickhouse-benchmark/pom.xml
@@ -18,10 +18,9 @@
1.4.4
- 2.5.3
2.7.2
8.0.23
- 2.5.3
+ 2.5.4
1.15.2
UTF-8
1.27
diff --git a/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/Basic.java b/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/Basic.java
index 2d397050d..079001636 100644
--- a/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/Basic.java
+++ b/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/Basic.java
@@ -4,7 +4,6 @@
import java.sql.Statement;
import java.util.Collections;
import java.util.Random;
-
import org.openjdk.jmh.annotations.Benchmark;
public class Basic extends JdbcBenchmark {
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 cc22324e7..62d44bf6b 100644
--- a/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/ClientState.java
+++ b/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/ClientState.java
@@ -4,7 +4,6 @@
import java.sql.SQLException;
import java.sql.Statement;
import java.util.Properties;
-
import org.openjdk.jmh.annotations.Level;
import org.openjdk.jmh.annotations.Param;
import org.openjdk.jmh.annotations.Scope;
diff --git a/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/Constants.java b/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/Constants.java
index e2873579b..27220a5ef 100644
--- a/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/Constants.java
+++ b/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/Constants.java
@@ -1,5 +1,8 @@
package tech.clickhouse.benchmark;
+/**
+ * Constant interface.
+ */
public interface Constants {
public static final String CLICKHOUSE_DRIVER = "clickhouse-jdbc";
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 24347d7c2..44eea7efe 100644
--- a/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/Insertion.java
+++ b/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/Insertion.java
@@ -4,7 +4,6 @@
// import java.util.Collections;
import java.util.Enumeration;
import java.util.Random;
-
import org.openjdk.jmh.annotations.Benchmark;
public class Insertion extends JdbcBenchmark {
diff --git a/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/JdbcBenchmark.java b/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/JdbcBenchmark.java
index 7faf1dce8..dac7ce228 100644
--- a/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/JdbcBenchmark.java
+++ b/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/JdbcBenchmark.java
@@ -1,7 +1,5 @@
package tech.clickhouse.benchmark;
-import org.openjdk.jmh.annotations.*;
-
import java.sql.Connection;
import java.sql.PreparedStatement;
import java.sql.SQLException;
@@ -9,7 +7,19 @@
import java.util.Enumeration;
import java.util.Objects;
import java.util.concurrent.TimeUnit;
-
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Threads;
+import org.openjdk.jmh.annotations.Warmup;
+
+/**
+ * Base class for JDBC driver benchmarking.
+ */
@State(Scope.Benchmark)
@Warmup(iterations = 10, timeUnit = TimeUnit.SECONDS, time = 1)
@Measurement(iterations = 10, timeUnit = TimeUnit.SECONDS, time = 1)
diff --git a/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/JdbcDriver.java b/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/JdbcDriver.java
index 5476eb9cd..4e8111a4e 100644
--- a/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/JdbcDriver.java
+++ b/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/JdbcDriver.java
@@ -16,12 +16,14 @@ public enum JdbcDriver {
// MariaDB Java Client
MariadbJavaClient("org.mariadb.jdbc.Driver",
- "jdbc:mariadb://%s:%s/%s?user=%s&password=%s&useSSL=false&useCompression=true&useServerPrepStmts=false&rewriteBatchedStatements=true&cachePrepStmts=true&serverTimezone=UTC",
+ "jdbc:mariadb://%s:%s/%s?user=%s&password=%s&useSSL=false&useCompression=true&useServerPrepStmts=false"
+ + "&rewriteBatchedStatements=true&cachePrepStmts=true&serverTimezone=UTC",
Constants.MYSQL_PORT),
// MySQL Connector/J
MysqlConnectorJava("com.mysql.cj.jdbc.Driver",
- "jdbc:mysql://%s:%s/%s?user=%s&password=%s&useSSL=false&useCompression=true&useServerPrepStmts=false&rewriteBatchedStatements=true&cachePrepStmts=true&connectionTimeZone=UTC",
+ "jdbc:mysql://%s:%s/%s?user=%s&password=%s&useSSL=false&useCompression=true&useServerPrepStmts=false"
+ + "&rewriteBatchedStatements=true&cachePrepStmts=true&connectionTimeZone=UTC",
Constants.MYSQL_PORT);
private final String className;
diff --git a/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/Query.java b/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/Query.java
index 650df25dc..020fa3087 100644
--- a/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/Query.java
+++ b/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/Query.java
@@ -4,7 +4,6 @@
import java.sql.Statement;
import java.sql.Timestamp;
import java.util.Random;
-
import org.openjdk.jmh.annotations.Benchmark;
public class Query extends JdbcBenchmark {
diff --git a/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/ServerState.java b/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/ServerState.java
index 71c2ab9c4..6e1436db9 100644
--- a/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/ServerState.java
+++ b/clickhouse-benchmark/src/main/java/tech/clickhouse/benchmark/ServerState.java
@@ -1,12 +1,13 @@
package tech.clickhouse.benchmark;
+import static java.time.temporal.ChronoUnit.SECONDS;
+
import java.net.Inet4Address;
import java.net.InetAddress;
import java.net.InterfaceAddress;
import java.net.NetworkInterface;
import java.time.Duration;
import java.util.Enumeration;
-
import org.openjdk.jmh.annotations.Level;
import org.openjdk.jmh.annotations.Scope;
import org.openjdk.jmh.annotations.Setup;
@@ -16,8 +17,6 @@
import org.testcontainers.containers.wait.strategy.Wait;
import org.testcontainers.images.builder.ImageFromDockerfile;
-import static java.time.temporal.ChronoUnit.SECONDS;
-
@State(Scope.Benchmark)
public class ServerState {
static String getLocalIpAddress() {
diff --git a/clickhouse-jdbc/pom.xml b/clickhouse-jdbc/pom.xml
index d7d0970b7..b57b275b0 100644
--- a/clickhouse-jdbc/pom.xml
+++ b/clickhouse-jdbc/pom.xml
@@ -72,6 +72,10 @@
com.fasterxml.jackson.core
jackson-databind
+
+ com.github.RoaringBitmap
+ RoaringBitmap
+
org.slf4j
slf4j-api
@@ -198,6 +202,10 @@
net.jpountz
${shade.base}.jpountz
+
+ org.roaringbitmap
+ ${shade.base}.bitmap
+
org.slf4j
${shade.base}.slf4j
diff --git a/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/response/ByteFragment.java b/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/response/ByteFragment.java
index a27728337..4ffcb2c6e 100644
--- a/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/response/ByteFragment.java
+++ b/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/response/ByteFragment.java
@@ -19,6 +19,7 @@ public ByteFragment(byte[] buf, int start, int len) {
}
public static ByteFragment fromString(String str) {
+ // https://bugs.openjdk.java.net/browse/JDK-6219899
byte[] bytes = str.getBytes(StandardCharsets.UTF_8);
return new ByteFragment(bytes, 0, bytes.length);
}
diff --git a/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/response/ClickHouseColumnInfo.java b/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/response/ClickHouseColumnInfo.java
index 95627cc97..240dd0d9a 100644
--- a/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/response/ClickHouseColumnInfo.java
+++ b/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/response/ClickHouseColumnInfo.java
@@ -1,9 +1,11 @@
package ru.yandex.clickhouse.response;
import java.util.TimeZone;
-
import ru.yandex.clickhouse.domain.ClickHouseDataType;
+/**
+ * This class represents a column defined in database.
+ */
public final class ClickHouseColumnInfo {
private static final String KEYWORD_NULLABLE = "Nullable";
@@ -22,12 +24,21 @@ public final class ClickHouseColumnInfo {
private int scale;
private ClickHouseColumnInfo keyInfo;
private ClickHouseColumnInfo valueInfo;
+ private String functionName;
@Deprecated
public static ClickHouseColumnInfo parse(String typeInfo, String columnName) {
return parse(typeInfo, columnName, null);
}
+ /**
+ * Parse given type string.
+ *
+ * @param typeInfo type defined in database
+ * @param columnName column name
+ * @param serverTimeZone server time zone
+ * @return parsed type
+ */
public static ClickHouseColumnInfo parse(String typeInfo, String columnName, TimeZone serverTimeZone) {
ClickHouseColumnInfo column = new ClickHouseColumnInfo(typeInfo, columnName);
int currIdx = 0;
@@ -61,68 +72,74 @@ public static ClickHouseColumnInfo parse(String typeInfo, String columnName, Tim
column.scale = dataType.getDefaultScale();
column.timeZone = serverTimeZone;
currIdx = endIdx;
- if (endIdx == typeInfo.length()
- || !typeInfo.startsWith("(", currIdx))
- {
+ if (endIdx == typeInfo.length() || !typeInfo.startsWith("(", currIdx)) {
return column;
}
switch (dataType) {
- case DateTime :
- String[] argsDT = splitArgs(typeInfo, currIdx);
- if (argsDT.length == 2) { // same as DateTime64
- column.scale = Integer.parseInt(argsDT[0]);
- column.timeZone = TimeZone.getTimeZone(argsDT[1].replace("'", ""));
- } else if (argsDT.length == 1) { // same as DateTime32
- // unfortunately this will fall back to GMT if the time zone
- // cannot be resolved
- TimeZone tz = TimeZone.getTimeZone(argsDT[0].replace("'", ""));
- column.timeZone = tz;
- }
- break;
- case DateTime32:
- String[] argsD32 = splitArgs(typeInfo, currIdx);
- if (argsD32.length == 1) {
- // unfortunately this will fall back to GMT if the time zone
- // cannot be resolved
- TimeZone tz = TimeZone.getTimeZone(argsD32[0].replace("'", ""));
- column.timeZone = tz;
- }
- break;
- case DateTime64:
- String[] argsD64 = splitArgs(typeInfo, currIdx);
- if (argsD64.length == 2) {
- column.scale = Integer.parseInt(argsD64[0]);
- column.timeZone = TimeZone.getTimeZone(argsD64[1].replace("'", ""));
- }
- break;
- case Decimal :
- String[] argsDecimal = splitArgs(typeInfo, currIdx);
- if (argsDecimal.length == 2) {
- column.precision = Integer.parseInt(argsDecimal[0]);
- column.scale = Integer.parseInt(argsDecimal[1]);
- }
- break;
- case Decimal32 :
- case Decimal64 :
- case Decimal128 :
- case Decimal256 :
- String[] argsScale = splitArgs(typeInfo, currIdx);
- column.scale = Integer.parseInt(argsScale[0]);
- break;
- case FixedString :
- String[] argsPrecision = splitArgs(typeInfo, currIdx);
- column.precision = Integer.parseInt(argsPrecision[0]);
- break;
- case Map:
- String[] argsMap = splitArgs(typeInfo, currIdx);
- if (argsMap.length == 2) {
- column.keyInfo = ClickHouseColumnInfo.parse(argsMap[0], columnName + "Key", serverTimeZone);
- column.valueInfo = ClickHouseColumnInfo.parse(argsMap[1], columnName + "Value", serverTimeZone);
- }
- break;
- default :
- break;
+ case AggregateFunction :
+ String[] argsAf = splitArgs(typeInfo, currIdx);
+ column.functionName = argsAf[0];
+ column.arrayBaseType = ClickHouseDataType.Unknown;
+ if (argsAf.length == 2) {
+ column.arrayBaseType = ClickHouseDataType.fromTypeString(argsAf[1]);
+ }
+ break;
+ case DateTime :
+ String[] argsDt = splitArgs(typeInfo, currIdx);
+ if (argsDt.length == 2) { // same as DateTime64
+ column.scale = Integer.parseInt(argsDt[0]);
+ column.timeZone = TimeZone.getTimeZone(argsDt[1].replace("'", ""));
+ } else if (argsDt.length == 1) { // same as DateTime32
+ // unfortunately this will fall back to GMT if the time zone
+ // cannot be resolved
+ TimeZone tz = TimeZone.getTimeZone(argsDt[0].replace("'", ""));
+ column.timeZone = tz;
+ }
+ break;
+ case DateTime32:
+ String[] argsD32 = splitArgs(typeInfo, currIdx);
+ if (argsD32.length == 1) {
+ // unfortunately this will fall back to GMT if the time zone
+ // cannot be resolved
+ TimeZone tz = TimeZone.getTimeZone(argsD32[0].replace("'", ""));
+ column.timeZone = tz;
+ }
+ break;
+ case DateTime64:
+ String[] argsD64 = splitArgs(typeInfo, currIdx);
+ if (argsD64.length == 2) {
+ column.scale = Integer.parseInt(argsD64[0]);
+ column.timeZone = TimeZone.getTimeZone(argsD64[1].replace("'", ""));
+ }
+ break;
+ case Decimal :
+ String[] argsDecimal = splitArgs(typeInfo, currIdx);
+ if (argsDecimal.length == 2) {
+ column.precision = Integer.parseInt(argsDecimal[0]);
+ column.scale = Integer.parseInt(argsDecimal[1]);
+ }
+ break;
+ case Decimal32 :
+ case Decimal64 :
+ case Decimal128 :
+ case Decimal256 :
+ String[] argsScale = splitArgs(typeInfo, currIdx);
+ column.scale = Integer.parseInt(argsScale[0]);
+ break;
+ case FixedString :
+ String[] argsPrecision = splitArgs(typeInfo, currIdx);
+ column.precision = Integer.parseInt(argsPrecision[0]);
+ break;
+ case Map:
+ String[] argsMap = splitArgs(typeInfo, currIdx);
+ if (argsMap.length == 2) {
+ column.keyInfo = ClickHouseColumnInfo.parse(argsMap[0], columnName + "Key", serverTimeZone);
+ column.valueInfo = ClickHouseColumnInfo.parse(argsMap[1], columnName + "Value", serverTimeZone);
+ }
+ break;
+ default:
+ break;
}
return column;
@@ -153,8 +170,9 @@ public String getOriginalTypeName() {
}
/**
- * @return the type name returned from the database, without modifiers, i.e.
- * Nullable or LowCardinality
+ * Get the type name returned from the database, without modifiers, i.e. Nullable or LowCardinality.
+ *
+ * @return the type name returned from the database
*/
public String getCleanTypeName() {
if (!nullable && !lowCardinality) {
@@ -226,4 +244,8 @@ public ClickHouseColumnInfo getKeyInfo() {
public ClickHouseColumnInfo getValueInfo() {
return this.valueInfo;
}
+
+ public String getFunctionName() {
+ return this.functionName;
+ }
}
diff --git a/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/response/ClickHouseResultSet.java b/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/response/ClickHouseResultSet.java
index 89ec7f9ec..944d06be6 100644
--- a/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/response/ClickHouseResultSet.java
+++ b/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/response/ClickHouseResultSet.java
@@ -20,7 +20,6 @@
import java.util.Collections;
import java.util.List;
import java.util.TimeZone;
-
import ru.yandex.clickhouse.ClickHouseArray;
import ru.yandex.clickhouse.ClickHouseConnection;
import ru.yandex.clickhouse.ClickHouseStatement;
@@ -30,13 +29,14 @@
import ru.yandex.clickhouse.response.parser.ClickHouseValueParser;
import ru.yandex.clickhouse.settings.ClickHouseProperties;
import ru.yandex.clickhouse.util.ClickHouseArrayUtil;
+import ru.yandex.clickhouse.util.ClickHouseBitmap;
import ru.yandex.clickhouse.util.ClickHouseValueFormatter;
import ru.yandex.clickhouse.util.Utils;
public class ClickHouseResultSet extends AbstractResultSet {
- private final static long[] EMPTY_LONG_ARRAY = new long[0];
+ private static final long[] EMPTY_LONG_ARRAY = new long[0];
private final TimeZone dateTimeTimeZone;
private final TimeZone dateTimeZone;
@@ -358,25 +358,25 @@ public Array getArray(int columnIndex) throws SQLException {
final Object array;
switch (colInfo.getArrayBaseType()) {
- case Date :
- array = ClickHouseArrayUtil.parseArray(
- getValue(columnIndex),
- properties.isUseObjectsInArrays(),
- dateTimeZone,
- colInfo
- );
- break;
- default :
- TimeZone timeZone = colInfo.getTimeZone() != null
- ? colInfo.getTimeZone()
- : dateTimeTimeZone;
- array = ClickHouseArrayUtil.parseArray(
- getValue(columnIndex),
- properties.isUseObjectsInArrays(),
- timeZone,
- colInfo
- );
- break;
+ case Date :
+ array = ClickHouseArrayUtil.parseArray(
+ getValue(columnIndex),
+ properties.isUseObjectsInArrays(),
+ dateTimeZone,
+ colInfo
+ );
+ break;
+ default :
+ TimeZone timeZone = colInfo.getTimeZone() != null
+ ? colInfo.getTimeZone()
+ : dateTimeTimeZone;
+ array = ClickHouseArrayUtil.parseArray(
+ getValue(columnIndex),
+ properties.isUseObjectsInArrays(),
+ timeZone,
+ colInfo
+ );
+ break;
}
return new ClickHouseArray(colInfo.getArrayBaseType(), array);
}
@@ -428,14 +428,15 @@ public String getString(int colNum) throws SQLException {
|| dataType == ClickHouseDataType.DateTime64)) {
TimeZone serverTimeZone = columnInfo.getTimeZone();
if (serverTimeZone == null) {
- serverTimeZone = ((ClickHouseConnection)getStatement().getConnection()).getServerTimeZone();
+ serverTimeZone = ((ClickHouseConnection) getStatement().getConnection()).getServerTimeZone();
}
TimeZone clientTimeZone = Utils.isNullOrEmptyString(properties.getUseTimeZone())
? TimeZone.getDefault()
: TimeZone.getTimeZone(properties.getUseTimeZone());
if (!clientTimeZone.equals(serverTimeZone)) {
- Timestamp newTs = ClickHouseValueParser.getParser(Timestamp.class).parse(value, columnInfo, serverTimeZone);
+ Timestamp newTs = ClickHouseValueParser.getParser(Timestamp.class)
+ .parse(value, columnInfo, serverTimeZone);
value = ByteFragment.fromString(ClickHouseValueFormatter.formatTimestamp(newTs, clientTimeZone));
}
}
@@ -468,7 +469,7 @@ public byte[] getBytes(int colNum) {
/**
* Tries to parse the value as a timestamp using the connection time zone if
- * applicable and return its representation as milliseconds since epoch
+ * applicable and return its representation as milliseconds since epoch.
*
* @param colNum
* column number
@@ -586,42 +587,63 @@ public Object getObject(int columnIndex) throws SQLException {
if (getValue(columnIndex).isNull()) {
return null;
}
- ClickHouseDataType chType = getColumnInfo(columnIndex).getClickHouseDataType();
+ ClickHouseColumnInfo columnInfo = getColumnInfo(columnIndex);
+ ClickHouseDataType chType = columnInfo.getClickHouseDataType();
switch (chType.getSqlType()) {
- case Types.BIGINT:
- if (chType == ClickHouseDataType.UInt64) {
- return getObject(columnIndex, BigInteger.class);
- }
+ case Types.BIGINT:
+ if (chType == ClickHouseDataType.UInt64) {
+ return getObject(columnIndex, BigInteger.class);
+ }
+ return getObject(columnIndex, Long.class);
+ case Types.INTEGER:
+ if (!chType.isSigned()){
return getObject(columnIndex, Long.class);
- case Types.INTEGER:
- if (!chType.isSigned()){
- return getObject(columnIndex, Long.class);
- }
- return getObject(columnIndex, Integer.class);
- case Types.TINYINT:
- case Types.SMALLINT:
- return getObject(columnIndex, Integer.class);
- case Types.VARCHAR: return getString(columnIndex);
- case Types.REAL: return getObject(columnIndex, Float.class);
- case Types.FLOAT:
- case Types.DOUBLE: return getObject(columnIndex, Double.class);
- case Types.DATE: return getDate(columnIndex);
- case Types.TIMESTAMP: return getTimestamp(columnIndex);
- case Types.BLOB: return getString(columnIndex);
- case Types.ARRAY: return getArray(columnIndex);
- case Types.DECIMAL: return getBigDecimal(columnIndex);
- case Types.NUMERIC: return getBigInteger(columnIndex);
- default:
- // do not return
+ }
+ return getObject(columnIndex, Integer.class);
+ case Types.TINYINT:
+ case Types.SMALLINT:
+ return getObject(columnIndex, Integer.class);
+ case Types.VARCHAR: return getString(columnIndex);
+ case Types.REAL: return getObject(columnIndex, Float.class);
+ case Types.FLOAT:
+ case Types.DOUBLE: return getObject(columnIndex, Double.class);
+ case Types.DATE: return getDate(columnIndex);
+ case Types.TIMESTAMP: return getTimestamp(columnIndex);
+ case Types.BLOB: return getString(columnIndex);
+ case Types.ARRAY: return getArray(columnIndex);
+ case Types.DECIMAL: return getBigDecimal(columnIndex);
+ case Types.NUMERIC: return getBigInteger(columnIndex);
+ default:
+ // do not return
}
switch (chType) {
- // case Array:
- // case Tuple:
- case Map:
- case UUID :
- return getObject(columnIndex, chType.getJavaClass());
- default :
- return getString(columnIndex);
+ // case Array:
+ // case Tuple:
+ case AggregateFunction:
+ // TODO support more functions
+ if ("groupBitmap".equals(columnInfo.getFunctionName())) {
+ ClickHouseDataType innerType = columnInfo.getArrayBaseType();
+ switch (innerType) {
+ // seems signed integers are not supported in ClickHouse
+ case Int8:
+ case Int16:
+ case Int32:
+ case Int64:
+ case UInt8:
+ case UInt16:
+ case UInt32:
+ case UInt64:
+ return getObject(columnIndex, ClickHouseBitmap.class);
+ default:
+ break;
+ }
+ }
+ return getString(columnIndex);
+ case Map:
+ case UUID :
+ return getObject(columnIndex, chType.getJavaClass());
+ default :
+ return getString(columnIndex);
}
} catch (Exception e) {
throw new ClickHouseUnknownException(
@@ -646,14 +668,12 @@ private static byte[] toBytes(ByteFragment value) {
return value.unescape();
}
- static long[] toLongArray(ByteFragment value, ClickHouseColumnInfo columnInfo)
- throws SQLException
- {
+ static long[] toLongArray(ByteFragment value, ClickHouseColumnInfo columnInfo) throws SQLException {
if (value.isNull()) {
return null;
}
- if (value.charAt(0) != '[' || value.charAt(value.length()-1) != ']') {
- throw new IllegalArgumentException("not an array: "+value);
+ if (value.charAt(0) != '[' || value.charAt(value.length() - 1) != ']') {
+ throw new IllegalArgumentException("not an array: " + value);
}
if (value.length() == 2) {
return EMPTY_LONG_ARRAY;
diff --git a/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/response/StreamSplitter.java b/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/response/StreamSplitter.java
index fed8737ad..b6d857f49 100644
--- a/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/response/StreamSplitter.java
+++ b/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/response/StreamSplitter.java
@@ -9,6 +9,7 @@
* We split the stream by the separator and pass the byte arrays to output.
*/
public class StreamSplitter {
+ private static final int MAX_ARRAY_LENGTH = Integer.MAX_VALUE - 8;
private static final int buflen = 65536;
// initial parameters
@@ -28,7 +29,6 @@ public class StreamSplitter {
private boolean closed;
-
public StreamSplitter(ByteFragment bf, byte sep) {
this.delegate = bf.asStream();
this.sep = sep;
@@ -43,7 +43,7 @@ public StreamSplitter(InputStream delegate, byte sep, int buflen) {
}
public StreamSplitter(InputStream delegate, byte sep) {
- this(delegate,sep, buflen);
+ this(delegate, sep, buflen);
}
public ByteFragment next() throws IOException {
@@ -51,17 +51,17 @@ public ByteFragment next() throws IOException {
if (posNext >= posRead) {
// need to read more from the stream
int readBytes = readFromStream();
- if(readBytes <= 0) {
+ if (readBytes <= 0) {
// if everything was sent out and there is nothing left in the stream
return null;
}
}
// looking for the separator
int positionSep;
- while((positionSep = indexOf(buf, sep, posNext, posRead)) < posNext) {
+ while ((positionSep = indexOf(buf, sep, posNext, posRead)) < posNext) {
// read from stream till we find the separator
int readBytes = readFromStream();
- if(readBytes <= 0) {
+ if (readBytes <= 0) {
// if there is nothing to read, return everything left as a result
positionSep = posRead;
break;
@@ -69,8 +69,8 @@ public ByteFragment next() throws IOException {
}
// if the separator is found, return the fragment
int fragmentStart = posNext;
- posNext = positionSep+1;
- return new ByteFragment(buf, fragmentStart, positionSep-fragmentStart);
+ posNext = positionSep + 1;
+ return new ByteFragment(buf, fragmentStart, positionSep - fragmentStart);
}
// if there is no separator in read but not sent fragment - read more data
@@ -80,8 +80,9 @@ protected int readFromStream() throws IOException {
return -1;
} else {
int read = delegate.read(buf, posRead, buf.length - posRead);
- if(read > 0)
+ if (read > 0) {
posRead += read;
+ }
return read;
}
} else {
@@ -89,37 +90,43 @@ protected int readFromStream() throws IOException {
shiftOrResize();
}
int read = delegate.read(buf, posRead, buf.length - posRead);
- if(read > 0)
+ if (read > 0) {
posRead += read;
+ }
return read;
}
}
- // if we have read till the end of buffer, we have to create a new buffer and move data by posNext (already send data position)
+ // if we have read till the end of buffer, we have to create a new buffer
+ // and move data by posNext (already send data position)
// if there is no sent data and buffer is still full - expand the buffer
private void shiftOrResize() {
- if(posNext > 0) {
+ if (posNext > 0) {
byte[] oldBuf = buf;
buf = new byte[buf.length];
- System.arraycopy(oldBuf, posNext, buf, 0, oldBuf.length-posNext);
+ System.arraycopy(oldBuf, posNext, buf, 0, oldBuf.length - posNext);
posRead -= posNext;
posNext = 0;
} else {
byte[] oldBuf = buf;
- buf = new byte[buf.length*2];
+ int len = buf.length * 2;
+ if (len > MAX_ARRAY_LENGTH) {
+ len = MAX_ARRAY_LENGTH;
+ }
+ buf = new byte[len];
System.arraycopy(oldBuf, 0, buf, 0, oldBuf.length);
}
}
private static int indexOf(byte[] array, byte target, int start, int end) {
- for (int i = start; i < end; i++) {
- if (array[i] == target) {
- return i;
- }
- }
- return -1;
- }
+ for (int i = start; i < end; i++) {
+ if (array[i] == target) {
+ return i;
+ }
+ }
+ return -1;
+ }
public void close() throws IOException {
closed = true;
@@ -134,23 +141,23 @@ public boolean isClosed() throws IOException {
public String toString() {
String bufStr = new String(buf, StandardCharsets.UTF_8).trim();
- return "StreamSplitter{" +
- "delegate=" + delegate +
- ", sep=" + sep +
- ", buf=" + bufStr +
- ", posRead=" + posRead +
- ", posNext=" + posNext +
- ", readOnce=" + readOnce +
- '}';
+ return "StreamSplitter{"
+ + "delegate=" + delegate
+ + ", sep=" + sep
+ + ", buf=" + bufStr
+ + ", posRead=" + posRead
+ + ", posNext=" + posNext
+ + ", readOnce=" + readOnce
+ + '}';
}
- public void mark() {
+ public void mark() {
markedRead = posRead;
markedNext = posNext;
- }
+ }
- public void reset() {
+ public void reset() {
posRead = markedRead;
posNext = markedNext;
- }
+ }
}
diff --git a/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/response/parser/ClickHouseBitmapParser.java b/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/response/parser/ClickHouseBitmapParser.java
new file mode 100644
index 000000000..a58f40ed3
--- /dev/null
+++ b/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/response/parser/ClickHouseBitmapParser.java
@@ -0,0 +1,53 @@
+package ru.yandex.clickhouse.response.parser;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.TimeZone;
+import ru.yandex.clickhouse.response.ByteFragment;
+import ru.yandex.clickhouse.response.ClickHouseColumnInfo;
+import ru.yandex.clickhouse.util.ClickHouseBitmap;
+
+final class ClickHouseBitmapParser extends ClickHouseValueParser {
+ private static ClickHouseBitmapParser instance;
+
+ static ClickHouseBitmapParser getInstance() {
+ if (instance == null) {
+ instance = new ClickHouseBitmapParser();
+ }
+ return instance;
+ }
+
+ private ClickHouseBitmapParser() {
+ // prevent instantiation
+ }
+
+ @Override
+ public ClickHouseBitmap parse(ByteFragment value, ClickHouseColumnInfo columnInfo, TimeZone resultTimeZone)
+ throws SQLException {
+ if (value.isNull()) {
+ return null;
+ }
+
+ // https://github.com/ClickHouse/ClickHouse/blob/master/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h#L100
+ ClickHouseBitmap rb = ClickHouseBitmap.wrap();
+
+ // FIXME use DataInput/DataOutput for stream after switching to RowBinary
+ byte[] bytes = value.unescape();
+ if (bytes.length == 0) {
+ return rb;
+ }
+
+ try {
+ rb = ClickHouseBitmap.deserialize(bytes, columnInfo.getArrayBaseType());
+ } catch (IOException e) {
+ throw new SQLException("Failed to deserialize ClickHouseBitmap", e);
+ }
+
+ return rb;
+ }
+
+ @Override
+ protected ClickHouseBitmap getDefaultValue() {
+ return ClickHouseBitmap.wrap();
+ }
+}
diff --git a/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/response/parser/ClickHouseDateValueParser.java b/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/response/parser/ClickHouseDateValueParser.java
index d05645b8f..6f46b39b1 100644
--- a/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/response/parser/ClickHouseDateValueParser.java
+++ b/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/response/parser/ClickHouseDateValueParser.java
@@ -14,15 +14,12 @@
import java.util.Objects;
import java.util.TimeZone;
import java.util.regex.Pattern;
-
import ru.yandex.clickhouse.except.ClickHouseException;
import ru.yandex.clickhouse.except.ClickHouseUnknownException;
import ru.yandex.clickhouse.response.ByteFragment;
import ru.yandex.clickhouse.response.ClickHouseColumnInfo;
abstract class ClickHouseDateValueParser extends ClickHouseValueParser {
- private static final ZoneId UTC_ZONE = ZoneId.of("UTC");
-
private static final Pattern PATTERN_EMPTY_DATE =
Pattern.compile("^(0000-00-00|0000-00-00 00:00:00|0)$");
@@ -47,7 +44,8 @@ protected LocalDateTime dateToLocalDate(String value, ClickHouseColumnInfo colum
protected LocalDateTime dateTimeToLocalDateTime(String value, ClickHouseColumnInfo columnInfo, TimeZone timeZone) {
TimeZone serverTimeZone = columnInfo.getTimeZone();
LocalDateTime localDateTime = parseAsLocalDateTime(value);
- if (serverTimeZone != null && (serverTimeZone.useDaylightTime() || serverTimeZone.getRawOffset() > 0)) { // non-UTC
+ if (serverTimeZone != null
+ && (serverTimeZone.useDaylightTime() || serverTimeZone.getRawOffset() > 0)) { // non-UTC
localDateTime = localDateTime.atZone(columnInfo.getTimeZone().toZoneId())
.withZoneSameInstant(java.time.ZoneId.of("UTC")).toLocalDateTime();
}
@@ -68,9 +66,7 @@ protected ZonedDateTime dateTimeToZonedDateTime(String value, ClickHouseColumnIn
}
@Override
- public T parse(ByteFragment value, ClickHouseColumnInfo columnInfo,
- TimeZone timeZone) throws ClickHouseException
- {
+ public T parse(ByteFragment value, ClickHouseColumnInfo columnInfo, TimeZone timeZone) throws ClickHouseException {
if (value.isNull()) {
return null;
@@ -87,77 +83,77 @@ public T parse(ByteFragment value, ClickHouseColumnInfo columnInfo,
}
switch (columnInfo.getEffectiveClickHouseDataType()) {
- case Date:
- try {
- return parseDate(s, columnInfo, timeZone);
- } catch (Exception e) {
- throw new ClickHouseUnknownException(
- "Error parsing '" + s + "' of data type '"
- + columnInfo.getOriginalTypeName()
- + "' as " + clazz.getName(),
- e);
- }
- case DateTime:
- case DateTime32:
- case DateTime64:
- try {
- return parseDateTime(s, columnInfo, timeZone);
- } catch (Exception e) {
- throw new ClickHouseUnknownException(
- "Error parsing '" + s + "' of data type '"
- + columnInfo.getOriginalTypeName()
- + "' as " + clazz.getName(),
- e);
- }
- case Int8:
- case Int16:
- case Int32:
- case Int64:
- case UInt8:
- case UInt16:
- case UInt32:
- try {
- long l = Long.parseLong(s);
- return parseNumber(l, columnInfo, timeZone);
- } catch (Exception e) {
- throw new ClickHouseUnknownException(
- "Error parsing '" + s + "' of data type '"
- + columnInfo.getOriginalTypeName()
- + "' as " + clazz.getName(),
- e);
- }
- case UInt64:
- // If we have a large nanos value, we trim to millis
- try {
- BigInteger bi = new BigInteger(s);
- if (bi.compareTo(BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
- return parseNumber(
- bi.divide(BigInteger.valueOf(1000_000L)).longValue(),
- columnInfo,
- timeZone);
- }
- return parseNumber(bi.longValue(), columnInfo, timeZone);
- } catch (Exception e) {
- throw new ClickHouseUnknownException(
- "Error parsing '" + s + "' of data type '"
- + columnInfo.getOriginalTypeName()
- + "' as " + clazz.getName(),
- e);
- }
- case String:
- case Unknown:
- try {
- return parseOther(s, columnInfo, timeZone);
- } catch (Exception e) {
- throw new ClickHouseUnknownException(
- "Error parsing '" + s + "' as " + clazz.getName(), e);
+ case Date:
+ try {
+ return parseDate(s, columnInfo, timeZone);
+ } catch (Exception e) {
+ throw new ClickHouseUnknownException(
+ "Error parsing '" + s + "' of data type '"
+ + columnInfo.getOriginalTypeName()
+ + "' as " + clazz.getName(),
+ e);
+ }
+ case DateTime:
+ case DateTime32:
+ case DateTime64:
+ try {
+ return parseDateTime(s, columnInfo, timeZone);
+ } catch (Exception e) {
+ throw new ClickHouseUnknownException(
+ "Error parsing '" + s + "' of data type '"
+ + columnInfo.getOriginalTypeName()
+ + "' as " + clazz.getName(),
+ e);
+ }
+ case Int8:
+ case Int16:
+ case Int32:
+ case Int64:
+ case UInt8:
+ case UInt16:
+ case UInt32:
+ try {
+ long l = Long.parseLong(s);
+ return parseNumber(l, columnInfo, timeZone);
+ } catch (Exception e) {
+ throw new ClickHouseUnknownException(
+ "Error parsing '" + s + "' of data type '"
+ + columnInfo.getOriginalTypeName()
+ + "' as " + clazz.getName(),
+ e);
+ }
+ case UInt64:
+ // If we have a large nanos value, we trim to millis
+ try {
+ BigInteger bi = new BigInteger(s);
+ if (bi.compareTo(BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
+ return parseNumber(
+ bi.divide(BigInteger.valueOf(1000_000L)).longValue(),
+ columnInfo,
+ timeZone);
}
- default:
+ return parseNumber(bi.longValue(), columnInfo, timeZone);
+ } catch (Exception e) {
throw new ClickHouseUnknownException(
"Error parsing '" + s + "' of data type '"
+ columnInfo.getOriginalTypeName()
+ "' as " + clazz.getName(),
- null);
+ e);
+ }
+ case String:
+ case Unknown:
+ try {
+ return parseOther(s, columnInfo, timeZone);
+ } catch (Exception e) {
+ throw new ClickHouseUnknownException(
+ "Error parsing '" + s + "' as " + clazz.getName(), e);
+ }
+ default:
+ throw new ClickHouseUnknownException(
+ "Error parsing '" + s + "' of data type '"
+ + columnInfo.getOriginalTypeName()
+ + "' as " + clazz.getName(),
+ null);
}
}
@@ -175,8 +171,7 @@ abstract T parseOther(String value, ClickHouseColumnInfo columnInfo,
TimeZone timeZone);
protected final ZoneId effectiveTimeZone(ClickHouseColumnInfo columnInfo,
- TimeZone timeZone)
- {
+ TimeZone timeZone) {
return timeZone != null
? timeZone.toZoneId()
: columnInfo.getTimeZone() != null
@@ -242,5 +237,4 @@ protected final LocalTime parseAsLocalTime(String value) {
protected final LocalTime parseAsLocalTime(long value) {
return parseAsLocalTime(String.valueOf(value));
}
-
}
diff --git a/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/response/parser/ClickHouseValueParser.java b/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/response/parser/ClickHouseValueParser.java
index 391f73a37..03ad23a53 100644
--- a/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/response/parser/ClickHouseValueParser.java
+++ b/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/response/parser/ClickHouseValueParser.java
@@ -25,6 +25,7 @@
import ru.yandex.clickhouse.except.ClickHouseUnknownException;
import ru.yandex.clickhouse.response.ByteFragment;
import ru.yandex.clickhouse.response.ClickHouseColumnInfo;
+import ru.yandex.clickhouse.util.ClickHouseBitmap;
public abstract class ClickHouseValueParser {
@@ -50,6 +51,7 @@ public abstract class ClickHouseValueParser {
register(LocalDateTime.class, ClickHouseLocalDateTimeParser.getInstance());
register(LocalTime.class, ClickHouseLocalTimeParser.getInstance());
register(Long.class, Long::decode, Long.valueOf(0L));
+ register(ClickHouseBitmap.class, ClickHouseBitmapParser.getInstance());
register(Map.class, ClickHouseMapParser.getInstance());
register(Object.class, s -> s);
register(OffsetDateTime.class, ClickHouseOffsetDateTimeParser.getInstance());
diff --git a/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/util/ClickHouseBitmap.java b/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/util/ClickHouseBitmap.java
new file mode 100644
index 000000000..155554c25
--- /dev/null
+++ b/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/util/ClickHouseBitmap.java
@@ -0,0 +1,531 @@
+package ru.yandex.clickhouse.util;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.Buffer;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Objects;
+
+import org.roaringbitmap.RoaringBitmap;
+import org.roaringbitmap.buffer.ImmutableRoaringBitmap;
+import org.roaringbitmap.buffer.MutableRoaringBitmap;
+import org.roaringbitmap.longlong.Roaring64Bitmap;
+import org.roaringbitmap.longlong.Roaring64NavigableMap;
+
+import ru.yandex.clickhouse.domain.ClickHouseDataType;
+
+public abstract class ClickHouseBitmap {
+ static class ClickHouseRoaringBitmap extends ClickHouseBitmap {
+ private final RoaringBitmap rb;
+
+ protected ClickHouseRoaringBitmap(RoaringBitmap bitmap, ClickHouseDataType innerType) {
+ super(bitmap, innerType);
+
+ this.rb = Objects.requireNonNull(bitmap);
+ }
+
+ @Override
+ public int getCardinality() {
+ return rb.getCardinality();
+ }
+
+ @Override
+ public void serialize(ByteBuffer buffer) {
+ rb.serialize(buffer);
+ }
+
+ @Override
+ public int serializedSizeInBytes() {
+ return rb.serializedSizeInBytes();
+ }
+
+ @Override
+ public int[] toIntArray() {
+ return rb.toArray();
+ }
+ }
+
+ static class ClickHouseImmutableRoaringBitmap extends ClickHouseBitmap {
+ private final ImmutableRoaringBitmap rb;
+
+ protected ClickHouseImmutableRoaringBitmap(ImmutableRoaringBitmap rb, ClickHouseDataType innerType) {
+ super(rb, innerType);
+
+ this.rb = Objects.requireNonNull(rb);
+ }
+
+ @Override
+ public int getCardinality() {
+ return rb.getCardinality();
+ }
+
+ @Override
+ public void serialize(ByteBuffer buffer) {
+ rb.serialize(buffer);
+ }
+
+ @Override
+ public int serializedSizeInBytes() {
+ return rb.serializedSizeInBytes();
+ }
+
+ @Override
+ public int[] toIntArray() {
+ return rb.toArray();
+ }
+ }
+
+ static class ClickHouseMutableRoaringBitmap extends ClickHouseBitmap {
+ private final MutableRoaringBitmap rb;
+
+ protected ClickHouseMutableRoaringBitmap(MutableRoaringBitmap bitmap, ClickHouseDataType innerType) {
+ super(bitmap, innerType);
+
+ this.rb = Objects.requireNonNull(bitmap);
+ }
+
+ @Override
+ public int getCardinality() {
+ return rb.getCardinality();
+ }
+
+ @Override
+ public void serialize(ByteBuffer buffer) {
+ rb.serialize(buffer);
+ }
+
+ @Override
+ public int serializedSizeInBytes() {
+ return rb.serializedSizeInBytes();
+ }
+
+ @Override
+ public int[] toIntArray() {
+ return rb.toArray();
+ }
+ }
+
+ static class ClickHouseRoaring64NavigableMap extends ClickHouseBitmap {
+ private final Roaring64NavigableMap rb;
+
+ protected ClickHouseRoaring64NavigableMap(Roaring64NavigableMap bitmap, ClickHouseDataType innerType) {
+ super(bitmap, innerType);
+
+ this.rb = Objects.requireNonNull(bitmap);
+ }
+
+ @Override
+ public int getCardinality() {
+ return rb.getIntCardinality();
+ }
+
+ @Override
+ public long getLongCardinality() {
+ return rb.getLongCardinality();
+ }
+
+ @Override
+ public void serialize(ByteBuffer buffer) {
+ int size = serializedSizeInBytes();
+ try (ByteArrayOutputStream bas = new ByteArrayOutputStream(size)) {
+ DataOutput out = new DataOutputStream(bas);
+ try {
+ rb.serialize(out);
+ } catch (IOException e) {
+ throw new IllegalArgumentException("Failed to serialize given bitmap", e);
+ }
+ buffer.put(bas.toByteArray(), 5, size - 5);
+ } catch (IOException e) {
+ throw new IllegalArgumentException("Failed to serialize given bitmap", e);
+ }
+ }
+
+ @Override
+ public int serializedSizeInBytes() {
+ return (int) rb.serializedSizeInBytes();
+ }
+
+ @Override
+ public long serializedSizeInBytesAsLong() {
+ return rb.serializedSizeInBytes();
+ }
+
+ @Override
+ public int[] toIntArray() {
+ long[] longs = toLongArray();
+ int len = longs.length;
+ int[] ints = new int[len];
+ for (int i = 0; i < len; i++) {
+ ints[i] = (int) longs[i];
+ }
+ return ints;
+ }
+
+ @Override
+ public long[] toLongArray() {
+ return rb.toArray();
+ }
+ }
+
+ public static ClickHouseBitmap wrap(byte... values) {
+ boolean isUnsigned = true;
+ int len = values.length;
+ int[] ints = new int[len];
+ for (int i = 0; i < len; i++) {
+ byte v = values[i];
+ ints[i] = v;
+ if (isUnsigned && v < 0) {
+ isUnsigned = false;
+ }
+ }
+
+ return wrap(RoaringBitmap.bitmapOf(ints), isUnsigned ? ClickHouseDataType.UInt8 : ClickHouseDataType.Int8);
+ }
+
+ public static ClickHouseBitmap wrap(short... values) {
+ boolean isUnsigned = true;
+ int len = values.length;
+ int[] ints = new int[len];
+ for (int i = 0; i < len; i++) {
+ short v = values[i];
+ ints[i] = v;
+ if (isUnsigned && v < 0) {
+ isUnsigned = false;
+ }
+ }
+
+ return wrap(RoaringBitmap.bitmapOf(ints), isUnsigned ? ClickHouseDataType.UInt16 : ClickHouseDataType.Int16);
+ }
+
+ public static ClickHouseBitmap wrap(int... values) {
+ boolean isUnsigned = true;
+ int len = values.length;
+ int[] ints = new int[len];
+ for (int i = 0; i < len; i++) {
+ int v = values[i];
+ ints[i] = v;
+ if (isUnsigned && v < 0) {
+ isUnsigned = false;
+ }
+ }
+
+ return wrap(RoaringBitmap.bitmapOf(ints), isUnsigned ? ClickHouseDataType.UInt32 : ClickHouseDataType.Int32);
+ }
+
+ public static ClickHouseBitmap wrap(long... values) {
+ boolean isUnsigned = true;
+ int len = values.length;
+ long[] longs = new long[len];
+ for (int i = 0; i < len; i++) {
+ long v = values[i];
+ longs[i] = v;
+ if (isUnsigned && v < 0) {
+ isUnsigned = false;
+ }
+ }
+
+ return wrap(Roaring64NavigableMap.bitmapOf(longs),
+ isUnsigned ? ClickHouseDataType.UInt64 : ClickHouseDataType.Int64);
+ }
+
+ public static ClickHouseBitmap wrap(Object bitmap, ClickHouseDataType innerType) {
+ final ClickHouseBitmap b;
+ if (bitmap instanceof RoaringBitmap) {
+ b = new ClickHouseRoaringBitmap((RoaringBitmap) bitmap, innerType);
+ } else if (bitmap instanceof MutableRoaringBitmap) {
+ b = new ClickHouseMutableRoaringBitmap((MutableRoaringBitmap) bitmap, innerType);
+ } else if (bitmap instanceof ImmutableRoaringBitmap) {
+ b = new ClickHouseImmutableRoaringBitmap((ImmutableRoaringBitmap) bitmap, innerType);
+ } else if (bitmap instanceof Roaring64Bitmap) {
+ b = new ClickHouseRoaring64NavigableMap(
+ Roaring64NavigableMap.bitmapOf(((Roaring64Bitmap) bitmap).toArray()), innerType);
+ } else if (bitmap instanceof Roaring64NavigableMap) {
+ b = new ClickHouseRoaring64NavigableMap((Roaring64NavigableMap) bitmap, innerType);
+ } else {
+ throw new IllegalArgumentException("Only RoaringBitmap is supported but got: " + bitmap);
+ }
+
+ return b;
+ }
+
+ public static ClickHouseBitmap deserialize(DataInputStream in, ClickHouseDataType innerType) throws IOException {
+ int byteLen = byteLength(innerType);
+ int flag = in.readUnsignedByte();
+ if (flag == 0) {
+ byte cardinality = (byte) in.readUnsignedByte();
+ byte[] bytes = new byte[2 + byteLen * cardinality];
+ bytes[0] = (byte) flag;
+ bytes[1] = cardinality;
+ in.read(bytes, 2, bytes.length - 2);
+
+ return ClickHouseBitmap.deserialize(bytes, innerType);
+ } else if (byteLen <= 4) {
+ int len = Utils.readVarInt(in);
+ byte[] bytes = new byte[len];
+ Utils.readFully(in, bytes);
+ RoaringBitmap b = new RoaringBitmap();
+ b.deserialize(flip(newBuffer(len).put(bytes)));
+ return ClickHouseBitmap.wrap(b, innerType);
+ } else {
+ // why? when serializing Roaring64NavigableMap, the initial 5 bytes were removed
+ // with 8 unknown bytes appended
+ throw new UnsupportedOperationException(
+ "Deserializing Roaring64NavigableMap with cardinality larger than 32 is currently not supported.");
+ }
+ }
+
+ public static ClickHouseBitmap deserialize(byte[] bytes, ClickHouseDataType innerType) throws IOException {
+ // https://github.com/ClickHouse/ClickHouse/blob/master/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h#L100
+ ClickHouseBitmap rb = ClickHouseBitmap.wrap();
+
+ if (bytes == null || bytes.length == 0) {
+ return rb;
+ }
+
+ int byteLen = byteLength(innerType);
+ ByteBuffer buffer = ByteBuffer.allocate(bytes.length);
+ if (buffer.order() != ByteOrder.LITTLE_ENDIAN) {
+ buffer = buffer.slice().order(ByteOrder.LITTLE_ENDIAN);
+ }
+ buffer = (ByteBuffer) ((Buffer) buffer.put(bytes)).flip();
+
+ if (buffer.get() == (byte) 0) { // small set
+ int cardinality = buffer.get();
+ if (byteLen == 1) {
+ byte[] values = new byte[cardinality];
+ for (int i = 0; i < cardinality; i++) {
+ values[i] = buffer.get();
+ }
+ rb = ClickHouseBitmap.wrap(values);
+ } else if (byteLen == 2) {
+ short[] values = new short[cardinality];
+ for (int i = 0; i < cardinality; i++) {
+ values[i] = buffer.getShort();
+ }
+ rb = ClickHouseBitmap.wrap(values);
+ } else if (byteLen == 4) {
+ int[] values = new int[cardinality];
+ for (int i = 0; i < cardinality; i++) {
+ values[i] = buffer.getInt();
+ }
+ rb = ClickHouseBitmap.wrap(values);
+ } else {
+ long[] values = new long[cardinality];
+ for (int i = 0; i < cardinality; i++) {
+ values[i] = buffer.getLong();
+ }
+ rb = ClickHouseBitmap.wrap(values);
+ }
+ } else { // serialized bitmap
+ int len = Utils.readVarInt(buffer);
+ if (buffer.remaining() < len) {
+ throw new IllegalStateException(
+ "Need " + len + " bytes to deserialize ClickHouseBitmap but only got " + buffer.remaining());
+ }
+ if (byteLen <= 4) {
+ RoaringBitmap b = new RoaringBitmap();
+ b.deserialize(buffer);
+ rb = ClickHouseBitmap.wrap(b, innerType);
+ } else {
+ // why? when serializing Roaring64NavigableMap, the initial 5 bytes were removed
+ // with 8 unknown bytes appended
+ throw new UnsupportedOperationException(
+ "Deserializing Roaring64NavigableMap with cardinality larger than 32 is currently not supported.");
+ }
+ }
+
+ return rb;
+ }
+
+ private static ByteBuffer newBuffer(int capacity) {
+ ByteBuffer buffer = ByteBuffer.allocate(capacity);
+ if (buffer.order() != ByteOrder.LITTLE_ENDIAN) {
+ buffer = buffer.slice().order(ByteOrder.LITTLE_ENDIAN);
+ }
+
+ return buffer;
+ }
+
+ private static ByteBuffer flip(ByteBuffer buffer) {
+ return (ByteBuffer) ((Buffer) buffer).flip();
+ }
+
+ private static int byteLength(ClickHouseDataType type) {
+ int byteLen = 0;
+ switch (Objects.requireNonNull(type)) {
+ case Int8:
+ case UInt8:
+ byteLen = 1;
+ break;
+ case Int16:
+ case UInt16:
+ byteLen = 2;
+ break;
+ case Int32:
+ case UInt32:
+ byteLen = 4;
+ break;
+ case Int64:
+ case UInt64:
+ byteLen = 8;
+ break;
+ default:
+ throw new IllegalArgumentException("Only native integer types are supported but we got: " + type.name());
+ }
+
+ return byteLen;
+ }
+
+ protected final ClickHouseDataType innerType;
+ protected final int byteLen;
+ protected final Object reference;
+
+ protected ClickHouseBitmap(Object bitmap, ClickHouseDataType innerType) {
+ this.innerType = innerType;
+ this.byteLen = byteLength(innerType);
+ this.reference = Objects.requireNonNull(bitmap);
+ }
+
+ public abstract int getCardinality();
+
+ public long getLongCardinality() {
+ return getCardinality();
+ }
+
+ public abstract void serialize(ByteBuffer buffer);
+
+ public abstract int serializedSizeInBytes();
+
+ public long serializedSizeInBytesAsLong() {
+ return serializedSizeInBytes();
+ }
+
+ public abstract int[] toIntArray();
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj) {
+ return true;
+ }
+
+ if (obj == null || getClass() != obj.getClass()) {
+ return false;
+ }
+
+ ClickHouseBitmap b = (ClickHouseBitmap) obj;
+ return Objects.equals(innerType, b.innerType) && Objects.equals(byteLen, b.byteLen)
+ && Objects.equals(reference, b.reference);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(innerType, byteLen, reference);
+ }
+
+ public long[] toLongArray() {
+ int[] ints = toIntArray();
+ int len = ints.length;
+ long[] longs = new long[len];
+ for (int i = 0; i < len; i++) {
+ longs[i] = ints[i];
+ }
+ return longs;
+ }
+
+ public ByteBuffer toByteBuffer() {
+ ByteBuffer buf;
+
+ int cardinality = getCardinality();
+ if (cardinality <= 32) {
+ buf = ByteBuffer.allocate(2 + byteLen * cardinality);
+ if (buf.order() != ByteOrder.LITTLE_ENDIAN) {
+ buf = buf.slice().order(ByteOrder.LITTLE_ENDIAN);
+ }
+ buf.put((byte) 0);
+ buf.put((byte) cardinality);
+ if (byteLen == 1) {
+ for (int v : toIntArray()) {
+ buf.put((byte) v);
+ }
+ } else if (byteLen == 2) {
+ for (int v : toIntArray()) {
+ buf.putShort((short) v);
+ }
+ } else if (byteLen == 4) {
+ for (int v : toIntArray()) {
+ buf.putInt(v);
+ }
+ } else { // 64
+ for (long v : toLongArray()) {
+ buf.putLong(v);
+ }
+ }
+ } else if (byteLen <= 4) {
+ int size = serializedSizeInBytes();
+ int varIntSize = Utils.getVarIntSize(size);
+
+ buf = ByteBuffer.allocate(1 + varIntSize + size);
+ if (buf.order() != ByteOrder.LITTLE_ENDIAN) {
+ buf = buf.slice().order(ByteOrder.LITTLE_ENDIAN);
+ }
+ buf.put((byte) 1);
+ Utils.writeVarInt(size, buf);
+ serialize(buf);
+ } else { // 64
+ // 1) exclude the leading 5 bytes - boolean flag + map size, see below:
+ // https://github.com/RoaringBitmap/RoaringBitmap/blob/0.9.9/RoaringBitmap/src/main/java/org/roaringbitmap/longlong/Roaring64NavigableMap.java#L1107
+ // 2) not sure what's the extra 8 bytes?
+ long size = serializedSizeInBytesAsLong() - 5 + 8;
+ int varIntSize = Utils.getVarLongSize(size);
+ // TODO add serialize(DataOutput) to handle more
+ int intSize = (int) size;
+ buf = ByteBuffer.allocate(1 + varIntSize + intSize);
+ if (buf.order() != ByteOrder.LITTLE_ENDIAN) {
+ buf = buf.slice().order(ByteOrder.LITTLE_ENDIAN);
+ }
+ buf.put((byte) 1);
+ Utils.writeVarInt(intSize, buf);
+ buf.putLong(1L); // what's this?
+ serialize(buf);
+ }
+
+ return (ByteBuffer) ((Buffer) buf).flip();
+ }
+
+ public byte[] toBytes() {
+ ByteBuffer buffer = toByteBuffer();
+ byte[] bytes = new byte[buffer.remaining()];
+ buffer.get(bytes);
+ return bytes;
+ }
+
+ public String toBitmapBuildExpression() {
+ StringBuilder sb = new StringBuilder();
+
+ if (byteLen <= 4) {
+ for (int v : toIntArray()) {
+ sb.append(',').append("to").append(innerType.name()).append('(').append(v).append(')');
+ }
+ } else {
+ for (long v : toLongArray()) {
+ sb.append(',').append("to").append(innerType.name()).append('(').append(v).append(')');
+ }
+ }
+
+ if (sb.length() > 0) {
+ sb.deleteCharAt(0).insert(0, '[').append(']');
+ } else {
+ sb.append("cast([] as Array(").append(innerType.name()).append(')').append(')');
+ }
+
+ return sb.insert(0, "bitmapBuild(").append(')').toString();
+ }
+
+ public Object unwrap() {
+ return this.reference;
+ }
+}
diff --git a/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/util/ClickHouseRowBinaryInputStream.java b/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/util/ClickHouseRowBinaryInputStream.java
index 4a362cc90..3fc97c263 100644
--- a/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/util/ClickHouseRowBinaryInputStream.java
+++ b/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/util/ClickHouseRowBinaryInputStream.java
@@ -1,5 +1,6 @@
package ru.yandex.clickhouse.util;
+import ru.yandex.clickhouse.domain.ClickHouseDataType;
import ru.yandex.clickhouse.settings.ClickHouseProperties;
import java.io.Closeable;
@@ -149,6 +150,22 @@ public long readInt64() throws IOException {
return Utils.readLong(in);
}
+ public BigInteger readInt128() throws IOException {
+ return Utils.readBigInteger(in, 16);
+ }
+
+ public BigInteger writeUInt128() throws IOException {
+ return Utils.readBigInteger(in, 16);
+ }
+
+ public BigInteger writeInt256() throws IOException {
+ return Utils.readBigInteger(in, 32);
+ }
+
+ public BigInteger writeUInt256() throws IOException {
+ return Utils.readBigInteger(in, 32);
+ }
+
public Timestamp readDateTime() throws IOException {
long value = readUInt32();
return new Timestamp(TimeUnit.SECONDS.toMillis(value));
@@ -389,6 +406,10 @@ public BigDecimal readDecimal256(int scale) throws IOException {
return res;
}
+ public ClickHouseBitmap readBitmap(ClickHouseDataType innerType) throws IOException {
+ return ClickHouseBitmap.deserialize(in, innerType);
+ }
+
@Override
public void close() throws IOException {
in.close();
diff --git a/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/util/ClickHouseRowBinaryStream.java b/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/util/ClickHouseRowBinaryStream.java
index 54d25b3ec..930a4da3b 100644
--- a/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/util/ClickHouseRowBinaryStream.java
+++ b/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/util/ClickHouseRowBinaryStream.java
@@ -9,6 +9,7 @@
import java.math.BigInteger;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
+import java.nio.channels.Channels;
import java.nio.charset.StandardCharsets;
import java.util.Date;
import java.util.Objects;
@@ -61,6 +62,10 @@ public void writeBytes(byte[] bytes) throws IOException {
out.write(bytes);
}
+ public void writeByteBuffer(ByteBuffer buffer) throws IOException {
+ Channels.newChannel(out).write(buffer);
+ }
+
/**
* Dangerous. Can only be used for rare optimizations, for example when the string is written in parts
* without prior concatenation. The size of the string in bytes must be passed through writeUnsignedLeb128.
@@ -342,4 +347,8 @@ public void writeUUIDArray(UUID[] uuids) throws IOException {
writeUUID(uuid);
}
}
+
+ public void writeBitmap(ClickHouseBitmap rb) throws IOException {
+ this.writeByteBuffer(Objects.requireNonNull(rb).toByteBuffer());
+ }
}
diff --git a/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/util/ClickHouseValueFormatter.java b/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/util/ClickHouseValueFormatter.java
index 9a37aeecd..a6bac9928 100644
--- a/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/util/ClickHouseValueFormatter.java
+++ b/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/util/ClickHouseValueFormatter.java
@@ -189,14 +189,17 @@ public static String formatInstant(Instant x, TimeZone timeZone) {
.format(x);
}
- public static String formatMap(Map, ?> map, TimeZone dateTimeZone,
- TimeZone dateTimeTimeZone) {
+ public static String formatBitmap(ClickHouseBitmap bitmap) {
+ return bitmap.toBitmapBuildExpression();
+ }
+
+ public static String formatMap(Map, ?> map, TimeZone dateTimeZone, TimeZone dateTimeTimeZone) {
StringBuilder sb = new StringBuilder();
for (Map.Entry, ?> e : map.entrySet()) {
Object key = e.getKey();
Object value = e.getValue();
sb.append(',');
-
+
if (key instanceof String) {
sb.append('\'').append(formatString((String) key)).append('\'');
} else {
@@ -267,6 +270,8 @@ public static String formatObject(Object x, TimeZone dateTimeZone,
return formatBigInteger((BigInteger) x);
} else if (x instanceof Collection) {
return ClickHouseArrayUtil.toString((Collection>) x, dateTimeZone, dateTimeTimeZone);
+ } else if (x instanceof ClickHouseBitmap) {
+ return formatBitmap((ClickHouseBitmap) x);
} else if (x instanceof Map) {
return formatMap((Map, ?>) x, dateTimeZone, dateTimeTimeZone);
} else if (x.getClass().isArray()) {
@@ -281,12 +286,14 @@ public static boolean needsQuoting(Object o) {
|| o instanceof Array
|| o instanceof Boolean
|| o instanceof Collection
+ // || o instanceof Iterable
|| o instanceof Map
|| o instanceof Number
- || o.getClass().isArray()) {
+ || o.getClass().isArray()
+ || o instanceof ClickHouseBitmap) {
return false;
}
-
+
return true;
}
diff --git a/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/util/Utils.java b/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/util/Utils.java
index 119fcaf80..f05cc974e 100644
--- a/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/util/Utils.java
+++ b/clickhouse-jdbc/src/main/java/ru/yandex/clickhouse/util/Utils.java
@@ -1,6 +1,7 @@
package ru.yandex.clickhouse.util;
import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.EOFException;
@@ -9,6 +10,7 @@
import java.io.OutputStream;
import java.math.BigDecimal;
import java.math.BigInteger;
+import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Collections;
@@ -236,6 +238,15 @@ public static int readUnsignedLeb128(DataInputStream inputStream) throws IOExcep
return value;
}
+ public static BigInteger readBigInteger(DataInputStream inputStream, int byteLength) throws IOException {
+ byte[] r = new byte[byteLength];
+ for (int i = r.length; i > 0; i--) {
+ r[i - 1] = inputStream.readByte();
+ }
+
+ return new BigInteger(r);
+ }
+
public static void writeShort(DataOutputStream outputStream, int value) throws IOException {
outputStream.write(0xFF & value);
outputStream.write(0xFF & (value >> 8));
@@ -272,6 +283,70 @@ public static void writeBigInteger(DataOutputStream outputStream, BigInteger val
}
}
+ public static int getVarIntSize(int value) {
+ int result = 0;
+ do {
+ result++;
+ value >>>= 7;
+ } while (value != 0);
+
+ return result;
+ }
+
+ public static int getVarLongSize(long value) {
+ int result = 0;
+ do {
+ result++;
+ value >>>= 7;
+ } while (value != 0);
+
+ return result;
+ }
+
+ public static void writeVarInt(int value, ByteBuffer buffer) {
+ while ((value & 0xFFFFFF80) != 0L) {
+ buffer.put((byte) ((value & 0x7F) | 0x80));
+ value >>>= 7;
+ }
+ buffer.put((byte) (value & 0x7F));
+ }
+
+ public static int readVarInt(DataInput in) throws IOException {
+ int result = 0;
+ int shift = 0;
+ int b;
+ do {
+ if (shift >= 32) {
+ // Out of range
+ throw new IndexOutOfBoundsException("varint too long");
+ }
+ // Get 7 bits from next byte
+ b = in.readUnsignedByte();
+ result |= (b & 0x7F) << shift;
+ shift += 7;
+ } while ((b & 0x80) != 0);
+
+ return result;
+ }
+
+ public static int readVarInt(ByteBuffer buffer) {
+ int result = 0;
+ int shift = 0;
+ int b;
+ do {
+ if (shift >= 32) {
+ // Out of range
+ throw new IndexOutOfBoundsException("varint too long");
+ }
+ // Get 7 bits from next byte
+ b = buffer.get();
+ result |= (b & 0x7F) << shift;
+ shift += 7;
+ } while ((b & 0x80) != 0);
+
+ return result;
+ }
+
public static BigInteger toBigInteger(BigDecimal num, int scale) {
BigDecimal ten = BigDecimal.valueOf(10);
BigDecimal s = ten.pow(scale);
diff --git a/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/ClickHouseContainerForTest.java b/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/ClickHouseContainerForTest.java
index 0dcf7e61c..70a696f63 100644
--- a/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/ClickHouseContainerForTest.java
+++ b/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/ClickHouseContainerForTest.java
@@ -1,5 +1,7 @@
package ru.yandex.clickhouse;
+import static java.time.temporal.ChronoUnit.SECONDS;
+
import java.time.Duration;
import org.testcontainers.containers.BindMode;
import org.testcontainers.containers.GenericContainer;
@@ -7,12 +9,9 @@
import org.testcontainers.images.builder.ImageFromDockerfile;
import org.testng.annotations.AfterSuite;
import org.testng.annotations.BeforeSuite;
-
import ru.yandex.clickhouse.settings.ClickHouseProperties;
import ru.yandex.clickhouse.util.ClickHouseVersionNumberUtil;
-import static java.time.temporal.ChronoUnit.SECONDS;
-
public class ClickHouseContainerForTest {
private static final int HTTP_PORT = 8123;
private static final int NATIVE_PORT = 9000;
@@ -41,10 +40,10 @@ public class ClickHouseContainerForTest {
final String imageNameWithTag = "yandex/clickhouse-server" + imageTag;
- clickhouseContainer = new GenericContainer<>( new ImageFromDockerfile()
+ clickhouseContainer = new GenericContainer<>(new ImageFromDockerfile()
.withDockerfileFromBuilder(builder ->
builder
- .from( imageNameWithTag )
+ .from(imageNameWithTag)
.run("apt-get update && apt-get install tzdata")
))
.withEnv("TZ", timezone)
diff --git a/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/integration/ClickHouseBitmapTest.java b/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/integration/ClickHouseBitmapTest.java
new file mode 100644
index 000000000..7a5c3a4ec
--- /dev/null
+++ b/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/integration/ClickHouseBitmapTest.java
@@ -0,0 +1,170 @@
+package ru.yandex.clickhouse.integration;
+
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertTrue;
+
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+
+import org.roaringbitmap.RoaringBitmap;
+import org.roaringbitmap.buffer.ImmutableRoaringBitmap;
+import org.roaringbitmap.buffer.MutableRoaringBitmap;
+import org.testng.annotations.AfterTest;
+import org.testng.annotations.BeforeTest;
+import org.testng.annotations.Test;
+
+import ru.yandex.clickhouse.ClickHouseContainerForTest;
+import ru.yandex.clickhouse.ClickHouseDataSource;
+import ru.yandex.clickhouse.domain.ClickHouseDataType;
+import ru.yandex.clickhouse.util.ClickHouseBitmap;
+
+public class ClickHouseBitmapTest {
+ private Connection conn;
+
+ @BeforeTest
+ public void setUp() throws Exception {
+ ClickHouseDataSource dataSource = ClickHouseContainerForTest.newDataSource();
+ conn = dataSource.getConnection();
+ }
+
+ @AfterTest
+ public void tearDown() throws Exception {
+ if (conn != null) {
+ conn.close();
+ }
+ }
+
+ private void checkBitmaps(ResultSet rs, ClickHouseBitmap empty, ClickHouseBitmap small, ClickHouseBitmap large)
+ throws SQLException {
+ // Charset charset = Charset.forName("ISO-8859-15");
+ Charset charset = StandardCharsets.UTF_8;
+
+ assertTrue(rs.next());
+ assertEquals(rs.getObject(1), 0L);
+ assertEquals(rs.getString(2), new String(empty.toBytes(), charset));
+ assertEquals(rs.getObject(2), rs.getObject(2, ClickHouseBitmap.class));
+ assertEquals(rs.getObject(2, ClickHouseBitmap.class), empty);
+
+ assertTrue(rs.next());
+ assertEquals(rs.getObject(1), 1L);
+ assertEquals(rs.getString(2), new String(small.toBytes(), charset));
+ assertEquals(rs.getObject(2), rs.getObject(2, ClickHouseBitmap.class));
+ assertEquals(rs.getObject(2, ClickHouseBitmap.class), small);
+
+ assertTrue(rs.next());
+ assertEquals(rs.getObject(1), 2L);
+ assertEquals(rs.getString(2), new String(large.toBytes(), charset));
+ assertEquals(rs.getObject(2), rs.getObject(2, ClickHouseBitmap.class));
+ assertEquals(rs.getObject(2, ClickHouseBitmap.class), large);
+
+ assertFalse(rs.next());
+ }
+
+ @Test
+ public void testRoaringBitmap() throws Exception {
+ if (conn == null) {
+ return;
+ }
+
+ int[] smallSet = new int[32];
+ for (int i = 0, len = smallSet.length; i < len; i++) {
+ int value = i + 100;
+ smallSet[i] = value;
+ }
+
+ int[] largeSet = new int[33];
+ for (int i = 0, len = largeSet.length; i < len; i++) {
+ int value = i + 60000;
+ largeSet[i] = value;
+ }
+
+ String testQuery = "select *, base64Encode(toString(rb)) as x from test_roaring_bitmap order by i";
+ ClickHouseBitmap empty = ClickHouseBitmap.wrap(RoaringBitmap.bitmapOf(), ClickHouseDataType.UInt32);
+ ClickHouseBitmap small = ClickHouseBitmap.wrap(smallSet);
+ ClickHouseBitmap large = ClickHouseBitmap.wrap(largeSet);
+ try (Statement s = conn.createStatement()) {
+ s.execute("DROP TABLE IF EXISTS test_roaring_bitmap");
+ // AggregateFunction(groupBitmap, UInt32) cannot be used inside Nullable type
+ // AggregateFunction(groupBitmap, Nullable(UInt32)) can be created but not
+ // usable
+ s.execute(
+ "CREATE TABLE IF NOT EXISTS test_roaring_bitmap(i UInt32, rb AggregateFunction(groupBitmap, UInt32)) ENGINE = Memory");
+ s.execute("insert into test_roaring_bitmap values(0, " + empty.toBitmapBuildExpression() + ")");
+ s.execute("insert into test_roaring_bitmap values(1, " + small.toBitmapBuildExpression() + ")");
+ s.execute("insert into test_roaring_bitmap values(2, " + large.toBitmapBuildExpression() + ")");
+
+ try (ResultSet rs = s.executeQuery(testQuery)) {
+ checkBitmaps(rs, empty, small, large);
+ }
+
+ s.execute("truncate table test_roaring_bitmap");
+ }
+
+ // FIXME too bad batching is not supported
+ try (PreparedStatement s = conn.prepareStatement("insert into test_roaring_bitmap values(?,?)")) {
+ s.setObject(1, 0L);
+ s.setObject(2, empty);
+ s.execute();
+ s.setObject(1, 1L);
+ s.setObject(2, small);
+ s.execute();
+ s.setObject(1, 2L);
+ s.setObject(2, large);
+ s.execute();
+ }
+
+ try (Statement s = conn.createStatement()) {
+ try (ResultSet rs = s.executeQuery(testQuery)) {
+ checkBitmaps(rs, empty, small, large);
+ }
+
+ s.execute("truncate table test_roaring_bitmap");
+ }
+
+ try (PreparedStatement s = conn.prepareStatement("insert into test_roaring_bitmap values(?,?)")) {
+ s.setObject(1, 0L);
+ s.setObject(2, ClickHouseBitmap.wrap(MutableRoaringBitmap.bitmapOf(), ClickHouseDataType.UInt32));
+ s.execute();
+ s.setObject(1, 1L);
+ s.setObject(2, ClickHouseBitmap.wrap(MutableRoaringBitmap.bitmapOf(smallSet), ClickHouseDataType.UInt32));
+ s.execute();
+ s.setObject(1, 2L);
+ s.setObject(2, ClickHouseBitmap.wrap(MutableRoaringBitmap.bitmapOf(largeSet), ClickHouseDataType.UInt32));
+ s.execute();
+ }
+
+ try (Statement s = conn.createStatement()) {
+ try (ResultSet rs = s.executeQuery(testQuery)) {
+ checkBitmaps(rs, empty, small, large);
+ }
+
+ s.execute("truncate table test_roaring_bitmap");
+ }
+
+ try (PreparedStatement s = conn.prepareStatement("insert into test_roaring_bitmap values(?,?)")) {
+ s.setObject(1, 0L);
+ s.setObject(2, ClickHouseBitmap.wrap(ImmutableRoaringBitmap.bitmapOf(), ClickHouseDataType.UInt32));
+ s.execute();
+ s.setObject(1, 1L);
+ s.setObject(2, ClickHouseBitmap.wrap(ImmutableRoaringBitmap.bitmapOf(smallSet), ClickHouseDataType.UInt32));
+ s.execute();
+ s.setObject(1, 2L);
+ s.setObject(2, ClickHouseBitmap.wrap(ImmutableRoaringBitmap.bitmapOf(largeSet), ClickHouseDataType.UInt32));
+ s.execute();
+ }
+
+ try (Statement s = conn.createStatement()) {
+ try (ResultSet rs = s.executeQuery(testQuery)) {
+ checkBitmaps(rs, empty, small, large);
+ }
+
+ s.execute("truncate table test_roaring_bitmap");
+ }
+ }
+}
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 21db30b14..02ea533b1 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
@@ -1,5 +1,10 @@
package ru.yandex.clickhouse.integration;
+import static org.testng.Assert.assertEquals;
+import static org.testng.AssertJUnit.assertNotNull;
+import static org.testng.AssertJUnit.assertNull;
+import static org.testng.AssertJUnit.assertTrue;
+
import java.io.ByteArrayInputStream;
import java.io.InputStream;
import java.io.UnsupportedEncodingException;
@@ -17,13 +22,11 @@
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
-
import org.mockito.internal.util.reflection.Whitebox;
import org.testng.Assert;
import org.testng.annotations.AfterTest;
import org.testng.annotations.BeforeTest;
import org.testng.annotations.Test;
-
import ru.yandex.clickhouse.ClickHouseConnection;
import ru.yandex.clickhouse.ClickHouseContainerForTest;
import ru.yandex.clickhouse.ClickHouseDataSource;
@@ -32,11 +35,6 @@
import ru.yandex.clickhouse.settings.ClickHouseProperties;
import ru.yandex.clickhouse.settings.ClickHouseQueryParam;
-import static org.testng.Assert.assertEquals;
-import static org.testng.AssertJUnit.assertNotNull;
-import static org.testng.AssertJUnit.assertNull;
-import static org.testng.AssertJUnit.assertTrue;
-
public class ClickHouseStatementImplTest {
private ClickHouseDataSource dataSource;
@@ -129,24 +127,27 @@ public void testSelectUInt64() throws SQLException {
@Test
public void testExternalData() throws SQLException, UnsupportedEncodingException {
ClickHouseStatement stmt = connection.createStatement();
- ResultSet rs = stmt.executeQuery(
+ String[] rows = "21.3.3.14".equals(connection.getServerVersion())
+ ? new String[] { "1\tGroup\n" }
+ : new String[] { "1\tGroup", "1\tGroup\n" };
+
+ for (String row : rows) {
+ try (ResultSet rs = stmt.executeQuery(
"select UserName, GroupName " +
- "from (select 'User' as UserName, 1 as GroupId) as g" +
- "any left join groups using GroupId",
- null,
+ "from (select 'User' as UserName, 1 as GroupId) as g" +
+ "any left join groups using GroupId", null,
Collections.singletonList(new ClickHouseExternalData(
"groups",
- new ByteArrayInputStream("1\tGroup".getBytes())
- ).withStructure("GroupId UInt8, GroupName String"))
- );
-
- rs.next();
-
- String userName = rs.getString("UserName");
- String groupName = rs.getString("GroupName");
-
- Assert.assertEquals(userName, "User");
- Assert.assertEquals(groupName, "Group");
+ new ByteArrayInputStream(row.getBytes())
+ ).withStructure("GroupId UInt8, GroupName String")))) {
+ Assert.assertTrue(rs.next());
+ String userName = rs.getString("UserName");
+ String groupName = rs.getString("GroupName");
+
+ Assert.assertEquals(userName, "User");
+ Assert.assertEquals(groupName, "Group");
+ }
+ }
}
@@ -169,11 +170,11 @@ private boolean genNextString() {
@Override
public int read() {
if (si >= s.length()) {
- if ( ! genNextString() ) {
+ if (!genNextString()) {
return -1;
}
}
- return s.charAt( si++ );
+ return s.charAt(si++);
}
};
}
@@ -181,17 +182,23 @@ public int read() {
@Test
public void testExternalDataStream() throws SQLException, UnsupportedEncodingException {
+ if ("21.3.3.14".equals(connection.getServerVersion())) {
+ return;
+ }
+
final ClickHouseStatement statement = connection.createStatement();
- connection.createStatement().execute("DROP TABLE IF EXISTS test.testExternalData");
- connection.createStatement().execute("CREATE TABLE test.testExternalData (id UInt64, s String) ENGINE = Memory");
- connection.createStatement().execute("insert into test.testExternalData select number, toString(number) from numbers(500,100000)");
+ statement.execute("DROP TABLE IF EXISTS test.testExternalData");
+ statement.execute(
+ "CREATE TABLE test.testExternalData (id UInt64, s String) ENGINE = Memory");
+ statement.execute(
+ "insert into test.testExternalData select number, toString(number) from numbers(500,100000)");
InputStream inputStream = getTSVStream(100000);
ClickHouseExternalData extData = new ClickHouseExternalData("ext_data", inputStream);
extData.setStructure("id UInt64, s String");
- ResultSet rs = connection.createStatement().executeQuery(
+ ResultSet rs = statement.executeQuery(
"select count() cnt from test.testExternalData where (id,s) in ext_data",
null,
Collections.singletonList(extData)
@@ -240,7 +247,8 @@ public void testSelectOne() throws SQLException {
public void testSelectManyRows() throws SQLException {
Statement stmt = connection.createStatement();
int limit = 10000;
- ResultSet rs = stmt.executeQuery("select concat('test', toString(number)) as str from system.numbers limit " + limit);
+ ResultSet rs = stmt.executeQuery(
+ "select concat('test', toString(number)) as str from system.numbers limit " + limit);
int i = 0;
while (rs.next()) {
String s = rs.getString("str");
@@ -312,7 +320,9 @@ public void run() {
final long timeout = 10;
String queryId = (String) readField(firstStatement, "queryId", timeout);
- assertNotNull(String.format("it's actually very strange. It seems the query hasn't been executed in %s seconds", timeout), queryId);
+ assertNotNull(
+ String.format("it's actually very strange. It seems the query hasn't been executed in %s seconds", timeout),
+ queryId);
assertNull("An exception happened while the query was being executed", exceptionAtomicReference.get());
@@ -349,7 +359,9 @@ public void run() {
thread.setDaemon(true);
thread.start();
final long timeout = 10;
- assertTrue(String.format("it's actually very strange. It seems the query hasn't been executed in %s seconds", timeout), countDownLatch.await(timeout, TimeUnit.SECONDS));
+ assertTrue(
+ String.format("it's actually very strange. It seems the query hasn't been executed in %s seconds", timeout),
+ countDownLatch.await(timeout, TimeUnit.SECONDS));
assertNull("An exception happened while the query was being executed", exceptionAtomicReference.get());
assertTrue("The query isn't being executed. It seems very strange", checkQuery(queryId, true,10));
diff --git a/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/integration/ClickhouseLZ4StreamTest.java b/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/integration/ClickhouseLZ4StreamTest.java
index 001435a30..03b41d54a 100644
--- a/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/integration/ClickhouseLZ4StreamTest.java
+++ b/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/integration/ClickhouseLZ4StreamTest.java
@@ -1,21 +1,19 @@
package ru.yandex.clickhouse.integration;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
import org.testng.Assert;
import org.testng.annotations.BeforeTest;
import org.testng.annotations.Test;
-
+import ru.yandex.clickhouse.ClickHouseConnection;
import ru.yandex.clickhouse.ClickHouseContainerForTest;
import ru.yandex.clickhouse.ClickHouseDataSource;
import ru.yandex.clickhouse.settings.ClickHouseProperties;
-import java.sql.Connection;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-
public class ClickhouseLZ4StreamTest {
private ClickHouseDataSource dataSource;
- private Connection connection;
+ private ClickHouseConnection connection;
@BeforeTest
public void setUp() throws Exception {
@@ -28,6 +26,10 @@ public void setUp() throws Exception {
@Test
public void testBigBatchCompressedInsert() throws SQLException {
+ if ("21.3.3.14".equals(connection.getServerVersion())) {
+ return;
+ }
+
connection.createStatement().execute("DROP TABLE IF EXISTS test.big_batch_insert");
connection.createStatement().execute(
"CREATE TABLE IF NOT EXISTS test.big_batch_insert (i Int32, s String) ENGINE = TinyLog"
diff --git a/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/integration/RowBinaryStreamTest.java b/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/integration/RowBinaryStreamTest.java
index 5cda9663e..afb4743a9 100644
--- a/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/integration/RowBinaryStreamTest.java
+++ b/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/integration/RowBinaryStreamTest.java
@@ -1,5 +1,10 @@
package ru.yandex.clickhouse.integration;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertThrows;
+import static org.testng.Assert.assertTrue;
+
import java.io.EOFException;
import java.io.IOException;
import java.math.BigInteger;
@@ -13,20 +18,24 @@
import java.util.Calendar;
import java.util.UUID;
import java.util.concurrent.atomic.AtomicLong;
-
+import org.roaringbitmap.RoaringBitmap;
+import org.roaringbitmap.buffer.ImmutableRoaringBitmap;
+import org.roaringbitmap.buffer.MutableRoaringBitmap;
+import org.roaringbitmap.longlong.Roaring64Bitmap;
+import org.roaringbitmap.longlong.Roaring64NavigableMap;
import org.testng.Assert;
import org.testng.annotations.BeforeTest;
import org.testng.annotations.Test;
-
import ru.yandex.clickhouse.ClickHouseConnection;
import ru.yandex.clickhouse.ClickHouseContainerForTest;
import ru.yandex.clickhouse.ClickHouseDataSource;
import ru.yandex.clickhouse.ClickHouseStatement;
+import ru.yandex.clickhouse.domain.ClickHouseDataType;
+import ru.yandex.clickhouse.util.ClickHouseBitmap;
import ru.yandex.clickhouse.util.ClickHouseRowBinaryInputStream;
import ru.yandex.clickhouse.util.ClickHouseRowBinaryStream;
import ru.yandex.clickhouse.util.ClickHouseStreamCallback;
-
-import static org.testng.Assert.assertEquals;
+import ru.yandex.clickhouse.util.ClickHouseVersionNumberUtil;
/**
* @author Dmitry Andreev
@@ -83,9 +92,7 @@ private void createTable(String table) throws SQLException {
public void multiRowTest() throws SQLException {
connection.createStatement().execute("DROP TABLE IF EXISTS test.big_data");
connection.createStatement().execute(
- "CREATE TABLE test.big_data (" +
- "value Int32" +
- ") ENGINE = TinyLog()"
+ "CREATE TABLE test.big_data (value Int32) ENGINE = TinyLog()"
);
final int count = 1000000;
@@ -122,6 +129,155 @@ public void testRowBinaryInputStream() throws Exception {
testRowBinaryStream(true);
}
+ private String createtestBitmapTable(ClickHouseDataType innerType) throws Exception {
+ String tableName = "test_binary_rb_" + innerType.name();
+ String arrType = "Array(" + innerType.name() + ")";
+ String rbTypeName = "AggregateFunction(groupBitmap, " + innerType.name() + ")";
+ try (ClickHouseStatement statement = connection.createStatement()) {
+ statement.execute("DROP TABLE IF EXISTS " + tableName);
+ statement.execute("CREATE TABLE IF NOT EXISTS " + tableName +
+ "(i UInt8, a " + arrType + ", b " + rbTypeName + ") engine=Memory");
+ }
+
+ return tableName;
+ }
+
+ private int[] genRoaringBitmapValues(int length, ClickHouseDataType innerType) {
+ int[] values = new int[length];
+
+ for (int i = 0; i < length; i++) {
+ values[i] = i;
+ }
+
+ return values;
+ }
+
+ private long[] genRoaring64BitmapValues(int length) {
+ long[] values = new long[length];
+
+ for (int i = 0; i < length; i++) {
+ values[i] = 100000L + i;
+ }
+
+ return values;
+ }
+
+ private void writeValues(ClickHouseRowBinaryStream stream,
+ int [] values, ClickHouseDataType innerType) throws IOException {
+ switch (innerType) {
+ case Int8:
+ case UInt8:
+ stream.writeUInt8Array(values);
+ break;
+ case Int16:
+ case UInt16:
+ stream.writeUInt16Array(values);
+ break;
+ case Int32:
+ case UInt32:
+ stream.writeInt32Array(values);
+ break;
+ default:
+ throw new IllegalArgumentException(innerType.name() + " is not supported!");
+ }
+ }
+
+ private void testBitmap(ClickHouseDataType innerType, int valueLength) throws Exception {
+ try (ClickHouseStatement statement = connection.createStatement()) {
+ String tableName = createtestBitmapTable(innerType);
+ int[] values = genRoaringBitmapValues(valueLength, innerType);
+ statement.sendRowBinaryStream("insert into table " + tableName, new ClickHouseStreamCallback() {
+ @Override
+ public void writeTo(ClickHouseRowBinaryStream stream) throws IOException {
+ stream.writeByte((byte) 1);
+ writeValues(stream, values, innerType);
+ stream.writeBitmap(ClickHouseBitmap.wrap(RoaringBitmap.bitmapOf(values), innerType));
+ stream.writeByte((byte) 2);
+ writeValues(stream, values, innerType);
+ stream.writeBitmap(ClickHouseBitmap.wrap(ImmutableRoaringBitmap.bitmapOf(values), innerType));
+ stream.writeByte((byte) 3);
+ writeValues(stream, values, innerType);
+ stream.writeBitmap(ClickHouseBitmap.wrap(MutableRoaringBitmap.bitmapOf(values), innerType));
+ }
+ });
+
+ for (int i = 0; i < 3; i++) {
+ String sql = "select b = bitmapBuild(a) ? 1 : 0 from " + tableName + " where i = " + (i+1);
+ try (ResultSet rs = statement.executeQuery(sql)) {
+ assertTrue(rs.next());
+ assertEquals(rs.getInt(1), 1);
+ assertFalse(rs.next());
+ }
+
+ sql = "select b from " + tableName + " where i = " + (i+1);
+ try (ClickHouseRowBinaryInputStream in = statement.executeQueryClickhouseRowBinaryStream(sql)) {
+ assertEquals(in.readBitmap(innerType), ClickHouseBitmap.wrap(RoaringBitmap.bitmapOf(values), innerType));
+ }
+ }
+
+ statement.execute("drop table if exists " + tableName);
+ }
+ }
+
+ private void testBitmap64(int valueLength) throws Exception {
+ ClickHouseDataType innerType = ClickHouseDataType.UInt64;
+ try (ClickHouseStatement statement = connection.createStatement()) {
+ String tableName = createtestBitmapTable(innerType);
+ long[] values = genRoaring64BitmapValues(valueLength);
+ statement.sendRowBinaryStream("insert into table " + tableName, new ClickHouseStreamCallback() {
+ @Override
+ public void writeTo(ClickHouseRowBinaryStream stream) throws IOException {
+ stream.writeByte((byte) 1);
+ stream.writeUInt64Array(values);
+ stream.writeBitmap(ClickHouseBitmap.wrap(Roaring64NavigableMap.bitmapOf(values), ClickHouseDataType.UInt64));
+ stream.writeByte((byte) 2);
+ stream.writeUInt64Array(values);
+ stream.writeBitmap(ClickHouseBitmap.wrap(Roaring64Bitmap.bitmapOf(values), ClickHouseDataType.UInt64));
+ }
+ });
+
+ String sql = "select bitmapBuild(a) = b ? 1 : 0 from " + tableName + " order by i";
+ try (ResultSet rs = statement.executeQuery(sql)) {
+ assertTrue(rs.next());
+ assertEquals(rs.getInt(1), 1);
+ assertTrue(rs.next());
+ assertEquals(rs.getInt(1), 1);
+ assertFalse(rs.next());
+ }
+
+ sql = "select b from " + tableName + " order by i";
+ try (ClickHouseRowBinaryInputStream in = statement.executeQueryClickhouseRowBinaryStream(sql)) {
+ if (valueLength <= 32) {
+ assertEquals(in.readBitmap(innerType), ClickHouseBitmap.wrap(Roaring64NavigableMap.bitmapOf(values), innerType));
+ } else {
+ assertThrows(UnsupportedOperationException.class, () -> in.readBitmap(innerType));
+ }
+ }
+
+ statement.execute("drop table if exists " + tableName);
+ }
+ }
+
+ @Test
+ public void testBitmap() throws Exception {
+ // TODO seems Int8, Int16 and Int32 are still not supported in ClickHouse
+ testBitmap(ClickHouseDataType.UInt8, 32);
+ testBitmap(ClickHouseDataType.UInt8, 256);
+ testBitmap(ClickHouseDataType.UInt16, 32);
+ testBitmap(ClickHouseDataType.UInt16, 65536);
+ testBitmap(ClickHouseDataType.UInt32, 32);
+ testBitmap(ClickHouseDataType.UInt32, 65537);
+
+ testBitmap64(32);
+
+ String versionNumber = connection.getServerVersion();
+ int majorVersion = ClickHouseVersionNumberUtil.getMajorVersion(versionNumber);
+ int minorVersion = ClickHouseVersionNumberUtil.getMinorVersion(versionNumber);
+ if (majorVersion > 20 || (majorVersion == 20 && minorVersion > 8)) {
+ testBitmap64(65537);
+ }
+ }
+
private void testRowBinaryStream(boolean rowBinaryResult) throws Exception {
createTable("test.raw_binary");
ClickHouseStatement statement = connection.createStatement();
@@ -279,7 +435,8 @@ public void writeTo(ClickHouseRowBinaryStream stream) throws IOException {
Assert.assertFalse(rs.next());
} else {
- ClickHouseRowBinaryInputStream is = connection.createStatement().executeQueryClickhouseRowBinaryStream("SELECT * FROM test.raw_binary ORDER BY date");
+ ClickHouseRowBinaryInputStream is = connection.createStatement().executeQueryClickhouseRowBinaryStream(
+ "SELECT * FROM test.raw_binary ORDER BY date");
assertEquals(is.readDate(), withTimeAtStartOfDay(date1));
assertEquals(is.readDateTime(), new Timestamp(timestamp));
@@ -444,7 +601,6 @@ private static void assertArrayEquals(int[] actual, short[] expected) {
}
assertEquals(actual, expectedInts);
-
}
private static void assertArrayEquals(short[] actual, int[] expected) {
@@ -454,7 +610,6 @@ private static void assertArrayEquals(short[] actual, int[] expected) {
}
assertEquals(actualInts, expected);
-
}
private static void assertArrayEquals(long[] actual, int[] expected) {
@@ -474,6 +629,5 @@ private static void assertArrayEquals(BigInteger[] actual, long[] expected) {
}
assertEquals(actual, expectedBigs);
-
}
}
diff --git a/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/util/ClickHouseBitmapTest.java b/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/util/ClickHouseBitmapTest.java
new file mode 100644
index 000000000..87b045c2c
--- /dev/null
+++ b/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/util/ClickHouseBitmapTest.java
@@ -0,0 +1,29 @@
+package ru.yandex.clickhouse.util;
+
+import static org.testng.Assert.assertEquals;
+
+import org.roaringbitmap.RoaringBitmap;
+import org.roaringbitmap.buffer.ImmutableRoaringBitmap;
+import org.roaringbitmap.buffer.MutableRoaringBitmap;
+import org.testng.annotations.Test;
+
+import ru.yandex.clickhouse.domain.ClickHouseDataType;
+
+public class ClickHouseBitmapTest {
+ @Test
+ public void testEmptyRoaringBitmap() {
+ byte[] expectedBytes = new byte[] { 0, 0 };
+
+ ClickHouseDataType[] types = new ClickHouseDataType[] { ClickHouseDataType.Int8, ClickHouseDataType.UInt8,
+ ClickHouseDataType.Int16, ClickHouseDataType.UInt16, ClickHouseDataType.Int32,
+ ClickHouseDataType.UInt32 };
+ Object[] bitmaps = new Object[] { RoaringBitmap.bitmapOf(), MutableRoaringBitmap.bitmapOf(),
+ ImmutableRoaringBitmap.bitmapOf() };
+
+ for (Object bm : bitmaps) {
+ for (ClickHouseDataType t : types) {
+ assertEquals(ClickHouseBitmap.wrap(bm, t).toBytes(), expectedBytes);
+ }
+ }
+ }
+}
diff --git a/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/util/ClickHouseRowBinaryStreamTest.java b/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/util/ClickHouseRowBinaryStreamTest.java
index 94bfe0ae1..b55f9a590 100644
--- a/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/util/ClickHouseRowBinaryStreamTest.java
+++ b/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/util/ClickHouseRowBinaryStreamTest.java
@@ -254,6 +254,25 @@ public void write(ClickHouseRowBinaryStream stream) throws Exception {
);
}
+ @Test
+ public void testUUIDArray() throws Exception {
+ check(
+ new StreamWriter() {
+ @Override
+ public void write(ClickHouseRowBinaryStream stream) throws Exception {
+ stream.writeUUIDArray(new UUID[] {
+ UUID.fromString("123e4567-e89b-12d3-a456-426655440000"),
+ UUID.fromString("123e4567-e89b-12d3-a456-426655440001")
+ });
+ }
+ },
+ new byte[] {
+ 2, -45, 18, -101, -24, 103, 69, 62, 18, 0, 0, 68, 85, 102, 66, 86, -92,
+ -45, 18, -101, -24, 103, 69, 62, 18, 1, 0, 68, 85, 102, 66, 86, -92
+ }
+ );
+ }
+
@Test
public void testWriteNullableInt32() throws Exception {
check(
diff --git a/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/util/UtilsTest.java b/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/util/UtilsTest.java
index a9e3c33ec..09b7efed5 100644
--- a/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/util/UtilsTest.java
+++ b/clickhouse-jdbc/src/test/java/ru/yandex/clickhouse/util/UtilsTest.java
@@ -4,6 +4,9 @@
import java.io.ByteArrayInputStream;
import java.io.DataInputStream;
+import java.nio.Buffer;
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
import org.testng.annotations.Test;
@@ -20,6 +23,28 @@ public void testUnsignedLeb128() throws Exception {
assertEquals(Utils.readUnsignedLeb128(input), 100000000);
}
+ @Test
+ public void testString() {
+ Charset charset = Charset.forName("ISO-8859-15");
+ byte[] b1 = new byte[] { (byte) 127, (byte) 128 };
+ String s = new String(b1, charset);
+ byte[] b2 = s.getBytes(charset);
+ assertEquals(b2, b1);
+ }
+
+ @Test
+ public void testVarInt() {
+ ByteBuffer buffer;
+ for (int i : new int[] { 0, 128, 255, 65535, 1023 * 1024 }) {
+ buffer = ByteBuffer.allocate(8);
+ Utils.writeVarInt(i, buffer);
+ Utils.writeVarInt(0 - i, buffer);
+ buffer = (ByteBuffer) ((Buffer) buffer.flip());
+ assertEquals(Utils.readVarInt(buffer), i);
+ assertEquals(Utils.readVarInt(buffer), 0 - i);
+ }
+ }
+
private DataInputStream prepareStream(byte[] input) {
return new DataInputStream(new ByteArrayInputStream(input));
}
diff --git a/pom.xml b/pom.xml
index ba33ac3d7..bde1bf428 100644
--- a/pom.xml
+++ b/pom.xml
@@ -79,9 +79,13 @@
UTF-8
UTF-8
1.8
+ 3.0.0
+ 3.4.2
+ 3.3.1
1.7.1
2.9.10
2.9.10.8
+ 0.9.9
1.7.30
1.10.19
2.27.2
@@ -123,11 +127,31 @@
jackson-databind
${jackson-databind.version}
+
+ com.github.ben-manes.caffeine
+ caffeine
+ ${caffeine.version}
+
+
+ com.lmax
+ disruptor
+ ${disruptor.version}
+
+
+ dnsjava
+ dnsjava
+ ${dnsjava.version}
+
org.lz4
lz4-java
${lz4.version}
+
+ com.github.RoaringBitmap
+ RoaringBitmap
+ ${roaring-bitmap.version}
+
org.slf4j
slf4j-api
@@ -377,4 +401,12 @@
+
+
+
+
+ jitpack.io
+ https://jitpack.io
+
+