rahil-c commented on code in PR #18274:
URL: https://github.com/apache/hudi/pull/18274#discussion_r2907383674
##########
rfc/rfc-99/rfc-99.md:
##########
@@ -209,4 +209,299 @@ SQL Extensions needs to be added to define the table in a
hudi type native way.
TODO: There is an open question regarding the need to maintain type ids to
track schema evolution and how it would interplay with NBCC.
-The main implementation change would require replacing the Avro schema
references with the new type system.
+The main implementation change would require replacing the Avro schema
references with the new type system.
+
+---
+
+## Variant Type Implementation
+
+This section documents the implementation of the VARIANT type in Hudi, which
provides first-class support for semi-structured data (e.g., JSON). The Variant
type is implemented following Spark 4.0's native VariantType specification.
+
+### Overview
+
+The Variant type enables Hudi to store and query semi-structured data
efficiently. It is particularly useful for:
+- Schema-on-read flexibility for evolving data structures
+- Storing JSON-like data without requiring predefined schemas
+
+### Architecture
+
+Variant support is built on a **layered architecture** with version-specific
adapters:
+
+```
+┌────────────────────────────────────────────────────┐
+│ Application Layer (Spark SQL) │
+│ SELECT parse_json('{"a": 1}') as data │
+└────────────────────────────────────────────────────┘
+ │
+ ▼
+┌────────────────────────────────────────────────────┐
+│ Spark Version Adapters │
+│ ┌──────────────────┐ ┌────────────────────────┐ │
+│ │ BaseSpark3Adapter│ │ BaseSpark4Adapter │ │
+│ │ (No Variant) │ │ (Full Variant) │ │
+│ └──────────────────┘ └────────────────────────┘ │
+└────────────────────────────────────────────────────┘
+ │
+ ▼
+┌────────────────────────────────────────────────────┐
+│ HoodieSchema.Variant │
+│ (Avro Logical Type + Record Schema) │
+└────────────────────────────────────────────────────┘
+ │
+ ▼
+┌────────────────────────────────────────────────────┐
+│ Parquet Storage │
+│ GROUP { value: BINARY, metadata: BINARY } │
+└────────────────────────────────────────────────────┘
+```
+
+### Variant Schema Definition
+
+The `HoodieSchema.Variant` class in `hudi-common` defines the Variant type:
+
+```java
+public static class Variant extends HoodieSchema {
+ private static final String VARIANT_METADATA_FIELD = "metadata";
+ private static final String VARIANT_VALUE_FIELD = "value";
+ private static final String VARIANT_TYPED_VALUE_FIELD = "typed_value";
+
+ private final boolean isShredded;
+ private final Option<HoodieSchema> typedValueSchema;
+}
+```
+
+#### Two Storage Modes
+
+1. **Unshredded Variant** (Default):
+ - Created with: `HoodieSchema.createVariant()`
+ - Structure: Record with two REQUIRED binary fields
+ - Fields: `metadata` (BYTES, REQUIRED), `value` (BYTES, REQUIRED)
+ - Use case: Simple semi-structured data storage
+
+2. **Shredded Variant** (Future Enhancement):
+ - Created with: `HoodieSchema.createVariantShredded(typedValueSchema)`
+ - Structure: Record with optional `typed_value` field
+ - Fields: `value` (BYTES, OPTIONAL), `metadata` (BYTES, REQUIRED),
`typed_value` (optional)
+ - Use case: Schema evolution where certain fields are extracted and typed
for optimized access
+
+#### Custom Avro Logical Type
+
+Variant uses a custom Avro logical type for identification:
+
+```java
+public static class VariantLogicalType extends LogicalType {
+ private static final String VARIANT_LOGICAL_TYPE_NAME = "variant";
+}
+```
+
+### On-Disk Representation (Parquet)
+
+Variant data is stored in Parquet as a GROUP type with binary fields:
+
+```
+message schema {
+ required group variant_column {
+ required binary value;
+ required binary metadata;
+ }
+}
+```
+
+#### Binary Format
+
+The Variant type follows Spark 4.0's internal binary representation:
+
+| Component | Description |
+|-----------|-------------|
+| **value** | Binary encoding of the actual data (scalars, objects, arrays) |
+| **metadata** | Dictionary of field names and type information for efficient
access |
+
+Example for `{"updated": true, "new_field": 123}`:
+
+```
+Value Bytes: [0x02, 0x02, 0x01, 0x00, 0x01, 0x00, 0x03, 0x04, 0x0C, 0x7B]
+Metadata Bytes: [0x01, 0x02, 0x00, 0x07, 0x10, "updated", "new_field"]
+```
+
+The metadata contains a dictionary of all field names, while the value
contains references to these fields plus the actual data values.
+
+### Schema Evolution Support
+
+Variant types provide **schema-on-read** flexibility:
+
+| Aspect | Behavior |
+|--------|----------|
+| Adding new fields | ✅ Supported - New JSON fields can be added without
schema changes |
+| Removing fields | ✅ Supported - Missing fields return null on read |
+| Type changes within JSON | ✅ Supported - Variant can store any
JSON-compatible type |
+| Table schema evolution | ✅ Supported - Variant column can be added to
existing tables |
+| Hudi schema evolution | ✅ Supported - Works with Hudi's standard schema
evolution |
+
+**Important**: The schema flexibility is within the Variant column itself. The
table-level schema (including the Variant column definition) still follows
Hudi's standard schema evolution rules.
+
+### Column Statistics and Indexing
+
+| Feature | Support Status |
+|---------|----------------|
+| Min/Max statistics | ❌ Not supported - Variant values are opaque binary
blobs |
+| Bloom filter index | ❌ Not supported for Variant columns |
+| Column statistics in MDT | ❌ Not supported |
+| Partition pruning | ❌ Not applicable to Variant columns |
+| Predicate pushdown | ❌ Limited - Only structural predicates (IS NULL, IS NOT
NULL) |
+
+**Recommendation**: For query performance, consider extracting
frequently-accessed fields into dedicated typed columns alongside the Variant
column.
+
+### Usage Guide
+
+#### Spark 4.0+ (Native Support)
+
+```sql
+-- Create table with Variant column
+CREATE TABLE events (
+ id STRING,
+ ts TIMESTAMP,
+ payload VARIANT
+) USING hudi
+OPTIONS (
+ primaryKey = 'id',
+ preCombineField = 'ts'
+);
+
+-- Insert with parse_json
+INSERT INTO events VALUES
+ ('1', current_timestamp(), parse_json('{"event": "click", "page":
"/home"}')),
+ ('2', current_timestamp(), parse_json('{"event": "purchase", "amount":
99.99}'));
+
+-- Query Variant data
+SELECT id, payload:event, payload:amount FROM events;
+
+-- Update Variant column
+UPDATE events SET payload = parse_json('{"event": "click", "page":
"/products"}') WHERE id = '1';
+
+-- Works with both COW and MOR tables
+CREATE TABLE events_mor (
+ id STRING,
+ ts TIMESTAMP,
+ payload VARIANT
+) USING hudi
+TBLPROPERTIES (
+ 'type' = 'mor',
+ 'primaryKey' = 'id',
+ 'preCombineField' = 'ts'
+);
+```
+
+#### Spark 3.x (Backward Compatibility Read)
+
+Spark 3.x does not support VariantType natively, but can read Variant tables
as struct:
+
+```sql
+-- Reading Spark 4.0 Variant table in Spark 3.x
+-- Variant column appears as: STRUCT<value: BINARY, metadata: BINARY>
+
+SELECT id, cast(payload.value as string) FROM events;
+```
+
+**Limitations in Spark 3.x**:
+- Cannot write Variant data
+- Variant column reads as raw struct with binary fields
+- No helper functions like `parse_json()` available
+
+### Cross-Engine Compatibility
+
+#### Flink Integration
+
+| Operation | Support |
+|-----------|---------|
+| Reading Spark-written Variant tables | ✅ Supported |
+| Variant representation in Flink | `ROW<value BYTES, metadata BYTES>` |
+| Writing Variant from Flink | ❌ Not yet implemented |
+
+Example Flink query reading Variant data:
+```sql
+-- Flink sees Variant as ROW type
+SELECT id, variant_col.value, variant_col.metadata FROM hudi_variant_table;
+```
+
+#### Avro Serialization (MOR Tables)
+
+For MOR tables, Variant data is serialized to Avro for log files:
+
+```
+{
+ "type": "record",
+ "logicalType": "variant",
+ "fields": [
+ {"name": "value", "type": "bytes"},
+ {"name": "metadata", "type": "bytes"}
+ ]
+}
+```
+
+### Backward Compatibility
+
+The implementation ensures backward compatibility through:
+
+1. **Storage Format**: Variants stored as regular Parquet records (no special
Parquet extension required)
+2. **Logical Type Annotation**: Avro logical type allows newer versions to
recognize Variant semantics
+3. **Graceful Degradation**: Older readers see Variant as `STRUCT<value:
BINARY, metadata: BINARY>`
+
+| Scenario | Behavior |
+|----------|----------|
+| Spark 4.0 writes, Spark 4.0 reads | Full Variant support |
+| Spark 4.0 writes, Spark 3.x reads | Struct with binary fields |
+| Spark 4.0 writes, Flink reads | ROW with binary fields |
+| Spark 3.x writes Variant | ❌ Not supported |
+
+### Limitations and Constraints
+
+1. **Spark Version Dependency**:
+ - Write support requires Spark 4.0+
+ - Spark 3.x limited to read-only access with degraded experience
+
+2. **Storage Overhead**:
+ - Metadata stored redundantly per row
+ - No column-level compression optimizations for Variant content
+
+3. **Query Performance**:
+ - No predicate pushdown into Variant content
+ - Full row scan required for Variant field access
+ - Consider extracting hot columns for better performance
+
+4. **Shredded Variant**:
+ - `typed_value` field defined in schema but not populated in current
implementation
Review Comment:
Can you give an example of what typed_value would be in the context of a
user query?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]