Skip to main content
Skip to main content
Edit this page

Integration Development Best Practices

This guide is for developers building integrations on top of ClickHouse — BI tool connectors, ETL/ELT pipelines, data catalog adapters, IDE plugins, or any software that connects to ClickHouse on behalf of end users. It covers connectivity choices, schema discovery, data type mapping, query patterns, observability, and ClickHouse Cloud specifics.

If you are a developer writing an application that happens to use ClickHouse directly, the language client documentation is a better starting point.

Choosing a connectivity protocol

ClickHouse exposes several interfaces. Choose the one that best fits your integration's language and architecture:

ProtocolPortBest for
HTTP API8123 (HTTP), 8443 (HTTPS)Custom connectors in any language, REST-oriented tools, maximum control over format and compression
JDBC8123 / 8443Java-based integrations, BI tools and frameworks that expect a standard JDBC DataSource
ODBC8123 / 8443Windows-native tools, Excel, legacy enterprise software
Native TCP9000 / 9440Internal tooling where maximum throughput is required; not recommended for external integrations as the protocol is not versioned for stability
MySQL wire protocol9004Tools with built-in MySQL support and no ClickHouse driver available
PostgreSQL wire protocol9005Tools with built-in PostgreSQL support and no ClickHouse driver available

HTTP API is the recommended foundation for new integrations. It is stable, format-agnostic, works from any language, and exposes the full feature set of ClickHouse. JDBC is the best choice when your integration targets the Java ecosystem or needs to interoperate with JDBC-aware frameworks.

ClickHouse Cloud

ClickHouse Cloud exposes only the HTTPS port (8443) and the secure native port (9440). Plaintext HTTP connections are not accepted. Design your integration to require TLS from the start.

Authentication

ClickHouse uses username and password authentication. Pass credentials either as HTTP Basic Auth headers or as URL parameters.

HTTP API

# Basic Auth (recommended — credentials not in URL)
curl --user "myuser:mypassword" \
     "https://my-service.clickhouse.cloud:8443/?query=SELECT+1"

# URL parameters (convenient for testing, avoid in production)
curl "https://my-service.clickhouse.cloud:8443/?user=myuser&password=mypassword&query=SELECT+1"

JDBC

Properties props = new Properties();
props.setProperty("user", System.getenv("CH_USER"));
props.setProperty("password", System.getenv("CH_PASSWORD"));
props.setProperty("ssl", "true");
props.setProperty("sslmode", "strict");

Connection conn = DriverManager.getConnection(
    "jdbc:ch:https://my-service.clickhouse.cloud:8443/my_db", props);

Never embed credentials in connection URLs that may appear in logs or error messages. Read them from environment variables or a secrets manager.

Dedicated service accounts

Create a dedicated ClickHouse user for your integration with only the permissions it needs. Avoid connecting as default:

CREATE USER integration_user IDENTIFIED BY 'strong_password';
GRANT SELECT ON my_database.* TO integration_user;
-- Grant INSERT if your integration writes data
GRANT INSERT ON my_database.* TO integration_user;

SSL/TLS

Always use TLS for connections to ClickHouse Cloud and strongly recommended for self-managed production clusters. For HTTPS connections, sslmode=strict (the default) verifies the server certificate. Use sslmode=none only in isolated development environments — never in production or user-facing integrations.

Schema discovery

ClickHouse exposes rich metadata through system.* tables. Use these to enumerate databases, tables, columns, and other objects for features like schema browsers, column pickers, and query editors.

Listing databases

SELECT name
FROM system.databases
WHERE engine NOT IN ('System')
ORDER BY name;

Listing tables

SELECT
    database,
    name,
    engine,
    total_rows,
    formatReadableSize(total_bytes) AS size,
    comment
FROM system.tables
WHERE database NOT IN ('system', 'information_schema', 'INFORMATION_SCHEMA')
  AND is_temporary = 0
ORDER BY database, name;

Listing columns

SELECT
    database,
    table,
    name,
    type,
    default_kind,
    default_expression,
    comment,
    is_in_primary_key,
    is_in_sorting_key
FROM system.columns
WHERE database NOT IN ('system', 'information_schema', 'INFORMATION_SCHEMA')
ORDER BY database, table, position;

INFORMATION_SCHEMA

ClickHouse also implements INFORMATION_SCHEMA for compatibility with tools that use standard SQL introspection queries. It covers a subset of metadata and is suitable when portability across databases matters:

SELECT table_schema, table_name, column_name, data_type, is_nullable
FROM information_schema.columns
WHERE table_schema NOT IN ('system', 'information_schema', 'INFORMATION_SCHEMA')
ORDER BY table_schema, table_name, ordinal_position;
Note

Prefer system.columns over INFORMATION_SCHEMA.columns for ClickHouse-specific metadata like is_in_sorting_key and is_in_primary_key, which are important for generating efficient queries. Also note that INFORMATION_SCHEMA.TABLES.TABLE_ROWS is always NULL in ClickHouse — use system.tables.total_rows instead for row count estimates. INFORMATION_SCHEMA also has no foreign key data (KEY_COLUMN_USAGE and REFERENTIAL_CONSTRAINTS are always empty — ClickHouse has no foreign key constraints).

Data type mapping

ClickHouse has a richer type system than most databases your integration will encounter. The following sections cover the types that require special handling.

Numeric types

ClickHouse signed integers map predictably. Unsigned integers require care — they exceed the range of their same-width signed Java/SQL counterparts and must be promoted:

ClickHouse TypeRecommended Java/SQL mapping
Int8, Int16, Int32, Int64Byte, Short, Integer, Long
Int128, Int256BigInteger
UInt8Short
UInt16Integer
UInt32Long
UInt64BigInteger
UInt128, UInt256BigInteger
Float32, Float64Float, Double
Decimal32/64/128/256BigDecimal

String types

String in ClickHouse is a raw byte sequence with no enforced encoding (UTF-8 is conventional). FixedString(N) is zero-padded to N bytes when read — strip trailing null bytes (\0) before displaying values to users.

Date and time types

ClickHouse TypeNotes
DateDays since 1970-01-01. No timezone. Map to LocalDate.
Date32Extended date range. No timezone. Map to LocalDate.
DateTimeUnix timestamp in seconds, stored with optional server/session timezone. Map to Instant or ZonedDateTime.
DateTime64(n)Sub-second precision (n = 0–9). Same timezone behavior as DateTime.

DateTime and DateTime64 values are stored as UTC internally. String representation depends on the server timezone and any timezone defined on the column. When the column has no explicit timezone, the server timezone is used for both reading and writing string values.

Note

session_timezone is an experimental setting that can invalidate partition pruning on DateTime columns. Do not use it in production. Convert timestamps to UTC in your application layer instead, or pass them as Unix epoch values using fromUnixTimestamp64Nano() to avoid any ambiguity.

Type modifiers

Two modifiers wrap other types and must be handled by your integration:

  • Nullable(T) — the column may contain NULL. Affects the type name returned in metadata (e.g., Nullable(Int32)). Strip the wrapper when mapping to target types.
  • LowCardinality(T) — a dictionary-encoded form of T, used for performance. Treat identically to the underlying type for all practical purposes.

Complex types

ClickHouse TypeBehavior
Array(T)Nested arrays supported. Via JDBC, returned as java.sql.Array. Via HTTP, returned as JSON arrays.
Map(K, V)Key-value pairs. Via HTTP, returned as a JSON object.
Tuple(T1, T2, ...)Fixed-length heterogeneous sequence. Via HTTP, returned as a JSON array.
Enum8, Enum16Returned as strings by default. Can be read as their underlying integer.
UUIDReturned as a string in xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx format over HTTP.
IPv4, IPv6Returned as dotted-decimal / colon-hex strings over HTTP.
JSONReturned as a JSON string. Supported in newer ClickHouse versions.

AggregateFunction columns hold binary-encoded aggregate state. They are not human-readable and should generally be excluded from column listings exposed to end users.

Query patterns

Use parameterized queries

Never concatenate user input into SQL strings. ClickHouse supports named query parameters that prevent SQL injection and improve query plan reuse.

HTTP API:

curl --user "user:password" \
     "https://host:8443/" \
     --data "SELECT * FROM events WHERE user_id = {user_id:String} AND event_date > {min_date:Date}" \
     -G --data-urlencode "param_user_id=alice" \
        --data-urlencode "param_min_date=2024-01-01"

Parameters are declared as {name:type} in the query and passed as param_name HTTP parameters.

JDBC:

try (PreparedStatement ps = conn.prepareStatement(
        "SELECT * FROM events WHERE user_id = ? AND event_date > ?")) {
    ps.setString(1, userId);
    ps.setObject(2, LocalDate.parse("2024-01-01"));
    ResultSet rs = ps.executeQuery();
}

Assign a query_id to every query

Set a deterministic query_id on each request. This lets you trace queries in system.query_log, cancel runaway queries, and implement idempotent retry logic.

HTTP API:

curl --user "user:password" \
     "https://host:8443/?query_id=my-connector-job-abc123" \
     --data "SELECT count() FROM events"

JDBC:

Statement stmt = conn.createStatement();
stmt.unwrap(StatementImpl.class).getLocalSettings().queryId("my-connector-job-abc123");
ResultSet rs = stmt.executeQuery("SELECT count() FROM events");

If you retry a query after a timeout, reuse the same query_id. ClickHouse will return the result of the already-running query rather than executing it twice.

Choose the right output format

ClickHouse supports dozens of output formats. Choose based on what your integration needs:

FormatUse when
JSONEachRowStreaming row-by-row JSON; easy to parse incrementally
JSONCompactCompact JSON with column names in a header; smaller than full JSON
CSV / TSVInteroperability with spreadsheets and generic tools
ParquetColumnar data exchange with data lakehouse tools
RowBinary / NativeMaximum throughput; binary format requiring type-aware deserialization
ArrowIn-memory columnar exchange with Apache Arrow-compatible tools

For most connector use cases, JSONEachRow is the best default: it is easy to stream and parse without buffering the full response.

curl --user "user:password" \
     "https://host:8443/?default_format=JSONEachRow" \
     --data "SELECT user_id, event_name, created_at FROM events LIMIT 1000"

Enable compression

LZ4 compression over HTTP significantly reduces network transfer for large result sets and inserts, at negligible CPU cost. Enable it by sending the Accept-Encoding header:

curl --user "user:password" \
     -H "Accept-Encoding: lz4" \
     "https://host:8443/?enable_http_compression=1" \
     --data "SELECT * FROM large_table"

Paginate large result sets

ClickHouse does not have native cursor-based pagination. Use LIMIT / OFFSET for small datasets, or keyset pagination for large ones:

-- Keyset pagination (efficient — avoids full scan for OFFSET)
SELECT user_id, event_name, created_at
FROM events
WHERE created_at < {last_seen_ts:DateTime64(3)}
ORDER BY created_at DESC
LIMIT 1000;

For very large exports, prefer streaming: consume JSONEachRow output incrementally rather than paginating.

Identifiers are case-sensitive

ClickHouse database names, table names, column names, and most function names are case-sensitive. A column defined as userId is different from userid. BI tools and ORMs that auto-capitalize identifiers will get Unknown column or Unknown function errors. Establish a lowercase or snake_case naming convention for all ClickHouse objects in integrations you control, and document this expectation for users creating tables.

64-bit integers in JSON output

If your integration consumes ClickHouse HTTP responses in JavaScript or TypeScript, note that Int64 and UInt64 values in JSON output are sent as numbers by default. JavaScript's JSON.parse() silently loses precision for integers beyond 2^53. Add output_format_json_quote_64bit_integers=1 to query parameters to receive them as quoted strings, then parse with a BigInt-aware library:

curl --user "user:password" \
     "https://host:8443/?output_format_json_quote_64bit_integers=1" \
     --data "SELECT id FROM events FORMAT JSONEachRow"

Set query timeouts

Protect your integration from runaway queries. Pass max_execution_time (seconds) as a query setting:

curl --user "user:password" \
     "https://host:8443/?max_execution_time=30" \
     --data "SELECT ..."
// JDBC — via server settings prefix
props.setProperty("clickhouse_setting_max_execution_time", "30");

Writing data

Use batch inserts

Always batch rows into a single INSERT rather than sending one row per request. The HTTP API accepts INSERT data as the request body:

# Insert CSV data
curl --user "user:password" \
     "https://host:8443/?query=INSERT+INTO+events+FORMAT+CSV" \
     --data-binary @events.csv

# Insert JSONEachRow inline
curl --user "user:password" \
     "https://host:8443/?query=INSERT+INTO+events+FORMAT+JSONEachRow" \
     --data '{"user_id":"alice","event_name":"login","created_at":"2024-01-15 12:00:00"}
{"user_id":"bob","event_name":"signup","created_at":"2024-01-15 12:01:00"}'

Aim for batches of 10,000–100,000 rows per request for optimal throughput. Smaller batches are the single most common production failure mode for connector-driven ingest: every INSERT creates a new on-disk data part, and ClickHouse merges parts asynchronously. If inserts arrive faster than merges complete, the part count crosses a threshold (default 300 active parts per partition) and ClickHouse raises Too many parts errors. Larger batches increase memory pressure. Never issue INSERT more than 1–2 times per second per table.

JDBC batch inserts

try (PreparedStatement ps = conn.prepareStatement(
        "INSERT INTO events (user_id, event_name, created_at) VALUES (?, ?, ?)")) {
    for (Event e : batch) {
        ps.setString(1, e.userId());
        ps.setString(2, e.eventName());
        ps.setObject(3, e.createdAt());
        ps.addBatch();
    }
    ps.executeBatch();
}

Async insert behavior

async_insert=1 buffers inserts server-side and flushes in bulk. Two important behaviors to understand:

  • With wait_for_async_insert=0 (fire-and-forget), the server returns HTTP 200 immediately — before the data is validated or written. If a row has a type error, the entire buffered batch is silently dropped with no error surfaced to the client. Use wait_for_async_insert=1 in latency-tolerant pipelines to get synchronous error feedback.
  • The SETTINGS clause must appear before VALUES in an INSERT statement. INSERT INTO t SETTINGS async_insert=1 VALUES (?,?) works; INSERT INTO t VALUES (?,?) SETTINGS async_insert=1 does not — the server fails to detect async-insert mode and parts accumulate rapidly.

JDBC PreparedStatement batches

In JDBC driver versions 0.8.6 and above, reusing a PreparedStatement across multiple executeBatch() calls re-inserts data from all previous batches due to a batch buffer not being cleared between executions. Create a new PreparedStatement instance for each batch cycle:

// Safe: new PreparedStatement per batch
for (List<Event> batch : batches) {
    try (PreparedStatement ps = conn.prepareStatement(
            "INSERT INTO events (user_id, event_name, created_at) VALUES (?, ?, ?)")) {
        for (Event e : batch) {
            ps.setString(1, e.userId());
            ps.setString(2, e.eventName());
            ps.setObject(3, e.createdAt());
            ps.addBatch();
        }
        ps.executeBatch();
    }
}

Materialized views inflate row counts

After creating a Materialized View on a source table, the rows_affected count returned by an INSERT includes rows written to both the source table and all MV target tables. An insert of 1,000 rows returns 2,000 rows affected with one MV. Row-count-based validation in ETL pipelines will fail. Use SELECT count() FROM source_table WHERE ... post-insert for validation instead.

Mutations are asynchronous

ALTER TABLE ... DELETE and ALTER TABLE ... UPDATE are asynchronous mutations. They return success immediately but run in the background as data parts are rewritten. A SELECT immediately after a DELETE may still return the deleted rows. For upsert patterns, use ReplacingMergeTree or CollapsingMergeTree instead of mutations. Poll system.mutations WHERE is_done = 0 if you need to wait for completion.

ALTER TABLE DELETE also does not clear deduplication checksums. If you delete rows and retry the same insert with the same content, ClickHouse will silently deduplicate and re-insert nothing. Use a new insert_deduplication_token value when re-inserting after deletions.

ClickHouse has no ACID transactions

Do not rely on BEGIN / COMMIT / ROLLBACK for data consistency. ClickHouse is append-optimized. Design writes to be idempotent:

  • Use insert_deduplication_token to make retries safe — ClickHouse will deduplicate inserts with the same token within a configurable window:
curl --user "user:password" \
     "https://host:8443/?insert_deduplication_token=batch-20240115-001" \
     --data "INSERT INTO events FORMAT JSONEachRow ..."
  • Use ReplacingMergeTree for upsert semantics — the engine merges duplicate primary keys, keeping the row with the highest version value.

JSON inserts and DEFAULT expressions

When inserting JSON with input_format_skip_unknown_fields=1, columns absent from the payload receive the type default (0, empty string) — not the column's DDL DEFAULT expression. If your schema uses DEFAULT now() or DEFAULT generateUUIDv4(), those expressions are only invoked when you also set input_format_defaults_for_omitted_fields=1:

curl --user "user:password" \
     "https://host:8443/?input_format_skip_unknown_fields=1&input_format_defaults_for_omitted_fields=1" \
     --data "INSERT INTO events FORMAT JSONEachRow ..."

Identifying your integration

Always identify your integration in the HTTP User-Agent header and in per-query metadata. This makes queries attributable in system.query_log, which is invaluable for debugging customer issues and monitoring usage.

Set a User-Agent header

Use the format product/version (context):

curl --user "user:password" \
     -H "User-Agent: MyBITool/3.2 (ClickHouse connector)" \
     "https://host:8443/?query=SELECT+1"

Via JDBC:

props.setProperty("client_name", "MyBITool/3.2");

Tag individual queries with log_comment

Attach operation context to each query so customer support and the customer themselves can filter system.query_log by feature or job:

curl --user "user:password" \
     "https://host:8443/?log_comment=dashboard%3Arevenue-by-region" \
     --data "SELECT region, sum(revenue) FROM sales GROUP BY region"

Query your tagged queries in system.query_log

SELECT
    query_id,
    query,
    log_comment,
    query_duration_ms,
    read_rows,
    read_bytes,
    exception
FROM system.query_log
WHERE http_user_agent LIKE 'MyBITool%'
  AND event_time > now() - INTERVAL 1 HOUR
  AND type = 'QueryFinish'
ORDER BY event_time DESC
LIMIT 50;

Error handling

HTTP 200 does not mean success

ClickHouse begins streaming results immediately and sends the 200 OK status header before it knows whether the query will succeed. If an error occurs mid-stream, the error is appended to the response body — but the HTTP status remains 200.

This also affects inserts: a large POST that triggers a server-side timeout returns HTTP 200 OK with X-ClickHouse-Exception-Code: 209 in the headers and no data committed.

Your integration must:

  1. Check the X-ClickHouse-Exception-Code response header on every response, not just non-200s
  2. Scan the response body for Code: NNN. DB::Exception: if you are processing it as a stream
  3. For DDL and short queries where latency is acceptable, add wait_end_of_query=1 to buffer the response server-side and get a reliable error status
# wait_end_of_query buffers the full response before sending — guarantees
# that a non-200 means failure, but doubles server memory for large results
curl --user "user:password" \
     "https://host:8443/?wait_end_of_query=1" \
     --data "SELECT ..."

HTTP status codes

StatusMeaning
200Query reached the server — check body and headers for errors
400Bad request (malformed query, invalid parameter)
401Authentication failed
403Permission denied
404Database or table not found
500Server-side error (query execution failure, OOM, etc.)

ClickHouse returns error details in the response body and as X-ClickHouse-Exception-Code and X-ClickHouse-Summary HTTP headers. Error format:

Code: 60. DB::Exception: Table my_db.unknown_table doesn't exist. (UNKNOWN_TABLE)

The numeric code (60) is stable across releases and suitable for programmatic handling.

Retry strategy

Retry on:

  • Network-level errors (connection refused, timeout)
  • HTTP 500 where the exception code indicates a transient condition (e.g., server overload)

Do not retry on:

  • HTTP 400 (bad query — retrying won't help)
  • HTTP 403 (wrong permissions — retrying won't help)

Use exponential backoff with jitter. Reuse the same query_id on retries for INSERT operations so ClickHouse can deduplicate.

Handling streaming errors

When using streaming output formats (e.g., JSONEachRow), ClickHouse may have already started writing rows before encountering an error. Errors in this case are appended at the end of the response stream rather than returned as an HTTP 500. Always read the full stream and check for a trailing error block:

{"user_id":"alice","value":1}
{"user_id":"bob","value":2}
{"exception":"Code: 241. Memory limit exceeded"}

ClickHouse Cloud considerations

Connection endpoints

ClickHouse Cloud service endpoints follow this pattern:

{service-id}.{region}.{cloud}.clickhouse.cloud:8443

Where cloud is aws, gcp, or azure. Always use port 8443 (HTTPS) — port 8123 is not available on Cloud.

Auto-pause and connection retries

ClickHouse Cloud services on the development tier may auto-pause after a period of inactivity. An initial connection after a pause may take a few seconds to respond. Design your integration to retry connection attempts with a short backoff before surfacing an error to the user.

ClickHouse Cloud API for programmatic management

If your integration needs to enumerate or provision Cloud services (rather than query data), use the ClickHouse Cloud API. It is separate from the query interface and uses API key authentication.

Testing your integration

Test against both OSS and ClickHouse Cloud

Behavioral differences between self-managed ClickHouse and ClickHouse Cloud are minimal for most integrations, but test both. Specifically:

  • ClickHouse Cloud always requires TLS
  • Some system table columns may differ between versions
  • Auto-pause behavior is Cloud-only

Cover edge-case data types

Most integration bugs are found with edge cases. Explicitly test:

  • Nullable columns — verify NULLs round-trip correctly
  • UInt64 values near Long.MAX_VALUE
  • FixedString — verify zero-byte stripping
  • DateTime64 with sub-second precision and non-UTC timezones
  • Arrays and Maps — verify nested types survive serialization
  • Empty result sets and single-row result sets

Use system.query_log to verify behavior

system.query_log is written asynchronously with a flush interval of approximately 7.5 seconds. Do not query it immediately after a test — add a small delay or poll with retry before expecting entries to appear.

After running your integration's test suite, inspect system.query_log to verify:

  • Queries are attributed to your integration's User-Agent
  • No unexpected full-table scans (check read_rows and whether the primary key is used)
  • Insert queries have the expected written_rows
SELECT
    query_start_time,
    query,
    read_rows,
    written_rows,
    query_duration_ms,
    exception
FROM system.query_log
WHERE http_user_agent LIKE 'MyIntegration%'
  AND event_time > now() - INTERVAL 10 MINUTE
ORDER BY query_start_time;

Implementation examples

HTTP API — Python connector skeleton

A minimal pattern for a Python-based connector that queries ClickHouse and streams results:

import requests
from typing import Iterator

class ClickHouseConnector:
    def __init__(self, host: str, port: int, user: str, password: str):
        self.base_url = f"https://{host}:{port}/"
        self.session = requests.Session()
        self.session.auth = (user, password)
        self.session.headers.update({
            "User-Agent": "MyConnector/1.0 (ClickHouse integration)",
            "Accept-Encoding": "lz4",
        })

    def query_stream(self, sql: str, query_id: str = None) -> Iterator[dict]:
        params = {
            "default_format": "JSONEachRow",
            "enable_http_compression": "1",
            "max_execution_time": "60",
        }
        if query_id:
            params["query_id"] = query_id

        with self.session.post(
            self.base_url,
            data=sql,
            params=params,
            stream=True,
        ) as resp:
            resp.raise_for_status()
            for line in resp.iter_lines():
                if line:
                    yield json.loads(line)

    def schema(self, database: str) -> list[dict]:
        sql = f"""
            SELECT name, type, is_in_primary_key, is_in_sorting_key
            FROM system.columns
            WHERE database = '{database}'
            ORDER BY table, position
        """
        return list(self.query_stream(sql))

JDBC — Java BI connector skeleton

A minimal pattern for a JDBC-based BI connector with connection pooling and query tagging:

import com.clickhouse.jdbc.DataSource;
import com.zaxxer.hikari.HikariConfig;
import com.zaxxer.hikari.HikariDataSource;

public class ClickHouseJdbcConnector {

    private final HikariDataSource pool;

    public ClickHouseJdbcConnector(String host, int port, String database,
                                    String user, String password) {
        Properties props = new Properties();
        props.setProperty("user", user);
        props.setProperty("password", password);
        props.setProperty("ssl", "true");
        props.setProperty("sslmode", "strict");
        props.setProperty("client_name", "MyBITool/1.0");
        props.setProperty("socket_keepalive", "true");
        props.setProperty("compress", "1");

        HikariConfig config = new HikariConfig();
        config.setMaximumPoolSize(10);
        config.setMaxLifetime(300_000);
        config.setConnectionTimeout(5_000);
        config.setDataSource(new DataSource(
            String.format("jdbc:ch:https://%s:%d/%s", host, port, database), props));

        this.pool = new HikariDataSource(config);
    }

    public List<Map<String, Object>> query(String sql, String queryId) throws SQLException {
        try (Connection conn = pool.getConnection();
             Statement stmt = conn.createStatement()) {
            // tag the query for system.query_log attribution
            stmt.unwrap(StatementImpl.class)
                .getLocalSettings()
                .queryId(queryId);

            List<Map<String, Object>> rows = new ArrayList<>();
            try (ResultSet rs = stmt.executeQuery(sql)) {
                ResultSetMetaData meta = rs.getMetaData();
                int cols = meta.getColumnCount();
                while (rs.next()) {
                    Map<String, Object> row = new LinkedHashMap<>();
                    for (int i = 1; i <= cols; i++) {
                        row.put(meta.getColumnName(i), rs.getObject(i));
                    }
                    rows.add(row);
                }
            }
            return rows;
        }
    }
}

Remote MCP integrations

Model Context Protocol (MCP) is an open standard that lets AI agents and LLM-powered tools discover and invoke capabilities exposed by a server. For ClickHouse integrations, MCP is the right surface when your integration is AI-native — meaning an agent or LLM is driving the queries — rather than a traditional BI tool, ETL pipeline, or application making programmatic requests.

Integration typeRecommended surface
BI tool / query builderJDBC or HTTP API
ETL / data pipelineHTTP API or JDBC
AI agent / LLM assistantRemote MCP server
IDE coding assistantLocal MCP server (stdio) or remote MCP

ClickHouse Cloud built-in remote MCP server

ClickHouse Cloud includes a fully managed remote MCP server. It requires no infrastructure to deploy and authenticates via OAuth 2.0.

Endpoint:

https://mcp.clickhouse.cloud/mcp

Transport: Streamable HTTP (MCP standard)

Authentication: OAuth 2.0 — the MCP client initiates a browser-based OAuth flow using ClickHouse Cloud credentials on first connect.

Capabilities: 13 read-only tools across querying, schema discovery, service management, backups, ClickPipes, and billing. All tools are annotated with readOnlyHint: true.

Enable it per service in the ClickHouse Cloud console under Connect → MCP. Once enabled, point any MCP client at the endpoint above. See the remote MCP setup guide for IDE-specific configuration steps.

If your integration targets ClickHouse Cloud users, this is the fastest path to MCP connectivity — recommend it over building your own server.

Open-source MCP server for self-hosted ClickHouse

For integrations that must support self-hosted ClickHouse instances, use the mcp-clickhouse open-source server as a starting point. It exposes three core tools (run_query, list_databases, list_tables) and supports both local stdio and remote HTTP/SSE transports.

Building your own remote MCP server

Build a custom MCP server when you need tools or behavior that neither the Cloud server nor the open-source server provides — for example, domain-specific query templates, write operations under controlled conditions, or integration with your own auth system.

Choose the right transport

TransportUse when
stdioLocal dev tools (Claude Code, Cursor, VS Code Copilot). No network exposure.
Streamable HTTPCloud-deployed servers, multi-tenant services, load-balanced deployments. The MCP standard transport for remote servers.
SSE (legacy)Legacy MCP clients that predate Streamable HTTP. Prefer HTTP for new servers.

For remote servers, Streamable HTTP is the current standard. Design stateless request handlers — do not store session state in memory, as load balancers will distribute requests across instances.

Authentication

ScenarioRecommended approach
User-facing (human authenticates)OAuth 2.0 with PKCE — consistent with ClickHouse Cloud's own approach
Service-to-service (agent authenticates)Static Bearer token via Authorization header; rotate regularly
Development / local-onlyDisable auth (CLICKHOUSE_MCP_AUTH_DISABLED=true); never in production

Always require authentication for any remotely accessible MCP server. The /health endpoint is the only route that should remain unauthenticated (for orchestrator probes).

Design read-only tools by default

Annotate every tool with readOnlyHint: true in its MCP metadata unless writes are explicitly part of your design. This signals to MCP hosts that the tool has no side effects, enabling better agent planning.

readOnlyHint is advisory only — it informs the LLM host but is not enforced. Apply server-side enforcement as well:

  • Connect to ClickHouse with a read-only user (no INSERT, ALTER, DROP grants)
  • Validate that query strings begin with SELECT or WITH before execution
  • Use ClickHouse's readonly setting (SET readonly = 1) at the session level as a secondary guard:
SET readonly = 1;
SELECT ...

If your server must support writes (e.g., agent-driven INSERT), expose write tools under a separate, explicitly named operation, require an additional confirmation parameter, and log every invocation.

Apply resource limits to all queries

Agents can generate unbounded queries. Always enforce limits to prevent runaway execution:

# Apply limits on the ClickHouse user level (recommended)
ALTER USER mcp_user SETTINGS
    max_execution_time = 30,       -- seconds
    max_result_rows = 10000,       -- rows returned to the agent
    max_bytes_to_read = 1073741824 -- 1 GB read limit

Or pass them per query via HTTP parameters:

curl --user "mcp_user:password" \
     "https://host:8443/?max_execution_time=30&max_result_rows=10000" \
     --data "SELECT ..."

10,000 rows is a reasonable ceiling for agent-readable results — LLMs cannot usefully process millions of rows. Design tool descriptions to guide agents toward aggregating queries rather than full table scans.

Write tool descriptions that constrain agent behavior

Tool descriptions are read by the LLM to decide how and when to invoke a tool. Vague descriptions lead to over-use and inefficient queries. Be specific:

# Too vague
Tool(
    name="run_query",
    description="Run a SQL query."
)

# Better
Tool(
    name="run_select_query",
    description=(
        "Execute a read-only SELECT query against ClickHouse. "
        "Use aggregation functions (count(), sum(), avg()) rather than returning raw rows when possible. "
        "Results are limited to 10,000 rows. "
        "Always include a WHERE clause or LIMIT to avoid full table scans."
    ),
    annotations={"readOnlyHint": True}
)

Include parameter descriptions that tell the agent what valid input looks like, including ClickHouse SQL syntax specifics (e.g., count() not COUNT(*), toDate() for date literals).

Guard against prompt injection

Query results returned by your MCP server flow back into the LLM's context. If a ClickHouse table contains user-generated text, that text could carry adversarial instructions targeting the agent.

Mitigations:

  • Limit result size — small result sets reduce the attack surface
  • Return structured data — prefer JSONEachRow and parse it server-side; avoid returning raw string columns that could contain markdown or instruction-like text directly into the agent's context
  • Sanitize schema names — when returning database, table, or column names, strip or escape characters that could be interpreted as markdown formatting or instructions

Identify your MCP server in query logs

Set a User-Agent and log_comment on all queries issued by your MCP server, exactly as you would for any integration. This makes it possible to distinguish agent-driven queries from human queries in system.query_log:

params = {
    "log_comment": f"mcp:{tool_name}/session:{session_id}",
    "query_id": request_id,
}
headers = {
    "User-Agent": "MyMCPServer/1.0 (ClickHouse MCP)",
}

Integration development checklist

  • Integration uses HTTPS / TLS for all connections
  • Credentials are read from environment variables or a secrets manager, not hardcoded
  • A dedicated ClickHouse user with minimal permissions is used
  • User-Agent header identifies the integration by name and version
  • Every query carries a query_id for traceability and idempotent retries
  • User-facing inputs pass through parameterized queries, not string concatenation
  • Schema discovery queries exclude system and information_schema databases
  • Nullable and LowCardinality type wrappers are stripped before type mapping
  • UInt64 and larger unsigned integers are mapped to BigInteger or equivalent
  • FixedString values are stripped of trailing null bytes before display
  • Bulk inserts use batches of 10,000–100,000 rows
  • insert_deduplication_token is set for retry-safe inserts
  • X-ClickHouse-Exception-Code header is checked on every response (HTTP 200 does not guarantee success)
  • Streaming responses are fully consumed and checked for trailing error blocks
  • Integration does not rely on JDBC transactions for consistency
  • Mutations (DELETE/UPDATE) are not used for high-frequency updates; ReplacingMergeTree used for upsert patterns instead
  • If using JavaScript/TypeScript: output_format_json_quote_64bit_integers=1 set to prevent Int64 precision loss
  • Table and column names use lowercase/snake_case (identifiers are case-sensitive)
  • JDBC: new PreparedStatement created per batch (reuse causes duplicate inserts in driver 0.8.6+)
  • Integration tested against both ClickHouse OSS and ClickHouse Cloud
  • Edge-case types (Nullable, UInt64, DateTime64, Array, Map) covered in tests
  • If building an MCP server:
  • Remote server uses Streamable HTTP transport and stateless request handlers
  • All tools annotated with readOnlyHint: true unless writes are explicitly required
  • Server-side enforcement: read-only ClickHouse user + SQL validation, not just readOnlyHint
  • max_execution_time, max_result_rows, and max_bytes_to_read enforced on the ClickHouse user or per-query
  • Tool descriptions guide agents toward aggregating queries; warn against full table scans
  • query_id and log_comment set on every MCP-issued query for observability
  • Prompt injection mitigations in place for result data flowing back to the LLM