Skip to content

Commit

Permalink
Merge pull request #747 from zhicwu/new-jdbc
Browse files Browse the repository at this point in the history
New JDBC driver
  • Loading branch information
zhicwu authored Nov 28, 2021
2 parents c4b411b + 1b4b1d3 commit e64fd4b
Show file tree
Hide file tree
Showing 185 changed files with 22,421 additions and 3,708 deletions.
2 changes: 1 addition & 1 deletion README.md
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ try (ClickHouseClient client = ClickHouseClient.newInstance(ClickHouseProtocol.G
ClickHouseResponse resp = client.connect(server)
.format(ClickHouseFormat.RowBinaryWithNamesAndTypes).set("send_logs_level", "trace")
.query("select id, name from some_table where id in :ids and name like :name").params(Arrays.asList(1,2,3), "%key%").execute().get()) {
// you can also use resp.recordStream() as well
// you can also use resp.stream() as well
for (ClickHouseRecord record : resp.records()) {
int id = record.getValue(0).asInteger();
String name = record.getValue(1).asString();
Expand Down
46 changes: 11 additions & 35 deletions clickhouse-benchmark/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,7 @@

<properties>
<clickhouse4j-driver.version>1.4.4</clickhouse4j-driver.version>
<mariadb-driver.version>2.7.3</mariadb-driver.version>
<mysql-driver.version>8.0.26</mysql-driver.version>
<native-driver.version>2.5.6</native-driver.version>
<postgresql-driver.version>42.2.23</postgresql-driver.version>
<native-driver.version>2.6.0</native-driver.version>
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
<jmh.version>1.33</jmh.version>
<shade.name>benchmarks</shade.name>
Expand Down Expand Up @@ -67,10 +64,10 @@

<!-- JDBC drivers -->
<dependency>
<groupId>ru.yandex.clickhouse</groupId>
<groupId>com.clickhouse</groupId>
<artifactId>clickhouse-jdbc</artifactId>
<version>${revision}</version>
<classifier>shaded</classifier>
<classifier>all</classifier>
<exclusions>
<exclusion>
<groupId>*</groupId>
Expand All @@ -79,9 +76,9 @@
</exclusions>
</dependency>
<dependency>
<groupId>cc.blynk.clickhouse</groupId>
<artifactId>clickhouse4j</artifactId>
<version>${clickhouse4j-driver.version}</version>
<groupId>com.github.housepower</groupId>
<artifactId>clickhouse-native-jdbc-shaded</artifactId>
<version>${native-driver.version}</version>
<exclusions>
<exclusion>
<groupId>*</groupId>
Expand All @@ -90,9 +87,9 @@
</exclusions>
</dependency>
<dependency>
<groupId>org.mariadb.jdbc</groupId>
<artifactId>mariadb-java-client</artifactId>
<version>${mariadb-driver.version}</version>
<groupId>cc.blynk.clickhouse</groupId>
<artifactId>clickhouse4j</artifactId>
<version>${clickhouse4j-driver.version}</version>
<exclusions>
<exclusion>
<groupId>*</groupId>
Expand All @@ -103,35 +100,14 @@
<dependency>
<groupId>mysql</groupId>
<artifactId>mysql-connector-java</artifactId>
<version>${mysql-driver.version}</version>
<exclusions>
<exclusion>
<groupId>*</groupId>
<artifactId>*</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>com.github.housepower</groupId>
<artifactId>clickhouse-native-jdbc-shaded</artifactId>
<version>${native-driver.version}</version>
<exclusions>
<exclusion>
<groupId>*</groupId>
<artifactId>*</artifactId>
</exclusion>
</exclusions>
<groupId>org.mariadb.jdbc</groupId>
<artifactId>mariadb-java-client</artifactId>
</dependency>
<dependency>
<groupId>org.postgresql</groupId>
<artifactId>postgresql</artifactId>
<version>${postgresql-driver.version}</version>
<exclusions>
<exclusion>
<groupId>*</groupId>
<artifactId>*</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.testcontainers</groupId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@
import com.clickhouse.client.ClickHouseResponse;
import com.clickhouse.client.ClickHouseValue;
import com.clickhouse.client.config.ClickHouseClientOption;
import com.clickhouse.client.grpc.config.ClickHouseGrpcClientOption;
import com.clickhouse.client.grpc.config.ClickHouseGrpcOption;

@State(Scope.Thread)
public class ClientState extends BaseState {
Expand Down Expand Up @@ -57,24 +57,25 @@ private ClickHouseClient createClient() {

ClickHouseClientBuilder builder = ClickHouseClient.builder();
if (bufferSize != null && !bufferSize.isEmpty()) {
builder.addOption(ClickHouseClientOption.MAX_BUFFER_SIZE, Integer.parseInt(bufferSize));
builder.option(ClickHouseClientOption.MAX_BUFFER_SIZE, Integer.parseInt(bufferSize));
}
if (compression != null && !compression.isEmpty()) {
builder.addOption(ClickHouseClientOption.COMPRESSION, compression.toUpperCase());
// builder.option(ClickHouseClientOption.COMPRESSION,
// compression.toUpperCase());
if (ClickHouseCompression.NONE.name().equalsIgnoreCase(compression)) {
builder.addOption(ClickHouseGrpcClientOption.USE_FULL_STREAM_DECOMPRESSION, true);
builder.option(ClickHouseGrpcOption.USE_FULL_STREAM_DECOMPRESSION, true);
}
}
if (threads != null && !threads.isEmpty()) {
builder.addOption(ClickHouseClientOption.MAX_THREADS_PER_CLIENT, Integer.parseInt(threads));
builder.option(ClickHouseClientOption.MAX_THREADS_PER_CLIENT, Integer.parseInt(threads));
}

if (window != null && !window.isEmpty()) {
builder.addOption(ClickHouseGrpcClientOption.FLOW_CONTROL_WINDOW, Integer.parseInt(window));
builder.option(ClickHouseGrpcOption.FLOW_CONTROL_WINDOW, Integer.parseInt(window));
}

return builder.addOption(ClickHouseClientOption.ASYNC, "async".equals(mode))
.addOption(ClickHouseGrpcClientOption.USE_OKHTTP, "okhttp".equals(transport)).build();
return builder.option(ClickHouseClientOption.ASYNC, "async".equals(mode))
.option(ClickHouseGrpcOption.USE_OKHTTP, "okhttp".equals(transport)).build();
}

@Setup(Level.Trial)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,8 @@

@State(Scope.Thread)
public class DriverState extends BaseState {
@Param(value = { "clickhouse4j", "clickhouse-jdbc", "clickhouse-native-jdbc-shaded", "mariadb-java-client",
"mysql-connector-java", "postgresql-jdbc" })
@Param(value = { "clickhouse4j", "clickhouse-http-jdbc", "clickhouse-grpc-jdbc", "clickhouse-jdbc",
"clickhouse-native-jdbc-shaded", "mariadb-java-client", "mysql-connector-java", "postgresql-jdbc" })
private String client;

@Param(value = { Constants.REUSE_CONNECTION, Constants.NEW_CONNECTION })
Expand All @@ -38,11 +38,13 @@ public class DriverState extends BaseState {
public void doSetup(ServerState serverState) throws Exception {
JdbcDriver jdbcDriver = JdbcDriver.from(client);

String compression = String.valueOf(Boolean.parseBoolean(System.getProperty("compression", "true")));

try {
driver = (java.sql.Driver) Class.forName(jdbcDriver.getClassName()).getDeclaredConstructor().newInstance();
url = String.format(jdbcDriver.getUrlTemplate(), serverState.getHost(),
serverState.getPort(jdbcDriver.getDefaultPort()), serverState.getDatabase(), serverState.getUser(),
serverState.getPassword());
serverState.getPassword(), compression);
conn = driver.connect(url, new Properties());

try (Statement s = conn.createStatement()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5,32 +5,38 @@
public enum JdbcDriver {
// ClickHouse4j
Clickhouse4j("cc.blynk.clickhouse.ClickHouseDriver",
"jdbc:clickhouse://%s:%s/%s?ssl=false&user=%s&password=%s&use_server_time_zone=false&use_time_zone=UTC",
"jdbc:clickhouse://%s:%s/%s?ssl=false&user=%s&password=%s&use_server_time_zone=false&use_time_zone=UTC&compress=%s",
Constants.HTTP_PORT),
// ClickHouse JDBC Driver
ClickhouseHttpJdbc("com.clickhouse.jdbc.ClickHouseDriver",
"jdbc:ch://%s:%s/%s?ssl=false&user=%s&password=%s&use_server_time_zone=false&use_time_zone=UTC&compress=%s",
Constants.HTTP_PORT),
ClickhouseGrpcJdbc("com.clickhouse.jdbc.ClickHouseDriver",
"jdbc:ch:grpc://%s:%s/%s?ssl=false&user=%s&password=%s&use_server_time_zone=false&use_time_zone=UTC&compress=%s",
Constants.GRPC_PORT),
ClickhouseJdbc("ru.yandex.clickhouse.ClickHouseDriver",
"jdbc:clickhouse://%s:%s/%s?ssl=false&user=%s&password=%s&use_server_time_zone=false&use_time_zone=UTC",
"jdbc:clickhouse://%s:%s/%s?ssl=false&user=%s&password=%s&use_server_time_zone=false&use_time_zone=UTC&compress=%s",
Constants.HTTP_PORT),
// ClickHouse Native JDBC Driver
ClickhouseNativeJdbcShaded("com.github.housepower.jdbc.ClickHouseDriver",
"jdbc:clickhouse://%s:%s/%s?ssl=false&user=%s&password=%s&use_server_time_zone=false&use_time_zone=UTC",
"jdbc:clickhouse://%s:%s/%s?ssl=false&user=%s&password=%s&use_server_time_zone=false&use_time_zone=UTC&compress=%s",
Constants.NATIVE_PORT),

// MariaDB Java Client
MariadbJavaClient("org.mariadb.jdbc.Driver",
"jdbc:mariadb://%s:%s/%s?user=%s&password=%s&useSSL=false&useCompression=true&useServerPrepStmts=false"
"jdbc:mariadb://%s:%s/%s?user=%s&password=%s&useSSL=false&useServerPrepStmts=false&useCompression=%s"
+ "&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&useServerPrepStmts=false"
+ "&rewriteBatchedStatements=true&cachePrepStmts=true&connectionTimeZone=UTC&useCompression=%s",
Constants.MYSQL_PORT),

// PostgreSQL JDBC Driver
PostgresqlJdbc("org.postgresql.Driver",
"jdbc:postgresql://%s:%s/%s?user=%s&password=%s&ssl=false&sslmode=disable&preferQueryMode=simple",
"jdbc:postgresql://%s:%s/%s?user=%s&password=%s&ssl=false&sslmode=disable&preferQueryMode=simple&compress=%s",
Constants.POSTGRESQL_PORT);

private final String className;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,7 @@ public void selectDateTime32Rows(Blackhole blackhole, DriverState state) throws
int num = state.getRandomNumber();
int rows = state.getSampleSize() + num;
try (Statement stmt = executeQuery(state,
"select toDateTime('2021-02-20 13:15:20') + number as d from system.numbers limit ?", rows)) {
"select toDateTime32(1613826920 + number) as d from system.numbers limit ?", rows)) {
ResultSet rs = stmt.getResultSet();
while (rs.next()) {
blackhole.consume(rs.getTimestamp(1));
Expand All @@ -24,20 +24,32 @@ public void selectDateTime64Rows(Blackhole blackhole, DriverState state) throws
int num = state.getRandomNumber();
int rows = state.getSampleSize() + num;
try (Statement stmt = executeQuery(state,
"select toDateTime64('2021-02-20 13:15:20.000000000', 9) + number as d from system.numbers limit ?",
rows)) {
"select toDateTime64(1613826920 + number / 1000000000, 9) as d from system.numbers limit ?", rows)) {
ResultSet rs = stmt.getResultSet();
while (rs.next()) {
blackhole.consume(rs.getTimestamp(1));
}
}
}

@Benchmark
public void selectDateTime64ObjectRows(Blackhole blackhole, DriverState state) throws Throwable {
int num = state.getRandomNumber();
int rows = state.getSampleSize() + num;
try (Statement stmt = executeQuery(state,
"select toDateTime64(1613826920 + number / 1000000000, 9) as d from system.numbers limit ?", rows)) {
ResultSet rs = stmt.getResultSet();
while (rs.next()) {
blackhole.consume(rs.getObject(1));
}
}
}

@Benchmark
public void selectInt32Rows(Blackhole blackhole, DriverState state) throws Throwable {
int num = state.getRandomNumber();
int rows = state.getSampleSize() + num;
try (Statement stmt = executeQuery(state, "select * from system.numbers limit ?", rows)) {
try (Statement stmt = executeQuery(state, "select toInt32(number) from system.numbers limit ?", rows)) {
ResultSet rs = stmt.getResultSet();
while (rs.next()) {
blackhole.consume(rs.getInt(1));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,6 @@
import com.clickhouse.client.data.ClickHouseLongValue;
import com.clickhouse.client.data.ClickHouseShortValue;
import com.clickhouse.client.data.ClickHouseStringValue;
import com.clickhouse.client.data.ClickHouseTimeValue;

@State(Scope.Benchmark)
@Warmup(iterations = 10, timeUnit = TimeUnit.SECONDS, time = 1)
Expand Down Expand Up @@ -89,7 +88,7 @@ public void setupSamples() {

// add(map, list, Object[].class, () -> ClickHouseArrayValue.of((Object[]) o));
add(map, list, LocalDate.class, () -> ClickHouseDateValue.ofNull());
add(map, list, LocalTime.class, () -> ClickHouseTimeValue.ofNull());
add(map, list, LocalTime.class, () -> ClickHouseDateTimeValue.ofNull(0));
add(map, list, LocalDateTime.class, () -> ClickHouseDateTimeValue.ofNull(0));
add(map, list, String.class, () -> ClickHouseStringValue.ofNull());

Expand Down Expand Up @@ -130,7 +129,7 @@ ClickHouseValue newValue(Class<?> clazz) {
} else if (LocalDate.class.equals(clazz)) {
return ClickHouseDateValue.ofNull();
} else if (LocalTime.class.equals(clazz)) {
return ClickHouseTimeValue.ofNull();
return ClickHouseDateTimeValue.ofNull(0);
} else if (LocalDateTime.class.equals(clazz)) {
return ClickHouseDateTimeValue.ofNull(0);
} else if (String.class.equals(clazz)) {
Expand Down
80 changes: 37 additions & 43 deletions clickhouse-client/README.md
Original file line number Diff line number Diff line change
@@ -1,27 +1,22 @@
# ClickHouse Java Client

Async Java client for ClickHouse. `clickhouse-client` is an abstract module, so it does not work by itself unless being used together with implementation module like `ckhouse-grpc-client` or `clickhouse-http-client`.
Async Java client for ClickHouse. `clickhouse-client` is an abstract module, so it does not work by itself until being used together with an implementation like `clickhouse-grpc-client` or `clickhouse-http-client`.

## Quick Start

```java
import java.time.LocalDateTime;
import java.util.concurrent.CompletableFuture;
import java.util.List;

import com.clickhouse.client.ClickHouseClient;
import com.clickhouse.client.ClickHouseCluster;
import com.clickhouse.client.ClickHouseNode;
import com.clickhouse.client.ClickHouseProtocol;
import com.clickhouse.client.ClickHouseRecord;
import com.clickhouse.client.ClickHouseResponse;
import com.clickhouse.client.ClickHouseResponseSummary;
import com.clickhouse.client.ClickHouseValue;
```xml
<dependency>
<groupId>com.clickhouse</groupId>
<artifactId>clickhouse-http-client</artifactId>
<version>0.3.2</version>
</dependency>
```

```java
// declare a server to connect to
ClickHouseNode server = ClickHouseNode.of("server1.domain", ClickHouseProtocol.GRPC, 9100, "my_db");
ClickHouseNode server = ClickHouseNode.of("server1.domain", ClickHouseProtocol.HTTP, 8123, "my_db");

// execute multiple queries one after another within one session
// execute multiple queries in a worker thread one after another within same session
CompletableFuture<List<ClickHouseResponseSummary>> future = ClickHouseClient.send(server,
"create database if not exists test",
"use test", // change current database from my_db to test
Expand All @@ -30,33 +25,32 @@ CompletableFuture<List<ClickHouseResponseSummary>> future = ClickHouseClient.sen
"select * from test_table limit 1",
"truncate table test_table",
"drop table if exists test_table");
// do something else in current thread, and then retrieve summaries
List<ClickHouseResponseSummary> results = future.get();

// declare a cluster
ClickHouseCluster cluster = ClickHouseCluster.builder()
// defaults to localhost:8123 and http protocol
.addNode(ClickHouseNode.builder().cluster("cluster1").tags("dc1", "rack1", "for-write").build())
.addNode(ClickHouseNode.of("1.2.3.4", ClickHouseProtocol.GRPC, 9100, "system", "dc2", "rack2", "for-read"))
.build();

// issue query against one node via grpc
String sql = "select * from numbers(100)";
try (ClickHouseClient client = ClickHouseClient.newInstance(ClickHouseProtocol.GRPC);
// connect to a node which understands gRPC
ClickHouseResponse response = client.connect(cluster).query(sql).execute().get()) {
for (ClickHouseRecord record : response.records()) {
// Don't cache ClickHouseValue as it's a container object reused among all records
ClickHouseValue v = record.getValue(0);
// converts to DateTime64(6)
LocalDateTime dateTime = v.asDateTime(6);
// converts to long/int/byte if you want to
long l = v.asLong();
int i = v.asInteger();
byte b = v.asByte();
}

// summary will be fully available after all records being retrieved
ClickHouseResponseSummary summary = response.getSummary();
// block current thread until queries completed, and then retrieve summaries
// List<ClickHouseResponseSummary> results = future.get();

try (ClickHouseClient client = ClickHouseClient.newInstance(server.getProtocol)) {
ClickHouseRequest<?> request = client.connect(server).format(ClickHouseFormat.RowBinaryWithNamesAndTypes);
// load data into a table and wait until it's completed
request.write().query("insert into my_table select c2, c3 from input('c1 UInt8, c2 String, c3 Int32')")
.data(myInputStream).execute().thenAccept(response -> {
response.close();
});

// query with named parameter
try (ClickHouseResponse response = request.query(ClickHouseParameterizedQuery.of(
"select * from numbers(:limit)")).params(100000).execute().get()) {
for (ClickHouseRecord r : response.records()) {
// Don't cache ClickHouseValue / ClickHouseRecord as they're reused for
// corresponding column / row
ClickHouseValue v = r.getValue(0);
// converts to DateTime64(6)
LocalDateTime dateTime = v.asDateTime(6);
// converts to long/int/byte if you want to
long l = v.asLong();
int i = v.asInteger();
byte b = v.asByte();
}
}
}
```
Loading

0 comments on commit e64fd4b

Please sign in to comment.