Skip to content
Merged
Show file tree
Hide file tree
Changes from 3 commits
Commits
Show all changes
27 commits
Select commit Hold shift + click to select a range
1bd4269
[Data] Add map namespace support for expression operations
ryankert01 Jan 6, 2026
2e157bd
Merge branch 'master' into map-expression
ryankert01 Jan 6, 2026
68bef64
address ai review
ryankert01 Jan 6, 2026
fe2642b
fix cursor bot suggestions
ryankert01 Jan 6, 2026
843cac1
Merge branch 'master' into map-expression
ryankert01 Jan 6, 2026
f16bfd1
Merge remote-tracking branch 'origin/master' into map-expression
ryankert01 Jan 12, 2026
df1fe8c
refactor tests
ryankert01 Jan 12, 2026
6461062
Merge branch 'master' into map-expression
ryankert01 Jan 12, 2026
fcd3652
Merge branch 'master' into map-expression
ryankert01 Jan 18, 2026
202a652
Merge branch 'master' into map-expression
owenowenisme Jan 21, 2026
50a2e64
Update python/ray/data/namespace_expressions/map_namespace.py
ryankert01 Jan 22, 2026
e613cfa
address commits
ryankert01 Jan 22, 2026
70a3760
Merge branch 'master' into map-expression
ryankert01 Jan 22, 2026
49268ec
Merge branch 'master' into map-expression
ryankert01 Jan 25, 2026
c390a24
create 3 helper functions to make the intent clearer
ryankert01 Jan 25, 2026
5e024c8
use numpy.repeat()
ryankert01 Jan 25, 2026
10e4b7c
text extractioon on empty chunkedArray
ryankert01 Jan 25, 2026
f9d53b8
Merge branch 'master' into map-expression
ryankert01 Jan 25, 2026
978132e
lint
ryankert01 Jan 25, 2026
2eff519
Merge remote-tracking branch 'origin/map-expression' into map-expression
ryankert01 Jan 25, 2026
7a11478
Merge branch 'master' into map-expression
goutamvenkat-anyscale Feb 4, 2026
dae4645
Merge branch 'master' into map-expression
ryankert01 Feb 8, 2026
59f8047
address comments
ryankert01 Feb 8, 2026
ef6f899
Merge branch 'master' into map-expression
ryankert01 Mar 2, 2026
5df9b8f
[fix] Update type hint for _get_child_array return value to Optional[…
ryankert01 Mar 3, 2026
183aec1
Merge branch 'master' into map-expression
ryankert01 Mar 3, 2026
2aebeb6
[fix] Refactor _rebuild_list_array and _get_result_type to support La…
ryankert01 Mar 3, 2026
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
13 changes: 13 additions & 0 deletions python/ray/data/expressions.py
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
if TYPE_CHECKING:
from ray.data.namespace_expressions.dt_namespace import _DatetimeNamespace
from ray.data.namespace_expressions.list_namespace import _ListNamespace
from ray.data.namespace_expressions.map_namespace import _MapNamespace
from ray.data.namespace_expressions.string_namespace import _StringNamespace
from ray.data.namespace_expressions.struct_namespace import _StructNamespace

Expand Down Expand Up @@ -634,6 +635,13 @@ def struct(self) -> "_StructNamespace":

return _StructNamespace(self)

@property
def map(self) -> "_MapNamespace":
"""Access map/dict operations for this expression."""
from ray.data.namespace_expressions.map_namespace import _MapNamespace

return _MapNamespace(self)

@property
def dt(self) -> "_DatetimeNamespace":
"""Access datetime operations for this expression."""
Expand Down Expand Up @@ -1481,6 +1489,7 @@ def download(uri_column_name: str) -> DownloadExpr:
"_ListNamespace",
"_StringNamespace",
"_StructNamespace",
"_MapNamespace",
"_DatetimeNamespace",
]

Expand All @@ -1499,6 +1508,10 @@ def __getattr__(name: str):
from ray.data.namespace_expressions.struct_namespace import _StructNamespace

return _StructNamespace
elif name == "_MapNamespace":
from ray.data.namespace_expressions.map_namespace import _MapNamespace

return _MapNamespace
elif name == "_DatetimeNamespace":
from ray.data.namespace_expressions.dt_namespace import _DatetimeNamespace

Expand Down
158 changes: 158 additions & 0 deletions python/ray/data/namespace_expressions/map_namespace.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,158 @@
from __future__ import annotations

from dataclasses import dataclass
from enum import Enum
from typing import TYPE_CHECKING

import pyarrow
import pyarrow.compute as pc

from ray.data.datatype import DataType
from ray.data.expressions import pyarrow_udf

if TYPE_CHECKING:
from ray.data.expressions import Expr, UDFExpr


class MapComponent(str, Enum):
KEYS = "keys"
VALUES = "values"


def _extract_map_component(
arr: pyarrow.Array, component: MapComponent
) -> pyarrow.Array:
"""
Extracts keys or values from a MapArray or ListArray<Struct>.

This serves as the primary implementation since PyArrow does not yet
expose dedicated compute kernels for map projection in the Python API.
"""
# 1. Handle Chunked Arrays (Recursion)
if isinstance(arr, pyarrow.ChunkedArray):
return pyarrow.chunked_array(
[_extract_map_component(chunk, component) for chunk in arr.chunks]
)

child_array = None

# Case 1: MapArray
if isinstance(arr, pyarrow.MapArray):
if component == MapComponent.KEYS:
child_array = arr.keys
else:
child_array = arr.items

# Case 2: ListArray<Struct<Key, Value>>
elif isinstance(arr, (pyarrow.ListArray, pyarrow.LargeListArray)):
flat_values = arr.values
if (
isinstance(flat_values, pyarrow.StructArray)
and flat_values.type.num_fields >= 2
):
idx = 0 if component == MapComponent.KEYS else 1
child_array = flat_values.field(idx)

if child_array is None:
# This can happen if the input array is not a supported map type.
# We allow this to proceed only if the array is empty or all-nulls,
# in which case we'll produce an empty or all-nulls output.
if len(arr) > 0 and arr.null_count < len(arr):
raise TypeError(
f"Expression is not a map type. .map.{component.value}() can only be "
f"called on MapArray or List<Struct<key, value>> types, but got {arr.type}."
)
child_array = pyarrow.array([], type=pyarrow.null())

# Reconstruct ListArray & Normalize Offsets
offsets = arr.offsets
if len(offsets) > 0: # Handle offsets changes
start_offset = offsets[0]
if start_offset.as_py() != 0:
# Slice child_array to match normalized offsets
end_offset = offsets[-1]
child_array = child_array.slice(
offset=start_offset.as_py(), length=(end_offset - start_offset).as_py()
)
offsets = pc.subtract(offsets, start_offset)

return pyarrow.ListArray.from_arrays(
offsets=offsets, values=child_array, mask=arr.is_null()
)


@dataclass
class _MapNamespace:
"""Namespace for map operations on expression columns.

This namespace provides methods for operating on map-typed columns
(including MapArrays and ListArrays of Structs) using PyArrow UDFs.

Example:
>>> from ray.data.expressions import col
>>> # Get keys from map column
>>> expr = col("headers").map.keys()
>>> # Get values from map column
>>> expr = col("headers").map.values()
"""

_expr: "Expr"

def keys(self) -> "UDFExpr":
"""Returns a list expression containing the keys of the map.

Example:
>>> from ray.data.expressions import col
>>> # Get keys from map column
>>> expr = col("headers").map.keys()

Returns:
A list expression containing the keys.
"""
return self._create_projection_udf(MapComponent.KEYS)

def values(self) -> "UDFExpr":
"""Returns a list expression containing the values of the map.

Example:
>>> from ray.data.expressions import col
>>> # Get values from map column
>>> expr = col("headers").map.values()

Returns:
A list expression containing the values.
"""
return self._create_projection_udf(MapComponent.VALUES)

def _create_projection_udf(self, component: MapComponent) -> "UDFExpr":
"""Helper to generate UDFs for map projections."""

return_dtype = DataType(object)
if self._expr.data_type.is_arrow_type():
arrow_type = self._expr.data_type.to_arrow_dtype()

is_physical_map = (
pyarrow.types.is_list(arrow_type)
and pyarrow.types.is_struct(arrow_type.value_type)
and arrow_type.value_type.num_fields >= 2
)

inner_arrow_type = None
if pyarrow.types.is_map(arrow_type):
inner_arrow_type = (
arrow_type.key_type
if component == MapComponent.KEYS
else arrow_type.item_type
)
elif is_physical_map:
idx = 0 if component == MapComponent.KEYS else 1
inner_arrow_type = arrow_type.value_type.field(idx).type

if inner_arrow_type:
return_dtype = DataType.list(DataType.from_arrow(inner_arrow_type))

@pyarrow_udf(return_dtype=return_dtype)
def _project_map(arr: pyarrow.Array) -> pyarrow.Array:
return _extract_map_component(arr, component)

return _project_map(self._expr)
121 changes: 121 additions & 0 deletions python/ray/data/tests/test_namespace_expressions.py
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,21 @@ def _create_dataset(

# Pytest parameterization for all dataset creation formats
DATASET_FORMATS = ["pandas", "arrow"]
MAP_DATASET_FORMATS = ["arrow"]


def _create_map_dataset(dataset_format: str):
"""Create a dataset backed by an Arrow MapArray column."""

map_items = [
{"attrs": {"color": "red", "size": "M"}},
{"attrs": {"brand": "Ray"}},
]
map_type = pa.map_(pa.string(), pa.string())
arrow_table = pa.table(
{"attrs": pa.array([row["attrs"] for row in map_items], type=map_type)}
)
return _create_dataset(map_items, dataset_format, arrow_table)


# ──────────────────────────────────────
Expand Down Expand Up @@ -530,6 +545,112 @@ def test_struct_nested_bracket(self, dataset_format):
assert rows_same(result, expected)


# ──────────────────────────────────────
# Map Namespace Tests
# ──────────────────────────────────────


@pytest.mark.parametrize("dataset_format", MAP_DATASET_FORMATS)
class TestMapNamespace:
"""Tests for map namespace operations."""

def test_map_keys(self, dataset_format):
ds = _create_map_dataset(dataset_format)

result = ds.with_column("keys", col("attrs").map.keys()).to_pandas()
result = result.drop(columns=["attrs"])

expected = pd.DataFrame({"keys": [["color", "size"], ["brand"]]})
assert rows_same(result, expected)

def test_map_values(self, dataset_format):
ds = _create_map_dataset(dataset_format)

result = ds.with_column("values", col("attrs").map.values()).to_pandas()
result = result.drop(columns=["attrs"])

expected = pd.DataFrame({"values": [["red", "M"], ["Ray"]]})
assert rows_same(result, expected)

def test_physical_map_extraction(self, dataset_format):
"""Test extraction works on List<Struct> (Physical Maps)."""
# Construct List<Struct<k, v>>
struct_type = pa.struct([pa.field("k", pa.string()), pa.field("v", pa.int64())])
list_type = pa.list_(struct_type)

data_py = [[{"k": "a", "v": 1}], [{"k": "b", "v": 2}]]
arrow_table = pa.Table.from_arrays(
[pa.array(data_py, type=list_type)], names=["data"]
)

items_data = [{"data": row} for row in data_py]
ds = _create_dataset(items_data, dataset_format, arrow_table)

result = (
ds.with_column("keys", col("data").map.keys())
.with_column("values", col("data").map.values())
.to_pandas()
)

expected = pd.DataFrame(
{
"data": data_py,
"keys": [["a"], ["b"]],
"values": [[1], [2]],
}
)
assert rows_same(result, expected)

def test_map_sliced_offsets(self, dataset_format):
"""Test extraction works correctly on sliced Arrow arrays (offset > 0)."""
items = [{"m": {"id": i}} for i in range(10)]
map_type = pa.map_(pa.string(), pa.int64())
arrays = pa.array([row["m"] for row in items], type=map_type)
table = pa.Table.from_arrays([arrays], names=["m"])

# Force offsets by slicing the table before ingestion
sliced_table = table.slice(offset=7, length=3)
ds = ray.data.from_arrow(sliced_table)

result = ds.with_column("vals", col("m").map.values()).to_pandas()
result = result.drop(columns=["m"])

expected = pd.DataFrame({"vals": [[7], [8], [9]]})
assert rows_same(result, expected)

def test_map_nulls_and_empty(self, dataset_format):
"""Test handling of null maps and empty maps."""
items_data = [{"m": {"a": 1}}, {"m": {}}, {"m": None}]

map_type = pa.map_(pa.string(), pa.int64())
arrays = pa.array([row["m"] for row in items_data], type=map_type)
arrow_table = pa.Table.from_arrays([arrays], names=["m"])
ds = _create_dataset(items_data, dataset_format, arrow_table)

# Use take_all() to avoid pandas casting errors with mixed None/list types
rows = (
ds.with_column("keys", col("m").map.keys())
.with_column("values", col("m").map.values())
.take_all()
)

assert list(rows[0]["keys"]) == ["a"] and list(rows[0]["values"]) == [1]
assert len(rows[1]["keys"]) == 0 and len(rows[1]["values"]) == 0
assert rows[2]["keys"] is None and rows[2]["values"] is None

def test_map_chaining(self, dataset_format):
ds = _create_map_dataset(dataset_format)

# map.keys() returns a list, so .list.len() should apply
result = ds.with_column(
"num_keys", col("attrs").map.keys().list.len()
).to_pandas()
result = result.drop(columns=["attrs"])

expected = pd.DataFrame({"num_keys": [2, 1]})
assert rows_same(result, expected)


# ──────────────────────────────────────
# Datetime Namespace Tests
# ──────────────────────────────────────
Expand Down