
Daft v0.7.11 → v0.7.13: Bidirectional ASOF Joins, Arrow PyCapsule, and Iceberg Idempotent Commits
Three Daft releases cut May 12–15. Bidirectional streaming ASOF joins, Arrow PyCapsule, Iceberg idempotent commits and table properties, Spark month arithmetic.
by Daft TeamThis post covers Daft v0.7.11, v0.7.12, and v0.7.13 together — three releases cut between May 12 and May 15. If you're on v0.7.10, upgrade straight to v0.7.13.
Arrow PyCapsule Interface — Zero-Copy Across Libraries (v0.7.11)
Community contributor @aaron-ang implemented the Arrow PyCapsule Interface for Daft DataFrames (#6745). Moving DataFrames between Python libraries no longer pays serialization costs through the Python heap — libraries pass direct pointers using the C Data Interface standard.
# /// script
# description = "Zero-copy DataFrame exchange via Arrow PyCapsule"
# requires-python = ">=3.10,<=3.13"
# dependencies = ["daft==0.7.13", "pyarrow", "typing-extensions"]
# ///
import daft
import pyarrow as pa
df = daft.from_pydict({"id": [1, 2, 3], "value": [10.0, 20.0, 30.0]})
# Zero-copy to PyArrow via __arrow_c_stream__
arrow_table = pa.table(df)
# Zero-copy back to Daft
df_roundtrip = daft.from_arrow(arrow_table)
df_roundtrip.show()The implementation follows the Arrow PyCapsule specification, so anything that speaks the standard — PyArrow, Polars, cuDF — exchanges Daft data without serialization. For large datasets this eliminates gigabytes of unnecessary allocations.
Streaming ASOF Joins — Right Side Larger Than Memory (v0.7.11)
@euanlimzx added streaming execution for ASOF joins, so the right-side table no longer has to fit in memory (#6846).
# /// script
# description = "Streaming ASOF join — match each event to most recent price"
# requires-python = ">=3.12"
# dependencies = ["daft==0.7.13", "typing-extensions"]
# ///
import daft
events = daft.from_pydict({
"symbol": ["AAPL", "AAPL", "MSFT"],
"timestamp": ["2024-01-15 10:00:00", "2024-01-15 11:30:00", "2024-01-15 09:45:00"],
"qty": [100, 50, 75],
})
prices = daft.from_pydict({
"symbol": ["AAPL", "AAPL", "MSFT", "MSFT"],
"timestamp": ["2024-01-15 09:30:00", "2024-01-15 11:00:00", "2024-01-15 09:30:00", "2024-01-15 10:30:00"],
"price": [150.0, 152.5, 380.0, 382.0],
})
result = events.join_asof(prices, on="timestamp", by="symbol")
result.show()Previous ASOF implementations could OOM when right-side tables exceeded worker memory. The streaming probe processes the right side in parallel batches and maintains only the state needed for timestamp matching — parallelism is now batch count, not entity cardinality. This is the foundation that the Scaling ASOF Joins post walks through in detail.
Forward ASOF Joins — "What Happens Next?" (v0.7.13)
@euanlimzx shipped forward ASOF joins (#6918), completing the bidirectional matching capability alongside backward ASOF joins.
Backward ASOF joins match each left row against the latest right row where right.on_key <= left.on_key — the "what happened before this" question. Forward ASOF joins do the inverse: for each left row, find the earliest right row where right.on_key >= left.on_key within the same entity group. The "what happens next" question.
# /// script
# description = "Forward ASOF join — match each trade to next earnings event"
# requires-python = ">=3.12"
# dependencies = ["daft==0.7.13", "typing-extensions"]
# ///
import daft
trades = daft.from_pydict({
"symbol": ["AAPL", "AAPL", "MSFT"],
"trade_date": ["2024-01-15", "2024-02-20", "2024-01-10"],
})
earnings = daft.from_pydict({
"symbol": ["AAPL", "AAPL", "MSFT", "MSFT"],
"earnings_date": ["2024-02-01", "2024-05-02", "2024-01-30", "2024-04-25"],
})
# For each trade, find the next upcoming earnings event
result = trades.join_asof(
earnings,
left_on="trade_date",
right_on="earnings_date",
by="symbol",
strategy="forward",
)
result.show()The implementation mirrors backward ASOF with three swaps: binary search finds the ceiling instead of the floor, candidate selection prefers smallest timestamps over largest, and distributed execution back-propagates carryover state instead of forward-propagating it.
Iceberg Idempotent Commits — Retry-Safe Writes (v0.7.11)
@rohitkulshreshtha implemented idempotent Iceberg commits via snapshot markers (#6905, #6841). write_iceberg gains an optional checkpoint= kwarg taking a daft.IdempotentCommit that bundles a checkpoint store and a user-supplied idempotence_key. The key gets written into the snapshot summary as daft.idempotence-key; if the job retries with the same key, Daft walks the snapshot history, recognizes the prior commit, and skips the rewrite.
df.write_iceberg(
table,
checkpoint=daft.IdempotentCommit(store=ckpt, idempotence_key="run-1"),
)IdempotentCommit is a frozen dataclass exported from daft and daft.checkpoint. Keys prefixed daft.idempotence- in snapshot_properties are reserved — Daft raises ValueError so you can't silently land a non-idempotent write tagged with the marker, then later switch to checkpoint= and have recovery miss the prior write.
This eliminates one of the more common production data-duplication classes — partial commits after infra failure — without forcing every pipeline to bolt on its own dedupe pass.
Iceberg Table Properties Are Now Honored (v0.7.12)
@YuangGao wired Iceberg table-level write properties into the writer factory (#6912). write.target-file-size-bytes and write.parquet.row-group-size-bytes are now read from the table metadata, falling back to Daft's execution config only when unset.
Before this change, Daft's Iceberg writer used the global parquet_target_filesize config and ignored per-table settings. Tables tuned for frequent small updates or bulk analytics would get written with Daft's defaults instead of their own. The fallback chain is backwards-compatible — Daft's defaults already match the Iceberg spec defaults (512 MB target, 128 MB row groups).
Spark-Compatible Month Arithmetic (v0.7.12)
@BABTUNA added add_months() and months_between() (#6913), two more pieces of the Spark migration story tracked in #3798.
# /// script
# description = "End-of-month clamping and fractional months"
# requires-python = ">=3.12"
# dependencies = ["daft==0.7.13", "typing-extensions"]
# ///
import daft
from daft import DataType
from daft.functions import add_months, months_between
df = daft.from_pydict({"date": ["2023-01-31", "2024-01-31", "2023-03-31"]})
df = df.with_column("date", df["date"].cast(DataType.date()))
df.with_column("plus_one_month", add_months(df["date"], 1)).show()
# 2023-01-31 → 2023-02-28
# 2024-01-31 → 2024-02-29 (leap year)
# 2023-03-31 → 2023-04-30
df2 = daft.from_pydict({
"end_date": ["1997-02-28 10:30:00"],
"start_date": ["1996-10-30 00:00:00"],
})
df2 = df2.with_columns({
"end_date": df2["end_date"].cast(DataType.timestamp("us")),
"start_date": df2["start_date"].cast(DataType.timestamp("us")),
})
df2.with_column(
"months",
months_between(df2["end_date"], df2["start_date"]),
).show()
# 3.94959677 — matches Spark's documented example exactlyadd_months uses chrono::Months for correct calendar arithmetic; months_between casts to timestamps and applies Spark's 8-decimal rounding so financial reporting pipelines that already trust Spark's semantics can swap in Daft without re-validating.
Everything Else
- SQL credential leak closed (#6933, @madvart) —
daft.read_sql()error messages no longer echo the connection URL, so JWT tokens and access-token query params stop leaking in exceptions.SQLConnection.__repr__now prints a safe dialect summary. The v0.7.11 fix only redacteduser:passworduserinfo; this one removes the URL from error output entirely. - Extension macro revert (#6925, @rchowell) — reverted the scalar
daft_funcoverloading from #6844 that broke third-party extension registration in v0.7.11. This is why v0.7.12 exists. A longer-term fix using interior mutability with a scalar function factory is on the roadmap. - Dashboard failed-operator highlighting (#6930, @cckellogg) — failed operators in the query dashboard now correctly highlight in red, so distributed query failures are visible at a glance instead of buried in logs.
- Per-task progress + topology markers (#6861, #6897, #6896, @samstokes, @cckellogg) — task-level progress, query failure detail, and bounded retention in the dashboard.
- Scalar broadcast fix for
to_struct()(#6931, @euanlimzx) — type coercion when broadcasting a scalar into a struct now works. - Lance namespace cleanup (#6939, @rchowell) — removed hallucinated
lance-namespacecode that was causing build issues. - Map operations (#6875, @qingfeng-occ) —
map_keys()extracts keys from Map columns. - List operations (#6812, @mytsvy) —
list_flatten()flattens nested lists. - Float16 dtype (#6859, @universalmind303) — half-precision floats for ML workloads.
- File operations (#6847, #6842, @chenghuichen) — image media type support, byte-range reads.
- Session views (#6786, @BABTUNA) — non-materialized temp views for complex queries.
- Qdrant extension docs (#6921, @Anush008) — community Qdrant integration now documented at docs.daft.ai.
- Iceberg map-type test re-enabled (#6911, @YuangGao) — round-trip coverage restored after the underlying type-conversion fix.
- Parquet roundtrip fix for fixed-size list arrays — @TuodiAunty.
- SQL credential leak fix (#6933, @madvart) - Stops echoing the connection URL in error messages and in
SQLConnection.__repr__.
Community Contributions
- @aaron-ang — Arrow PyCapsule Interface
- @YuangGao — Iceberg table-level write properties, map-type test re-enablement
- @BABTUNA —
add_months(),months_between(), session views - @Anush008 — Qdrant extension documentation
- @qingfeng-occ —
map_keys() - @mytsvy —
list_flatten() - @TuodiAunty — Parquet roundtrip fix for fixed-size list arrays
- @chenghuichen — image media type, byte-range file reads
Upgrade
uv add "daft>=0.7.13"Or grab the latest nightly:
uv pip install daft --pre --extra-index-url https://nightly.daft.ai
