JDBC connector guide
Overview
This guide is for Java developers building a BI tool connector, ETL integration, IDE plugin, or data catalog adapter that connects to ClickHouse via JDBC. It covers the full connector lifecycle: dependency setup, connection pooling, schema discovery, type mapping, query execution, batch inserts, and error handling.
This guide is opinionated. It documents the patterns that work reliably in production and calls out the known pitfalls in the current driver. It does not duplicate the JDBC API reference — for driver configuration options, supported settings, and version history, see the JDBC driver reference.
For language-agnostic connector patterns (HTTP API, format selection, observability, Cloud specifics), see:
Maven setup
The current driver artifact is com.clickhouse:clickhouse-jdbc. Use the all classifier to include all transitive dependencies in a single jar, which avoids classpath conflicts in environments like application servers and BI tool plugin runtimes.
Maven:
Gradle:
The transitive = false flag is required when using the all classifier in Gradle — otherwise Gradle also resolves the transitive dependency graph and you end up with duplicate classes on the classpath.
Do not use the legacy artifact ru.yandex.clickhouse:clickhouse-jdbc. It is unmaintained and does not support current ClickHouse features or ClickHouse Cloud.
Driver registration and connection URL
The driver class is com.clickhouse.jdbc.ClickHouseDriver. In environments that support JDBC 4.0 service discovery (most modern JVMs and frameworks), explicit registration is not required. Register it manually only if your container does not auto-discover drivers from the classpath:
URL format
Always use https:// for ClickHouse Cloud. For self-managed instances with TLS, use https://. For local development only, http:// is acceptable.
DriverManager example
Never embed credentials in the URL string. Connection URLs may appear in logs, error messages, and thread dumps. Read credentials from environment variables or a secrets manager.
client_name sets the http_user_agent field in system.query_log, which is how you identify your connector's queries when debugging customer issues.
Connection pooling with HikariCP
ClickHouse uses a stateless HTTP transport: each query is an independent HTTP request. There are no server-side sessions, no transaction state between requests, and no concept of a "connection" in the PostgreSQL or MySQL sense. What you are pooling is the underlying HTTP client and its keep-alive TCP connections — not logical database sessions.
This means connection pools behave well with ClickHouse, but some pool settings require tuning for the HTTP layer.
Recommended HikariCP configuration
Key settings explained
maximumPoolSize: 10 — For typical BI workloads where queries are issued serially per dashboard or per user session, 10 connections is sufficient. ClickHouse is fast at executing queries, so the limiting factor is usually query latency, not connection availability. Increase this only if profiling shows connection wait time.
maxLifetime: 270_000 (4.5 minutes) — This must be less than ClickHouse Cloud's HTTP keep-alive timeout (approximately 10 seconds idle at the load balancer level, but connections are cycled more aggressively). Setting maxLifetime to 270,000 ms ensures HikariCP retires connections before they become stale at the network layer. Without this, long-lived idle connections receive broken-pipe errors on the next use. Note: 270s is a conservative value; tune it down if you observe connection errors in production.
connectionTimeout: 30_000 (30 seconds) — ClickHouse Cloud services on the development tier auto-pause after inactivity. Waking a paused service can take several seconds. Setting a 30-second connection timeout prevents false failures on the first connection after a pause.
connectionTestQuery: SELECT 1 — HikariCP uses this to validate connections before handing them to your application. SELECT 1 is a no-op that returns in milliseconds and confirms the HTTP transport is alive.
dataSource vs jdbcUrl — Use setDataSource with an explicit ClickHouseDataSource instance (as shown above) rather than setJdbcUrl. The jdbcUrl path works but gives you less control over driver-level properties and makes it harder to pass Properties objects cleanly.
Schema discovery
ClickHouse exposes metadata via JDBC DatabaseMetaData and via system.* tables. Prefer system.* tables for any schema discovery that goes beyond basic enumeration — they carry ClickHouse-specific metadata that DatabaseMetaData does not expose, and they avoid the type-handling issues described below.
Listing databases
Via DatabaseMetaData:
Via direct SQL (preferred — filters system internals):
Listing tables
Via DatabaseMetaData:
Via direct SQL (preferred — includes engine and row-count metadata useful for BI tools):
Listing columns
DatabaseMetaData.getColumns() has two known issues with ClickHouse:
- Complex column types (
Array,Map,Tuple,Nested) can produceNullPointerExceptionin some driver versions when the type cannot be mapped to a standardjava.sql.Typesconstant. - The
TYPE_NAMEcolumn stripsNullable(...)andLowCardinality(...)wrappers, so you cannot determine nullability or encoding from the JDBC metadata alone.
Use system.columns instead:
The type column returns the full type declaration including all wrappers. Strip them in your type-mapping layer (see Stripping type modifiers).
getPrimaryKeys() limitation
DatabaseMetaData.getPrimaryKeys() returns approximate or empty results for ClickHouse tables. ClickHouse has no primary key constraint in the relational sense. The effective physical sort order is defined by the table's ORDER BY clause, which is stored in system.tables.sorting_key.
For accurate sort key discovery:
Type mapping
Stripping type modifiers
ClickHouse wraps base types in two modifiers that appear in type names returned by system.columns:
Nullable(T)— the column accepts NULL values.LowCardinality(T)— the column uses dictionary encoding for performance. Treat it as identical toTfor type-mapping purposes.
Both wrappers can be nested: Nullable(LowCardinality(String)) is valid. Strip them before looking up a Java type:
Full type mapping table
| ClickHouse type | Java class | Notes |
|---|---|---|
Int8 | Byte | |
Int16 | Short | |
Int32 | Integer | |
Int64 | Long | |
Int128, Int256 | BigInteger | |
UInt8 | Short | Promoted to avoid unsigned overflow |
UInt16 | Integer | Promoted |
UInt32 | Long | Promoted |
UInt64 | BigInteger | Values above Long.MAX_VALUE overflow silently if cast to long — always use BigInteger |
UInt128, UInt256 | BigInteger | |
Float32 | Float | |
Float64 | Double | |
Decimal(P, S) | BigDecimal | |
String | String | Raw byte sequence; UTF-8 by convention |
FixedString(N) | String | Zero-padded to N bytes — strip trailing \0 before display |
Date, Date32 | LocalDate | Use rs.getObject(col, LocalDate.class) — java.sql.Date applies the JVM timezone and silently shifts dates |
DateTime, DateTime64 | ZonedDateTime | Use rs.getObject(col, ZonedDateTime.class) — LocalDateTime loses timezone context |
UUID | String or UUID | |
Boolean | Boolean | |
Enum8, Enum16 | String | Returned as the enum label |
Array(T) | java.sql.Array | Call .getArray() to get a Java array; element type follows this mapping |
Map(K, V) | Object | Cast via rs.getObject(col); the concrete type is a Map |
Tuple(T1, T2, ...) | Object[] | |
IPv4, IPv6 | String | Dotted-decimal / colon-hex |
Handling Nullable and LowCardinality
For any column whose system.columns.type starts with Nullable(, always call rs.wasNull() after reading the value to detect actual NULLs:
LowCardinality columns are transparent at the JDBC layer. Read them the same way as their underlying type.
Executing queries
PreparedStatement parameters
Use PreparedStatement for all queries that incorporate user-supplied values. Never concatenate user input into SQL strings.
Pass LocalDate for Date/Date32 columns and ZonedDateTime for DateTime/DateTime64 columns. Using java.sql.Date or java.sql.Timestamp causes the JVM's default timezone to be applied, which produces incorrect values when the JVM timezone differs from the ClickHouse server timezone.
Streaming results
By default, the JDBC driver buffers the entire result set in memory before returning the first row. For large result sets, this causes high memory pressure and delays before the first row is visible to the caller.
To enable forward-only streaming mode:
For very large exports where memory overhead is unacceptable, consider using the HTTP API directly with JSONEachRow format and chunked transfer, which gives you full control over the streaming pipeline.
Query tagging
Tag every query with a query_id so it is traceable in system.query_log. This enables idempotent retries (reusing the same query_id on a retry causes ClickHouse to return the result of the already-running query rather than executing twice) and supports customer debugging workflows.
Use a deterministic ID format that encodes the operation type and a correlation token (e.g., a request ID from your framework). This makes it possible to look up a specific query without scanning the full log.
Batch inserts
PreparedStatement batch
Use PreparedStatement.addBatch() / executeBatch() for bulk inserts. Target 10,000–100,000 rows per batch. Smaller batches create too many data parts and trigger Too many parts errors; larger batches increase per-request memory pressure.
Critical: create a new PreparedStatement per batch
In JDBC driver versions 0.8.6 and above, reusing a PreparedStatement across multiple executeBatch() calls re-inserts all rows from every previous batch in addition to the current one. The internal batch buffer is not cleared between executions.
Always create a new PreparedStatement instance for each batch cycle. The try-with-resources pattern enforces this naturally:
Do not cache PreparedStatement objects for INSERT across batch calls. Caching is safe for SELECT statements, but not for INSERT batches until this driver bug is resolved.
Async insert via JDBC
async_insert=1 buffers rows server-side and flushes in bulk, which lets you send small batches without causing a part explosion. Enable it as a connection property:
wait_for_async_insert=1 makes the server respond only after the buffered data has been flushed and validated. Without it (wait_for_async_insert=0), the server returns immediately and type errors in the buffered batch are silently discarded with no error surfaced to the client. Use wait_for_async_insert=1 in any pipeline where data loss must be detectable.
One syntax constraint: the SETTINGS clause in an INSERT must appear before VALUES. The driver constructs this correctly when you use PreparedStatement, but if you are building INSERT strings manually, verify the clause ordering:
Error handling
ClickHouse exception codes
The JDBC driver wraps all ClickHouse errors as java.sql.SQLException. The ClickHouse error code is available via e.getErrorCode(). These codes are stable integers suitable for programmatic handling — do not parse the error message string.
| Code | Constant name | Notes |
|---|---|---|
| 60 | UNKNOWN_TABLE | Table does not exist — do not retry |
| 81 | READONLY | Transient: usually ZooKeeper connectivity loss — retry with backoff |
| 159 | TIMEOUT_EXCEEDED | Query exceeded max_execution_time — do not retry without modifying the query |
| 241 | MEMORY_LIMIT_EXCEEDED | Query exceeded memory limit — do not retry without modifying the query |
| 164 | READONLY_SETTING | Attempt to set a readonly server setting |
| 516 | AUTHENTICATION_FAILED | Wrong credentials — do not retry |
Retry strategy
Retry on:
java.net.SocketException,java.net.SocketTimeoutException— network-level failures- Error code
81(READONLY) — transient ZooKeeper issue - HTTP 500 responses where the exception code is not in the do-not-retry list
Do not retry on:
- Error code
60(UNKNOWN_TABLE) — the schema is wrong - Error codes for syntax errors, access denied, or memory/timeout limits — retrying will produce the same outcome
Use exponential backoff with jitter. On INSERT retries, reuse the same query_id or set insert_deduplication_token so ClickHouse can deduplicate:
ClickHouse Cloud specifics
Always use jdbc:ch:https:// — port 8123 (plaintext HTTP) is not available on ClickHouse Cloud. Attempts to connect on port 8123 will fail with a connection refused or TLS error. The correct port is 8443.
Auto-pause — Development-tier services pause after a period of inactivity. The first connection after a pause can take several seconds while the service wakes. Set connectionTimeout=30000 in your HikariCP config (as shown above) and implement a retry on the first connection attempt before surfacing an error to the user.
Single endpoint, multiple nodes — A ClickHouse Cloud endpoint hides a multi-node cluster behind a load balancer. Do not assume one endpoint equals one server. Session-level settings (SET ...) do not persist across requests because consecutive requests may land on different nodes. Pass settings as query parameters or connection properties instead.
sslmode=strict — The default and the correct setting for Cloud. Validates the server certificate chain. Never set sslmode=none in any user-facing or production integration.
Migration from V1 to V2
The V2 API (com.clickhouse:clickhouse-jdbc 0.6+) is the current default. V1 (0.3.x / 0.4.x) is legacy and no longer receives fixes. This section covers the breaking changes you will encounter when migrating connector code written against V1.
Breaking type changes
| Column type | V1 return type | V2 return type |
|---|---|---|
Date, Date32 | ZonedDateTime | LocalDate |
Array(T) | List<?> | java.sql.Array |
Tuple(T1, T2, ...) | List<Object> | Object[] |
Update all ResultSet.getObject() call sites that handle these types. Schema discovery code that inspects type names at the JDBC metadata level is generally unaffected.
Unknown configuration keys now throw
In V1, unrecognized properties passed to DriverManager.getConnection() were silently ignored. In V2, they throw ClientMisconfigurationException. During migration, set the following property to suppress this while you audit your configuration:
Remove this once you have validated that all property keys are intentional.
JDBC vs client property separation
V2 distinguishes between JDBC-layer properties (e.g., ssl, sslmode) and ClickHouse client properties. In V1, these were mixed together. In V2, JDBC properties must be set on the Properties object passed to DriverManager or DataSource, while server-level settings use the clickhouse_setting_ prefix:
Mixing them in the wrong way was tolerated in V1 but will throw in V2 unless ignore_unknown_config_key=true is set.