Job Board
Consulting

The VARIANT Data Type in Spark 4.0: Semi-Structured Data Without Schema Headaches

Spark 4.0 added a native VARIANT type (SPARK-45827) for storing semi-structured data in a compact binary format you can query directly — no upfront schema, no from_json ceremony on every read. The published benchmarks show roughly 8x faster reads than storing the same payload as a JSON string column, and Spark 4.1 adds shredding to push that further. This article shows how the Scala API works, when to reach for VARIANT, and when you still want a strongly typed StructType.

For the broader release picture, see What's New in Spark 4.0 for Scala Developers and Upgrading from Spark 3.x to Spark 4.0: A Practical Guide.


Why VARIANT Exists

Before Spark 4.0, every team handling JSON-shaped data ended up at the same fork in the road:

  • Store JSON as StringType and parse on read. Cheap to write, painful to query. Every from_json call needs a schema, and every analyst either invents one inline or pastes one from a doc that's two months out of date. The string column is opaque to the optimizer.
  • Pin a schema with StructType up front. Fast and type-safe, but every new field upstream is a schema migration. If the producer adds device.os.locale on Thursday, your from_json either silently drops it (returns null) or fails the job depending on your columnNameOfCorruptRecord settings.

VARIANT is the third option. It stores semi-structured data in a binary encoding that preserves the full nested structure, lets you query paths directly with dot/bracket syntax, and does not require you to declare a schema. The encoding is an open specification shared with Parquet and the Delta Lake project, so the format is not Spark-proprietary.

The performance gain comes from the fact that VARIANT is parsed once at ingest, not on every query. Databricks' published numbers (Runtime 15.0 with Photon) show 8x improvement over equivalent String columns for both nested and flat schemas. Spark 4.0 ships the core type and functions; Spark 4.1 adds shredding — projecting frequently-accessed paths into columnar storage so common queries skip the binary decode entirely.


The Core Functions

Spark 4.0 ships a small, focused set of VARIANT functions. They are all available from SQL; in Scala you reach them through expr() or selectExpr(), since most do not have direct Column method equivalents.

Function Returns Purpose
parse_json(str) VARIANT Parse a JSON string into a VARIANT value. Throws on invalid JSON.
try_parse_json(str) VARIANT Same, but returns null on invalid input.
variant_get(v, path, type) typed Extract a path and cast it to a concrete Spark type. Throws on cast failure.
try_variant_get(v, path, type) typed Same, but returns null on extraction or cast failure.
schema_of_variant(v) STRING Returns the inferred schema of a single VARIANT value as a DDL string.
schema_of_variant_agg(v) STRING Aggregate version — infers a unified schema across all rows.
is_variant_null(v) BOOLEAN Distinguishes JSON null from SQL NULL.

There is also the : colon-path syntax (raw:store.bicycle.price) which is shorthand for variant_get — handy in SQL, less useful from the Scala DSL.


Reading JSON into VARIANT

The natural entry point is parse_json. Given a string column of JSON, one call gets you a queryable VARIANT.

import org.apache.spark.sql.functions._
import spark.implicits._

val rawEvents = Seq(
  """{"user": "alice", "action": "click", "target": {"id": "btn_save", "page": "/checkout"}}""",
  """{"user": "bob",   "action": "purchase", "amount": 49.99, "items": ["widget_a"]}""",
  """{"user": "carol", "action": "view", "target": {"id": "card_3", "page": "/home"}, "tags": ["promo", "new_user"]}"""
).toDF("json")

val events = rawEvents.withColumn("payload", expr("parse_json(json)"))

events.printSchema()
// root
//  |-- json: string (nullable = true)
//  |-- payload: variant (nullable = true)

Note the schema: payload is now variant, a first-class type — not a struct, not a string. The three rows have three different shapes and that is fine. The VARIANT column does not need them to agree.

If your input is already non-string JSON (e.g., from a Kafka source where the value is parsed by the connector), you can build VARIANT values from arbitrary Spark expressions; parse_json is just the most common path.


Querying Nested Fields

This is where VARIANT pays off. You query paths directly, with optional type casts.

import org.apache.spark.sql.functions._

events.selectExpr(
  "variant_get(payload, '$.user', 'string')        AS user",
  "variant_get(payload, '$.action', 'string')      AS action",
  "variant_get(payload, '$.target.id', 'string')   AS target_id",
  "variant_get(payload, '$.amount', 'double')      AS amount"
).show(false)

// +-----+--------+---------+------+
// |user |action  |target_id|amount|
// +-----+--------+---------+------+
// |alice|click   |btn_save |NULL  |
// |bob  |purchase|NULL     |49.99 |
// |carol|view    |card_3   |NULL  |
// +-----+--------+---------+------+

A few things to notice:

  • Missing paths return null. There is no exception when amount is absent on a click event or target.id is absent on a purchase. That's the point — VARIANT is for data where presence is not guaranteed.
  • The path uses JSONPath-style $.foo.bar syntax. Bracket indexing works for arrays: $.items[0].
  • The third argument is a Spark SQL type, not a Scala type. Pass it as a string ('string', 'double', 'array<string>', 'struct<id: string, page: string>').

If a value is present but cannot be cast to the requested type, variant_get throws under ANSI mode (which is now the default in Spark 4.0). When the source data is messy, swap to try_variant_get:

// Some rows have amount as a string like "49.99", some as a number, some absent.
events.selectExpr(
  "try_variant_get(payload, '$.amount', 'double') AS amount"
).show()

try_variant_get is to variant_get what try_cast is to cast. Reach for it at the expression level where null is the genuine semantic; keep variant_get everywhere else so the optimizer (and your future self) can see strict type expectations.


Extracting a Struct in One Shot

Plucking individual fields gets verbose. When you know the shape you want, cast the whole VARIANT (or a subpath) to a StructType:

events.selectExpr(
  "variant_get(payload, '$.target', 'struct<id: string, page: string>') AS target"
).show(false)

// +------------------+
// |target            |
// +------------------+
// |{btn_save, /checkout}|
// |NULL              |
// |{card_3, /home}   |
// +------------------+

This is the seam between schema-less and schema-bound. Store the data as VARIANT, then project the well-known parts of the payload into a typed struct at query time. New fields that the producer adds later are still in the underlying VARIANT — you just have not asked for them yet.


Inferring a Schema

When you don't know the shape, ask Spark. schema_of_variant_agg walks every row and returns a unified DDL string.

val mixed = Seq(
  """{"flag": null}""",
  """{"other": 1}""",
  null
).toDF("json")
  .withColumn("payload", expr("try_parse_json(json)"))
  .withColumn("flag", expr("variant_get(payload, '$.flag', 'string')"))

mixed.selectExpr(
  "flag",
  "flag IS NULL                                    AS sql_is_null",
  "is_variant_null(variant_get(payload, '$.flag')) AS variant_is_null"
).show()

// +----+-----------+---------------+
// |flag|sql_is_null|variant_is_null|
// +----+-----------+---------------+
// |NULL|       true|           true|   <- explicit JSON null
// |NULL|       true|          false|   <- key absent
// |NULL|       true|           NULL|   <- whole row was SQL NULL
// +----+-----------+---------------+
This distinction matters for change-data-capture pipelines, where a `null` patch often means "unset this field" and a missing key means "no change". If you've been losing that information by stringifying JSON and round-tripping it, VARIANT preserves it. --- ## Shredding in Spark 4.1 Spark 4.0 ships VARIANT as a single binary blob per row. Querying a path means decoding that blob — fast, but not free, and not as fast as columnar storage for the same field. Spark 4.1 adds **shredding**: when you persist a VARIANT column to Parquet (or Delta), commonly-accessed paths can be materialized as their own typed sub-columns alongside the binary representation. Queries that touch a shredded path skip the binary decode entirely and read a normal Parquet column. Queries that touch an un-shredded path fall back to the binary blob. The format is fully backward-compatible — readers that don't understand shredding still see a valid VARIANT column. You don't need to change your queries to benefit from shredding. The shredding decision is made at write time based on the storage engine's configuration. From Scala code, `parse_json` and `variant_get` look identical in 4.0 and 4.1. The win is in the Parquet/Delta layer. For Delta Lake specifically, see [Delta Lake 4.0: What Scala Engineers Need to Know](/latest/2026/03/26/delta-lake-4-0-scala-engineers/) for the storage-side details. --- ## When to Use VARIANT vs `StructType` This is the decision you actually have to make. The short version: | Situation | Reach for | |---|---| | Schema is stable, every field is required, you control the producer | `StructType` | | Schema is stable but optional fields proliferate, you tolerate nulls | `StructType` with nullable fields | | Schema drifts over time, you don't want a migration every time a field is added | **`VARIANT`** | | Payloads from many sources with overlapping but non-identical shapes | **`VARIANT`** | | Audit log / event store / EDR / clickstream where the schema is "whatever the upstream emits" | **`VARIANT`** | | You need to enforce shape at the table boundary | `StructType` | | You need partition or Z-order columns based on this field | `StructType` (VARIANT cannot be a partition or clustering key) | | You need MERGE on a field within the payload | Project that field into a typed column | The mixed-use pattern that works in practice: store the full payload as VARIANT, *and* project the handful of fields you actually partition on, join on, or aggregate frequently into their own typed columns. That gets you the storage flexibility of VARIANT plus the optimizer-friendly query shape on the hot paths. --- ## Known Limitations A few things to be aware of, especially if you're coming from a more permissive storage layer: - **No partitioning, clustering, or Z-ordering on VARIANT columns.** If you need those, project the field out into a normal column. - **Limited streaming source support.** Most file-based and Kafka sources work, but check the specific connector. The Spark Connect Scala client gained VARIANT support in 4.0 via [SPARK-50063](https://issues.apache.org/jira/browse/SPARK-50063). - **Cast errors are real under ANSI mode.** `variant_get(v, '$.amount', 'int')` will throw if the value at that path is `"49.99"`. Use `try_variant_get` for dirty inputs, exactly like `try_cast`. - **`schema_of_variant_agg` is expensive.** Treat it as a one-time exploration tool, not a per-query helper. - **No direct `Column` methods.** Everything goes through `expr()` or `selectExpr()` for now. The `org.apache.spark.sql.functions._` package may grow direct methods in later releases; for 4.0 and 4.1 you write `expr("parse_json(...)")`. --- ## The Bottom Line `VARIANT` closes the gap between "store JSON as a string and pay on every read" and "pin a schema and migrate every time the producer breathes." It is not a replacement for `StructType` — well-known, stable shapes still belong in typed structs. It is the right answer for the messy half of the data you've been awkwardly stringifying. If you're starting a new pipeline in Spark 4.0+ and the payload is genuinely semi-structured, default to VARIANT. If you're migrating an existing pipeline, the first move is usually to convert a string-of-JSON column to VARIANT at the boundary and leave the downstream `from_json` calls alone — then peel them back to `variant_get` as you touch each query. The encoding is open, the format is shared with Parquet and Delta, and 4.1's shredding lands the performance story squarely in columnar territory. For the full type and function reference, see the [Spark 4.0 release notes](https://spark.apache.org/releases/spark-release-4-0-0.html) and the [Variant encoding specification](https://github.com/apache/parquet-format/blob/master/VariantEncoding.md).
Article Details
More Spark Scala News and Updates

Created: 2026-05-12

Last Updated: 2026-05-12 10:10:31 PM