ClickHouse is a columnar store designed around the assumption that your data has a known, stable schema. JSON is the opposite: variable keys, mixed types, nested structures, and shapes that shift over time. Working with JSON in ClickHouse requires picking the right storage strategy, because the cost of getting it wrong shows up in query latency, storage amplification, and ingest throughput. There is no single correct answer - there are four distinct approaches, each suited to a different part of the problem.
The Four Approaches
Raw String is the simplest: store your JSON as a String column and parse it at query time with JSONExtract* functions. No schema inference, no special settings. The upside is flexibility and zero ingest cost for JSON parsing. The downside is that every query that touches JSON fields pays the full parsing cost, and the data is stored without columnar benefit for values inside the JSON.
Named Tuple is the right choice when you know the full structure at table creation time. A named Tuple column stores each field in its own sub-column on disk with individual compression. This gives significantly better query performance than a String column with JSONExtract, though sub-column access may in some cases still require reading the full Tuple depending on ClickHouse version and storage format.
CREATE TABLE events (
id UInt64,
ts DateTime,
payload Tuple(
user_id UInt32,
action LowCardinality(String),
score Float32
)
) ENGINE = MergeTree ORDER BY (ts, id);
-- Access sub-fields with dot notation
SELECT payload.user_id, count()
FROM events
WHERE payload.action = 'purchase'
GROUP BY payload.user_id;
Map type handles the case where you have a dynamic set of keys with values of a uniform type. Map(String, String) or Map(String, UInt64) lets you store variable keys per row, access them by key, and filter on them. The constraint is that all values must share the same type - a Map cannot hold a mix of strings and integers. It also performs noticeably worse than Tuple on analytical aggregations, with benchmarks showing Map aggregations running 5x or more slower than Tuple equivalents for the same query.
The native JSON type (introduced as experimental in v24.8, production-ready since v25.3) is ClickHouse's answer to truly dynamic schemas. It accepts any valid JSON, infers sub-column types at insert time, and stores each discovered path as a separate column internally. You get columnar storage for paths that appear frequently, without declaring them upfront.
JSONExtract Functions
When your JSON lives in a String column, the JSONExtract* family is how you get values out. The most common ones are JSONExtractString, JSONExtractInt, JSONExtractFloat, JSONExtractBool, JSONExtractRaw, JSONExtractKeys, and the generic JSONExtract with an explicit type argument.
-- Extract typed values from a JSON string column
SELECT
JSONExtractInt(raw_event, 'user_id') AS user_id,
JSONExtractInt(raw_event, 'response_code') AS status,
JSONExtractFloat(raw_event, 'latency_ms') AS latency,
JSONExtractRaw(raw_event, 'tags') AS tags_raw
FROM http_logs
WHERE JSONExtractInt(raw_event, 'response_code') = 500;
JSONExtractRaw returns a JSON fragment as a string rather than a typed value - useful when the target field is itself a nested object or array you intend to pass to another function. JSONExtractKeys returns an array of the keys at a given path:
SELECT JSONExtractKeys('{"a": 1, "b": {"c": 2}}') AS top_level_keys;
-- ['a', 'b']
The critical performance issue with this function family is that each call parses the entire JSON string independently. If a query extracts five fields from the same raw_event column, the JSON is parsed five times per row. On a table with billions of rows, that is a meaningful CPU cost. The mitigation is to either materialize the fields you query most frequently (see below), or parse the JSON once into a Tuple and extract fields from that result:
-- Parse JSON once into a Tuple, then extract individual fields
SELECT
tupleElement(parsed, 1) AS user_id,
tupleElement(parsed, 2) AS status,
tupleElement(parsed, 3) AS latency
FROM (
SELECT JSONExtract(
raw_event,
'Tuple(user_id Int64, response_code Int32, latency_ms Float32)'
) AS parsed
FROM http_logs
)
WHERE status = 500;
Note: wrapping individual JSONExtract* calls in a plain subquery does not eliminate re-parsing — ClickHouse still calls each function independently. Parsing into a single Tuple with one JSONExtract call is the correct technique to parse each row's JSON only once.
The Native JSON Type
The native JSON type was introduced in v24.8 (where it required SET allow_experimental_json_type = 1) and became production-ready in v25.3 (March 2025), where the experimental flag is no longer needed. Rather than storing text and parsing at read time, ClickHouse parses JSON at insert time and writes each discovered path as a separate sub-column in columnar format.
CREATE TABLE logs (
id UInt64,
ts DateTime,
body JSON
) ENGINE = MergeTree ORDER BY (ts, id);
-- Insert raw JSON strings directly
INSERT INTO logs VALUES
(1, now(), '{"level": "error", "service": "auth", "user_id": 42}'),
(2, now(), '{"level": "info", "service": "api", "duration_ms": 130}');
-- Access sub-columns with dot notation
SELECT body.level, body.service, body.user_id
FROM logs
WHERE body.level = 'error';
The storage model is what makes this compelling. Paths that appear in many rows get their own columnar storage, with compression and min/max indices. The max_dynamic_paths parameter (default 1024, set at column definition time) controls how many distinct JSON paths get promoted to dedicated sub-columns per data part. Paths beyond that limit are stored together in a shared binary structure, which is slower to query but prevents unbounded column explosion from high-cardinality key spaces.
-- Limit dynamic sub-columns to 256 distinct paths
CREATE TABLE telemetry (
id UInt64,
data JSON(max_dynamic_paths=256)
) ENGINE = MergeTree ORDER BY id;
The trade-off against String plus JSONExtract is real. Insert throughput for JSON columns can be an order of magnitude lower than inserting into a String column, because the insert path now includes JSON parsing, type inference, and columnar write fan-out. For high-throughput ingest pipelines, this can be the deciding factor. Query performance for aggregations on frequently-occurring paths is comparable to native typed columns. Paths that end up in the shared overflow structure perform more like Map.
The JSON type graduated from experimental to production-ready in v25.3 (March 2025). The older Object('json') type from v22.3 is deprecated and should not be used in new deployments. Schema changes, certain function interactions, and replication behavior around JSON columns still carry caveats worth testing before committing to JSON at scale.
Materializing Fields at Ingest Time
The most reliable performance strategy for JSON in ClickHouse is to not query raw JSON at all for your critical access paths. Extract the fields you actually filter and aggregate on at ingest time, either into real columns or into materialized columns defined on the table itself.
Materialized columns are computed once on insert from the raw value and stored like any regular column:
CREATE TABLE http_logs (
raw_event String,
response_code UInt16 MATERIALIZED JSONExtractInt(raw_event, 'response_code'),
user_id Int64 MATERIALIZED JSONExtractInt(raw_event, 'user_id'),
latency_ms Float32 MATERIALIZED JSONExtractFloat(raw_event, 'latency_ms')
) ENGINE = MergeTree ORDER BY (response_code, user_id);
These participate in compression, min/max indices, and can be used in ORDER BY and WHERE without touching raw_event at all. The raw JSON remains available for ad-hoc extraction when you need fields outside the materialized set. This hybrid pattern - extract what you know, keep the raw string for everything else - gives you near-native query performance on frequent access patterns without discarding the source data.
A Materialized View feeding a separate target table is a more powerful variant. It lets you apply arbitrary transformations, fan out to multiple destination tables with different schemas, and change extraction logic without altering the landing table. For pipelines ingesting via Kafka using the Kafka engine, a MV writing to a structured destination is the standard architecture. The Kafka table acts as a buffer; the MV's SELECT clause does the extraction once per message; the destination table sees only typed columns.
The practical decision tree: if your schema is fully known, use Tuple. If keys are dynamic but values share a type, consider Map. If you are building a log or event store with truly arbitrary JSON and query flexibility is the priority, the native JSON type is where ClickHouse is headed - but benchmark your ingest throughput before committing. For everything in between, raw String with materialized columns for your hot fields remains the most predictable and operationally safe choice.