Skip to content

Commit

Permalink
Spec: Add variant type (#10831)
Browse files Browse the repository at this point in the history
---------

Co-authored-by: Aihua Xu <[email protected]>
  • Loading branch information
aihuaxu and sfc-gh-aixu authored Jan 29, 2025
1 parent c787134 commit 5da8f5f
Showing 1 changed file with 24 additions and 2 deletions.
26 changes: 24 additions & 2 deletions format/spec.md
Original file line number Diff line number Diff line change
Expand Up @@ -182,6 +182,20 @@ A **`list`** is a collection of values with some element type. The element field

A **`map`** is a collection of key-value pairs with a key type and a value type. Both the key field and value field each have an integer id that is unique in the table schema. Map keys are required and map values can be either optional or required. Both map keys and map values may be any type, including nested types.

#### Semi-structured Types

A **`variant`** is a value that stores semi-structured data. The structure and data types in a variant are not necessarily consistent across rows in a table or data file. The variant type and binary encoding are defined in the [Parquet project](https://github.com/apache/parquet-format/blob/master/VariantEncoding.md), with support currently available for V1. Support for Variant is added in Iceberg v3.

Variants are similar to JSON with a wider set of primitive values including date, timestamp, timestamptz, binary, and decimals.

Variant values may contain nested types:
1. An array is an ordered collection of variant values.
2. An object is a collection of fields that are a string key and a variant value.

As a semi-structured type, there are important differences between variant and Iceberg's other types:
1. Variant arrays are similar to lists, but may contain any variant value rather than a fixed element type.
2. Variant objects are similar to structs, but may contain variable fields identified by name and field values may be any variant value rather than a fixed field type.

#### Primitive Types

Supported primitive types are defined in the table below. Primitive types added after v1 have an "added by" version that is the first spec version in which the type is allowed. For example, nanosecond-precision timestamps are part of the v3 spec; using v3 types in v1 or v2 tables can break forward compatibility.
Expand Down Expand Up @@ -454,7 +468,7 @@ Partition field IDs must be reused if an existing partition spec contains an equ

| Transform name | Description | Source types | Result type |
|-------------------|--------------------------------------------------------------|-----------------------------------------------------------------------------------------------------------|-------------|
| **`identity`** | Source value, unmodified | Any | Source type |
| **`identity`** | Source value, unmodified | Any except for `variant` | Source type |
| **`bucket[N]`** | Hash of value, mod `N` (see below) | `int`, `long`, `decimal`, `date`, `time`, `timestamp`, `timestamptz`, `timestamp_ns`, `timestamptz_ns`, `string`, `uuid`, `fixed`, `binary` | `int` |
| **`truncate[W]`** | Value truncated to width `W` (see below) | `int`, `long`, `decimal`, `string`, `binary` | Source type |
| **`year`** | Extract a date or timestamp year, as years from 1970 | `date`, `timestamp`, `timestamptz`, `timestamp_ns`, `timestamptz_ns` | `int` |
Expand Down Expand Up @@ -1164,6 +1178,7 @@ Maps with non-string keys must use an array representation with the `map` logica
|**`struct`**|`record`||
|**`list`**|`array`||
|**`map`**|`array` of key-value records, or `map` when keys are strings (optional).|Array storage must use logical type name `map` and must store elements that are 2-field records. The first field is a non-null key and the second field is the value.|
|**`variant`**|`record` with `metadata` and `value` fields. `metadata` and `value` must not be assigned field IDs and the fields are accessed through names. |Shredding is not supported in Avro.|

Notes:

Expand Down Expand Up @@ -1218,6 +1233,7 @@ Lists must use the [3-level representation](https://github.com/apache/parquet-fo
| **`struct`** | `group` | | |
| **`list`** | `3-level list` | `LIST` | See Parquet docs for 3-level representation. |
| **`map`** | `3-level map` | `MAP` | See Parquet docs for 3-level representation. |
| **`variant`** | `group` with `metadata` and `value` fields. `metadata` and `value` must not be assigned field IDs and the fields are accessed through names.| `VARIANT` | See Parquet docs for [Variant encoding](https://github.com/apache/parquet-format/blob/master/VariantEncoding.md) and [Variant shredding encoding](https://github.com/apache/parquet-format/blob/master/VariantShredding.md). |


When reading an `unknown` column, any corresponding column must be ignored and replaced with `null` values.
Expand Down Expand Up @@ -1249,6 +1265,7 @@ When reading an `unknown` column, any corresponding column must be ignored and r
| **`struct`** | `struct` | | |
| **`list`** | `array` | | |
| **`map`** | `map` | | |
| **`variant`** | `struct` with `metadata` and `value` fields. `metadata` and `value` must not be assigned field IDs. | `iceberg.struct-type`=`VARIANT` | Shredding is not supported in ORC. |

Notes:

Expand Down Expand Up @@ -1295,6 +1312,8 @@ The types below are not currently valid for bucketing, and so are not hashed. Ho
| **`float`** | `hashLong(doubleToLongBits(double(v))` [5]| `1.0F``-142385009`, `0.0F``1669671676`, `-0.0F``1669671676` |
| **`double`** | `hashLong(doubleToLongBits(v))` [5]| `1.0D``-142385009`, `0.0D``1669671676`, `-0.0D``1669671676` |

A 32-bit hash is not defined for `variant` because there are multiple representations for equivalent values.

Notes:

1. Integer and long hash results must be identical for all integer values. This ensures that schema evolution does not change bucket partition values if integer types are promoted.
Expand Down Expand Up @@ -1341,6 +1360,7 @@ Types are serialized according to this table:
|**`struct`**|`JSON object: {`<br />&nbsp;&nbsp;`"type": "struct",`<br />&nbsp;&nbsp;`"fields": [ {`<br />&nbsp;&nbsp;&nbsp;&nbsp;`"id": <field id int>,`<br />&nbsp;&nbsp;&nbsp;&nbsp;`"name": <name string>,`<br />&nbsp;&nbsp;&nbsp;&nbsp;`"required": <boolean>,`<br />&nbsp;&nbsp;&nbsp;&nbsp;`"type": <type JSON>,`<br />&nbsp;&nbsp;&nbsp;&nbsp;`"doc": <comment string>,`<br />&nbsp;&nbsp;&nbsp;&nbsp;`"initial-default": <JSON encoding of default value>,`<br />&nbsp;&nbsp;&nbsp;&nbsp;`"write-default": <JSON encoding of default value>`<br />&nbsp;&nbsp;&nbsp;&nbsp;`}, ...`<br />&nbsp;&nbsp;`] }`|`{`<br />&nbsp;&nbsp;`"type": "struct",`<br />&nbsp;&nbsp;`"fields": [ {`<br />&nbsp;&nbsp;&nbsp;&nbsp;`"id": 1,`<br />&nbsp;&nbsp;&nbsp;&nbsp;`"name": "id",`<br />&nbsp;&nbsp;&nbsp;&nbsp;`"required": true,`<br />&nbsp;&nbsp;&nbsp;&nbsp;`"type": "uuid",`<br />&nbsp;&nbsp;&nbsp;&nbsp;`"initial-default": "0db3e2a8-9d1d-42b9-aa7b-74ebe558dceb",`<br />&nbsp;&nbsp;&nbsp;&nbsp;`"write-default": "ec5911be-b0a7-458c-8438-c9a3e53cffae"`<br />&nbsp;&nbsp;`}, {`<br />&nbsp;&nbsp;&nbsp;&nbsp;`"id": 2,`<br />&nbsp;&nbsp;&nbsp;&nbsp;`"name": "data",`<br />&nbsp;&nbsp;&nbsp;&nbsp;`"required": false,`<br />&nbsp;&nbsp;&nbsp;&nbsp;`"type": {`<br />&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;`"type": "list",`<br />&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;`...`<br />&nbsp;&nbsp;&nbsp;&nbsp;`}`<br />&nbsp;&nbsp;`} ]`<br />`}`|
|**`list`**|`JSON object: {`<br />&nbsp;&nbsp;`"type": "list",`<br />&nbsp;&nbsp;`"element-id": <id int>,`<br />&nbsp;&nbsp;`"element-required": <bool>`<br />&nbsp;&nbsp;`"element": <type JSON>`<br />`}`|`{`<br />&nbsp;&nbsp;`"type": "list",`<br />&nbsp;&nbsp;`"element-id": 3,`<br />&nbsp;&nbsp;`"element-required": true,`<br />&nbsp;&nbsp;`"element": "string"`<br />`}`|
|**`map`**|`JSON object: {`<br />&nbsp;&nbsp;`"type": "map",`<br />&nbsp;&nbsp;`"key-id": <key id int>,`<br />&nbsp;&nbsp;`"key": <type JSON>,`<br />&nbsp;&nbsp;`"value-id": <val id int>,`<br />&nbsp;&nbsp;`"value-required": <bool>`<br />&nbsp;&nbsp;`"value": <type JSON>`<br />`}`|`{`<br />&nbsp;&nbsp;`"type": "map",`<br />&nbsp;&nbsp;`"key-id": 4,`<br />&nbsp;&nbsp;`"key": "string",`<br />&nbsp;&nbsp;`"value-id": 5,`<br />&nbsp;&nbsp;`"value-required": false,`<br />&nbsp;&nbsp;`"value": "double"`<br />`}`|
| **`variant`**| `JSON string: "variant"`|`"variant"`|

Note that default values are serialized using the JSON single-value serialization in [Appendix D](#appendix-d-single-value-serialization).

Expand Down Expand Up @@ -1490,6 +1510,7 @@ This serialization scheme is for storing single values as individual binary valu
| **`struct`** | Not supported |
| **`list`** | Not supported |
| **`map`** | Not supported |
| **`variant`** | Not supported |

### JSON single-value serialization

Expand Down Expand Up @@ -1518,6 +1539,7 @@ This serialization scheme is for storing single values as individual binary valu
| **`map`** | **`JSON object of key and value arrays`** | `{ "keys": ["a", "b"], "values": [1, 2] }` | Stores arrays of keys and values; individual keys and values are serialized using this JSON single-value format |



## Appendix E: Format version changes

### Version 3
Expand All @@ -1527,7 +1549,7 @@ Default values are added to struct fields in v3.
* The `write-default` is a forward-compatible change because it is only used at write time. Old writers will fail because the field is missing.
* Tables with `initial-default` will be read correctly by older readers if `initial-default` is always null for optional fields. Otherwise, old readers will default optional columns with null. Old readers will fail to read required fields which are populated by `initial-default` because that default is not supported.

Types `unknown`, `timestamp_ns`, and `timestamptz_ns` are added in v3.
Types `variant`, `unknown`, `timestamp_ns`, and `timestamptz_ns` are added in v3.

All readers are required to read tables with unknown partition transforms, ignoring the unsupported partition fields when filtering.

Expand Down

0 comments on commit 5da8f5f

Please sign in to comment.