Skip to main content

Java Client (V2)

Java client library to communicate with a DB server through its protocols. The current implementation only supports the HTTP interface. The library provides its own API to send requests to a server. The library also provides tools to work with different binary data formats (RowBinary & Native).

Setup

<dependency>
<groupId>com.clickhouse</groupId>
<artifactId>client-v2</artifactId>
<version>0.7.1</version>
</dependency>

Initialization

The Client object is initialized by com.clickhouse.client.api.Client.Builder#build(). Each client has its own context and no objects are shared between them. The Builder has configuration methods for convenient setup.

Example:

 Client client = new Client.Builder()
.addEndpoint("https://clickhouse-cloud-instance:8443/")
.setUsername(user)
.setPassword(password)
.build();

Client is AutoCloseable and should be closed when not needed anymore.

Authentication

Authentication is configured per client at the initialization phase. There are three authentication methods supported: by password, by access token, by SSL Client Certificate.

Authentication by a password requires setting user name password by calling setUsername(String) and setPassword(String):

 Client client = new Client.Builder()
.addEndpoint("https://clickhouse-cloud-instance:8443/")
.setUsername(user)
.setPassword(password)
.build();

Authentication by an access token requires setting access token by calling setAccessToken(String):

 Client client = new Client.Builder()
.addEndpoint("https://clickhouse-cloud-instance:8443/")
.setAccessToken(userAccessToken)
.build();

Authentication by a SSL Client Certificate require setting username, enabling SSL Authentication, setting a client sertificate and a client key by calling setUsername(String), useSSLAuthentication(boolean), setClientCertificate(String) and setClientKey(String) accordingly:

Client client = new Client.Builder()
.useSSLAuthentication(true)
.setUsername("some_user")
.setClientCertificate("some_user.crt")
.setClientKey("some_user.key")
Note

SSL Authentication may be hard to troubleshoot on production because many errors from SSL libraries provide not enough information. For example, if client certificate and key do not match then server will terminate connection immediately (in case of HTTP it will be connection initiation stage where no HTTP requests are send so no response is sent).

Please use tools like openssl to verify certificates and keys:

  • check key integrity: openssl rsa -in [key-file.key] -check -noout
  • check client certificate has matching CN for a user:
    • get CN from an user certificate - openssl x509 -noout -subject -in [user.cert]
    • verify same value is set in database select name, auth_type, auth_params from system.users where auth_type = 'ssl_certificate' (query will output auth_params with something like {"common_names":["some_user"]})

Configuration

All settings are defined by instance methods (a.k.a configuration methods) that make the scope and context of each value clear. Major configuration parameters are defined in one scope (client or operation) and do not override each other.

Configuration is defined during client creation. See com.clickhouse.client.api.Client.Builder.

Client Configuration

Configuration MethodArgumentsDescription
addEndpoint(String endpoint)- enpoint - URL formatted a server address.Adds a server endpoint to list of available servers. Currently only one endpoint is supported.
addEndpoint(Protocol protocol, String host, int port, boolean secure)- protocol - connection protocol com.clickhouse.client.api.enums.Protocol#HTTP.
- host - IP or hostname of a server.
- secure - if communication should use secure version of the protocol (HTTPs)
Adds a server endpoint to list of available servers. Currently only one endpoint is supported.
setOption(String key, String value)- key - String key of the client configuration option.
- value - String value of the option
Sets raw value of client options. Useful when reading configuration from properties files.
setUsername(String username)- username - User's username to use while authenticationSets username for an authentication method that is selected by further configuration
setPassword(String password)- password - secret value for password authenticationSets a secret for password authentication and effectively selects as authentication method
setAccessToken(String accessToken)- accessToken - String representation of an access tokenSets an access token to authenticate witha sets corresponding authentication method
useSSLAuthentication(boolean useSSLAuthentication)- useSSLAuthentication - flag that indicates if SSL auth should be usedSets SSL Client Certificate as an authentication method
enableConnectionPool(boolean enable)- enable - flag that indicates if the option should be enabledSets if a connection pool is enabled
setConnectTimeout(long timeout, ChronoUnit unit)- timeout - timeout in some time unit.
- unit - time unit of the timeout
Sets connection initiation timeout for any outgoing connection. This affects time wait on getting socket connect.
setConnectionRequestTimeout(long timeout, ChronoUnit unit)- timeout - timeout in some time unit.
- unit - time unit of the timeout
Sets connection request timeout. This take effect only for getting connection from a pool.
setMaxConnections(int maxConnections)- maxConnections - number of connectionsSets how many connections can a client open to each server endpoint.
setConnectionTTL(long timeout, ChronoUnit unit)- timeout - timeout in some time unit.
- unit - time unit of the timeout
Sets connection TTL after which connection will be considered as not active
setKeepAliveTimeout(long timeout, ChronoUnit unit)- timeout - timeout in some time unit.
- unit - time unit of the timeout
Sets HTTP connection keep-alive timeout. This option may be used to disable Keep-Alive by setting timeout to zero - 0
setConnectionReuseStrategy(ConnectionReuseStrategy strategy)- strategy - enum com.clickhouse.client.api.ConnectionReuseStrategy constantSelects which strategy connection pool should use: LIFO if connection should be reused as soon as they are returned to a pool or FIFO to use connection in the order they become available (returned connection are not used immediately).
setSocketTimeout(long timeout, ChronoUnit unit)- timeout - timeout in some time unit.
- unit - time unit of the timeout
Sets socket timeout that affects read and write operations
setSocketRcvbuf(long size)- size - size in bytesSets TCP socket receive buffer. This buffer out of the JVM memory.
setSocketSndbuf(long size)- size - size in bytesSets TCP socket receive buffer. This buffer out of the JVM memory.
setSocketKeepAlive(boolean value)- value - flag that indicates if option should be enabled.Sets option SO_KEEPALIVE for every TCP socket created by the client. TCP Keep Alive enables mechanism that will check liveness of the connection and will help to detect abruptly terminated ones.
setSocketTcpNodelay(boolean value)- value - flag that indicates if option should be enabled.Sets option SO_NODELAY for every TCP socket created by the client. This TCP option will make socket to push data as soon as possible.
setSocketLinger(int secondsToWait)- secondsToWait - number of seconds.Set linger time for every TCP socket created by the client.
compressServerResponse(boolean enabled)- enabled - flag that indicates if the option should be enabledSets if server should compress its responses.
compressClientRequest(boolean enabled)- enabled - flag that indicates if the option should be enabledSets if client should compress its requests.
useHttpCompression(boolean enabled)- enabled - flag that indicates if the option should be enabledSets if HTTP compression should be used for client/server communications if corresponding options are enabled
setLZ4UncompressedBufferSize(int size)- size - size in bytesSets size of a buffer that will receive uncompressed portion of a data stream. If buffer is underestimated - a new one will be created and corresponding warning will be present in logs.
setDefaultDatabase(String database)- database - name of a databaseSets default database.
addProxy(ProxyType type, String host, int port)- type - proxy type.
- host - proxy host name or IP Address.
- port - proxy port
Sets proxy to be used for communication with a server. Setting proxy is required if proxy requires authentication.
setProxyCredentials(String user, String pass)- user - proxy username.
- pass - password
Sets user credentials to authenticate with a proxy.
setExecutionTimeout(long timeout, ChronoUnit timeUnit)- timeout - timeout in some time unit.
- timeUnit - time unit of the timeout
Sets maximum execution timeout for queries
setHttpCookiesEnabled(boolean enabled)enabled - flag that indicates if the option should be enabledSet if HTTP cookies should be remembered and sent to server back.
setSSLTrustStore(String path)path - file path on local (client side) systemSets if client should use SSL truststore for server host validation.
setSSLTrustStorePassword(String password)password - secret valueSets password to be used to unlock SSL truststore specified by setSSLTrustStore(String path)
setSSLTrustStoreType(String type)type - truststore type nameSets type of the truststore specified by setSSLTrustStore(String path).
setRootCertificate(String path)path - file path on local (client side) systemSets if client should use specified root (CA) certificate for server host to validation.
setClientCertificate(String path)path - file path on local (client side) systemSets client certificate path to be used while initiating SSL connection and to be used by SSL authentication
setClientKey(String path)path - file path on local (client side) systemSets client private key to be used for encrypting SSL communication with a server.
useServerTimeZone(boolean useServerTimeZone)useServerTimeZone - flag that indicates if the option should be enabledSets if client should use server timezone when decoding DateTime and Date column values. If enabled then server timezone should be set by setServerTimeZone(String timeZone)
useTimeZone(String timeZone)timeZone - string value of java valid timezone ID (see java.time.ZoneId)Sets if specified timezone should be used when decoding DateTime and Date column values. Will override server timezone
setServerTimeZone(String timeZone)timeZone - string value of java valid timezone ID (see java.time.ZoneId)Sets server side timezone. UTC timezone will be used by default.
useAsyncRequests(boolean async)async - flag that indicates if the option should be enabled.Sets if client should execute request in a separate thread. Disabled by default because application knows better how to organize multithreaded tasks and running tasks in separate thread do not help with performance.
setSharedOperationExecutor(ExecutorService executorService)executorService - instance of executor service.Sets executor service for operation tasks.
setClientNetworkBufferSize(int size)- size - size in bytesSets size of a buffer in application memory space that is used to copy data back-and-forth between socket and application. Greater reduces system calls to TCP stack, but affects how much memory is spent on every connection. This buffer is also subject for GC because connections are shortlive. Also keep in mind that allocating big continious block of memory might be a problem.
retryOnFailures(ClientFaultCause ...causes)- causes - enum constant of com.clickhouse.client.api.ClientFaultCauseSets recoverable/retriable fault types.
setMaxRetries(int maxRetries)- maxRetries - number of retriesSets maximum number of retries for failures defined by retryOnFailures(ClientFaultCause ...causes)
allowBinaryReaderToReuseBuffers(boolean reuse)- reuse - flag that indicates if the option should be enabledMost datasets contain numeric data encoded as small byte sequences. By default reader will allocate required buffer, read data into it and then transform into a target Number class. That may cause significant GC preasure because of many small objects are being allocated and released. If this option is enabled then reader will use preallocated buffers to do numbers transcoding. It is safe because each reader has own set of buffers and readers are used by one thread.
httpHeader(String key, String value)- key - HTTP header key.
- value - string value of the header.
Sets value for a single HTTP header. Previous value is overriden.
httpHeader(String key, Collection values)- key - HTTP header key.
- values - list of string values.
Sets values for a single HTTP header. Previous value is overriden.
httpHeaders(Map headers)- header - map with HTTP headers and their values.Sets multiple HTTP header values at a time.
serverSetting(String name, String value)- name - name of a query level setting.
- value - string value of the setting.
Sets what settings to pass to server along with each query. Individual operation settings may override it. The List of settings
serverSetting(String name, Collection values)- name - name of a query level setting.
- values - string values of the setting.
Sets what settings to pass to server along with each query. Individual operation settings may override it. The List of settings. This method is useful to set settings with multiple values, for example roles
columnToMethodMatchingStrategy(ColumnToMethodMatchingStrategy strategy)- strategy - implementation of a column-field matching strategySets custom strategy to be used for matching DTO class fields and DB columns when registering DTO.
useHTTPBasicAuth(boolean useBasicAuth)- useBasicAuth - flag that indicates if the option should be enabledSets if basic HTTP authentication should be used for user-password authentication. Default is enabled. Using this type of authentication resolves issues with passwords containing special characters that cannot be transferred over HTTP headers.

Common Definitions

ClickHouseFormat

Enum of supported formats. It includes all formats that ClickHouse supports.

  • raw - user should transcode raw data
  • full - the client can transcode data by itself and accepts a raw data stream
  • - - operation not supported by ClickHouse for this format

This client version supports:

FormatInputOutput
TabSeparatedrawraw
TabSeparatedRawrawraw
TabSeparatedWithNamesrawraw
TabSeparatedWithNamesAndTypesrawraw
TabSeparatedRawWithNamesrawraw
TabSeparatedRawWithNamesAndTypesrawraw
Templaterawraw
TemplateIgnoreSpacesraw-
CSVrawraw
CSVWithNamesrawraw
CSVWithNamesAndTypesrawraw
CustomSeparatedrawraw
CustomSeparatedWithNamesrawraw
CustomSeparatedWithNamesAndTypesrawraw
SQLInsert-raw
Valuesrawraw
Vertical-raw
JSONrawraw
JSONAsStringraw-
JSONAsObjectraw-
JSONStringsrawraw
JSONColumnsrawraw
JSONColumnsWithMetadatarawraw
JSONCompactrawraw
JSONCompactStrings-raw
JSONCompactColumnsrawraw
JSONEachRowrawraw
PrettyJSONEachRow-raw
JSONEachRowWithProgress-raw
JSONStringsEachRowrawraw
JSONStringsEachRowWithProgress-raw
JSONCompactEachRowrawraw
JSONCompactEachRowWithNamesrawraw
JSONCompactEachRowWithNamesAndTypesrawraw
JSONCompactStringsEachRowrawraw
JSONCompactStringsEachRowWithNamesrawraw
JSONCompactStringsEachRowWithNamesAndTypesrawraw
JSONObjectEachRowrawraw
BSONEachRowrawraw
TSKVrawraw
Pretty-raw
PrettyNoEscapes-raw
PrettyMonoBlock-raw
PrettyNoEscapesMonoBlock-raw
PrettyCompact-raw
PrettyCompactNoEscapes-raw
PrettyCompactMonoBlock-raw
PrettyCompactNoEscapesMonoBlock-raw
PrettySpace-raw
PrettySpaceNoEscapes-raw
PrettySpaceMonoBlock-raw
PrettySpaceNoEscapesMonoBlock-raw
Prometheus-raw
Protobufrawraw
ProtobufSinglerawraw
ProtobufListrawraw
Avrorawraw
AvroConfluentraw-
Parquetrawraw
ParquetMetadataraw-
Arrowrawraw
ArrowStreamrawraw
ORCrawraw
Oneraw-
Npyrawraw
RowBinaryfullfull
RowBinaryWithNamesfullfull
RowBinaryWithNamesAndTypesfullfull
RowBinaryWithDefaultsfull-
Nativefullraw
Null-raw
XML-raw
CapnProtorawraw
LineAsStringrawraw
Regexpraw-
RawBLOBrawraw
MsgPackrawraw
MySQLDumpraw-
DWARFraw-
Markdown-raw
Formraw-

Insert API

insert(String tableName, InputStream data, ClickHouseFormat format)

Accepts data as an InputStream of bytes in the specified format. It is expected that data is encoded in the format.

Signatures

CompletableFuture<InsertResponse> insert(String tableName, InputStream data, ClickHouseFormat format, InsertSettings settings)
CompletableFuture<InsertResponse> insert(String tableName, InputStream data, ClickHouseFormat format)

Parameters

tableName - a target table name.

data - an input stream of an encoded data.

format - a format in which the data is encoded.

settings - request settings.

Return value

Future of InsertResponse type - result of the operation and additional information like server side metrics.

Examples

try (InputStream dataStream = getDataStream()) {
try (InsertResponse response = client.insert(TABLE_NAME, dataStream, ClickHouseFormat.JSONEachRow,
insertSettings).get(3, TimeUnit.SECONDS)) {

log.info("Insert finished: {} rows written", response.getMetrics().getMetric(ServerMetrics.NUM_ROWS_WRITTEN).getLong());
} catch (Exception e) {
log.error("Failed to write JSONEachRow data", e);
throw new RuntimeException(e);
}
}

insert(String tableName, List<?> data, InsertSettings settings)

Sends a write request to database. The list of objects is converted into an efficient format and then is sent to a server. The class of the list items should be registed up-front using register(Class, TableSchema) method.

Signatures

client.insert(String tableName, List<?> data, InsertSettings settings)
client.insert(String tableName, List<?> data)

Parameters

tableName - name of the target table.

data - collection DTO (Data Transfer Object) objects.

settings - request settings.

Return value

Future of InsertResponse type - the result of the operation and additional information like server side metrics.

Examples

// Important step (done once) - register class to pre-compile object serializer according to the table schema. 
client.register(ArticleViewEvent.class, client.getTableSchema(TABLE_NAME));


List<ArticleViewEvent> events = loadBatch();

try (InsertResponse response = client.insert(TABLE_NAME, events).get()) {
// handle response, then it will be closed and connection that served request will be released.
}

InsertSettings

Configuration options for insert operations.

Configuration methods

setQueryId(String queryId)
Sets query ID that will be assigned to the operation
setDeduplicationToken(String token)
Sets the deduplication token. This token will be sent to the server and can be used to identify the query.
waitEndOfQuery(Boolean waitEndOfQuery)
Requests the server to wait for the and of the query before sending response.
setInputStreamCopyBufferSize(int size)
Copy buffer size. The buffer is used during write operations to copy data from user provided input stream to an output stream.
serverSetting(String name, String value)
Sets individual server settings for an operation
serverSetting(String name, Collection values)
Sets individual server settings with multiple values for an operation. Items of the collection should `String` values
setDBRoles(Collection dbRoles)
Sets DB roles to be set before executing an operation. Items of the collection should be `String` values

InsertResponse

Response object that holds result of insert operation. It is only available if the client got response from a server.

Note

This object should be closed as soon as possible to release a connection because the connection cannot be re-used until all data of previous response is fully read.

OperationMetrics getMetrics()
Returns object with operation metrics
String getQueryId()
Returns query ID assigned for the operation by application (thru operation settings or by server).

Query API

query(String sqlQuery)

Sends sqlQuery as is. Response format is set by query settings. QueryResponse will hold a reference to the response stream that should be consumed by a reader for the supportig format.

Signatures

CompletableFuture<QueryResponse> query(String sqlQuery, QuerySettings settings)
CompletableFuture<QueryResponse> query(String sqlQuery)

Parameters

sqlQuery - a single SQL statement. The Query is sent as is to a server.

settings - request settings.

Return value

Future of QueryResponse type - a result dataset and additional information like server side metrics. The Response object should be closed after consuming the dataset.

Examples

final String sql = "select * from " + TABLE_NAME + " where title <> '' limit 10";

// Default format is RowBinaryWithNamesAndTypesFormatReader so reader have all information about columns
try (QueryResponse response = client.query(sql).get(3, TimeUnit.SECONDS);) {

// Create a reader to access the data in a convenient way
ClickHouseBinaryFormatReader reader = client.newBinaryFormatReader(response);

while (reader.hasNext()) {
reader.next(); // Read the next record from stream and parse it

// get values
double id = reader.getDouble("id");
String title = reader.getString("title");
String url = reader.getString("url");

// collecting data
}
} catch (Exception e) {
log.error("Failed to read data", e);
}

// put business logic outside of the reading block to release http connection asap.

query(String sqlQuery, Map<String, Object> queryParams, QuerySettings settings)

Sends sqlQuery as is. Additionally will send query parameters so the server can compile the SQL expression.

Signatures

CompletableFuture<QueryResponse> query(String sqlQuery, Map<String, Object> queryParams, QuerySettings settings)

Parameters

sqlQuery - sql expression with placeholders {}.

queryParams - map of variables to complete the sql expression on server.

settings - request settings.

Return value

Future of QueryResponse type - a result dataset and additional information like server side metrics. The Response object should be closed after consuming the dataset.

Examples


// define parameters. They will be sent to the server along with the request.
Map<String, Object> queryParams = new HashMap<>();
queryParams.put("param1", 2);

try (QueryResponse queryResponse =
client.query("SELECT * FROM " + table + " WHERE col1 >= {param1:UInt32}", queryParams, new QuerySettings()).get()) {

// Create a reader to access the data in a convenient way
ClickHouseBinaryFormatReader reader = client.newBinaryFormatReader(response);

while (reader.hasNext()) {
reader.next(); // Read the next record from stream and parse it

// reading data
}

} catch (Exception e) {
log.error("Failed to read data", e);
}

queryAll(String sqlQuery)

Queries a data in RowBinaryWithNamesAndTypes format. Returns the result as a collection. Read performance is the same as with the reader but more memory is required to hold the whole dataset.

Signatures

List<GenericRecord> queryAll(String sqlQuery)

Parameters

sqlQuery - sql expression to query data from a server.

Return value

Complete dataset represented by a list of GenericRecord objects that provide access in row style for the result data.

Examples

try {
log.info("Reading whole table and process record by record");
final String sql = "select * from " + TABLE_NAME + " where title <> ''";

// Read whole result set and process it record by record
client.queryAll(sql).forEach(row -> {
double id = row.getDouble("id");
String title = row.getString("title");
String url = row.getString("url");

log.info("id: {}, title: {}, url: {}", id, title, url);
});
} catch (Exception e) {
log.error("Failed to read data", e);
}

QuerySettings

Configuration options for query operations.

Configuration methods

setQueryId(String queryId)
Sets query ID that will be assigned to the operation
setFormat(ClickHouseFormat format)
Sets response format. See `RowBinaryWithNamesAndTypes` for the full list.
setMaxExecutionTime(Integer maxExecutionTime)
Sets operation execution time on server. Will not affect read timeout.
waitEndOfQuery(Boolean waitEndOfQuery)
Requests the server to wait for the and of the query before sending response.
setUseServerTimeZone(Boolean useServerTimeZone)
Server timezone (see client config) will be used to parse date/time types in the result of an operation. Default `false`
setUseTimeZone(String timeZone)
Requests server to use `timeZone` for time conversion. See session_timezone.
serverSetting(String name, String value)
Sets individual server settings for an operation
serverSetting(String name, Collection values)
Sets individual server settings with multiple values for an operation. Items of the collection should `String` values
setDBRoles(Collection dbRoles)
Sets DB roles to be set before executing an operation. Items of the collection should be `String` values

QueryResponse

Response object that holds result of query execution. It is only available if the client got a response from a server.

Note

This object should be closed as soon as possible to release a connection because the connection cannot be re-used until all data of previous response is fully read.

ClickHouseFormat getFormat()
Returns a format in which data in the response is encoded.
InputStream getInputStream()
Returns uncompressed byte stream of data in the specified format.
OperationMetrics getMetrics()
Returns object with operation metrics
String getQueryId()
Returns query ID assigned for the operation by application (thru operation settings or by server).
TimeZone getTimeZone()
Returns timezone that should be used for handling Date/DateTime types in the response.

Examples

Common API

getTableSchema(String table)

Fetches table schema for the table.

Signatures

TableSchema getTableSchema(String table)
TableSchema getTableSchema(String table, String database)

Parameters

table - table name for which schema data should be fetched.

database - database where the target table is defined.

Return value

Returns a TableSchema object with list of table columns.

getTableSchemaFromQuery(String sql)

Fetches schema from a SQL statement.

Signatures

TableSchema getTableSchemaFromQuery(String sql)

Parameters

sql - "SELECT" SQL statement which schema should be returned.

Return value

Returns a TableSchema object with columns matching the sql expression.

TableSchema

register(Class<?> clazz, TableSchema schema)

Compiles SerDe layer for the Java Class to use for writing/reading data with schema. The method will create a serializer and deserializer for the pair getter/setter and corresponding column. Column match is found by extracting its name from a method name. For example, getFirstName will be for the column first_name or firstname.

Signatures

void register(Class<?> clazz, TableSchema schema)

Parameters

clazz - Class representing the POJO used to read/write data.

schema - Data schema to use for matching with POJO properties.

Examples

client.register(ArticleViewEvent.class, client.getTableSchema(TABLE_NAME));

Usage Examples

Complete examples code is stored in the repo in a 'example` folder: